From 161d0a86cac97a57954e3e4905dffadeb9f27b9e Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:02:09 +0800 Subject: [PATCH 01/39] Initial version of craft binary mq message format Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 140 ++++++++++ cdc/sink/codec/craft.go | 301 ++++++++++++++++++++ cdc/sink/codec/craft_decoder.go | 475 ++++++++++++++++++++++++++++++++ cdc/sink/codec/craft_encoder.go | 286 +++++++++++++++++++ cdc/sink/codec/craft_model.go | 303 ++++++++++++++++++++ cdc/sink/codec/craft_test.go | 297 ++++++++++++++++++++ cdc/sink/codec/interface.go | 5 + cdc/sink/codec/json.go | 8 +- cdc/sink/codec/json_test.go | 54 +--- pkg/errors/errors.go | 2 + 10 files changed, 1816 insertions(+), 55 deletions(-) create mode 100644 cdc/sink/codec/codec_test.go create mode 100644 cdc/sink/codec/craft.go create mode 100644 cdc/sink/codec/craft_decoder.go create mode 100644 cdc/sink/codec/craft_encoder.go create mode 100644 cdc/sink/codec/craft_model.go create mode 100644 cdc/sink/codec/craft_test.go diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go new file mode 100644 index 00000000000..e9a80a0722d --- /dev/null +++ b/cdc/sink/codec/codec_test.go @@ -0,0 +1,140 @@ +// Copyright 2021 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "bytes" + "compress/zlib" + "encoding/json" + "fmt" + + "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var ( + codecRowCases = [][]*model.RowChangedEvent{{{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + }}, {{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + }, { + CommitTs: 2, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("bb")}}, + }, { + CommitTs: 3, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("bb")}}, + }, { + CommitTs: 4, + Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("cc")}}, + }}, {}} + + codecDDLCases = [][]*model.DDLEvent{{{ + CommitTs: 1, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table a", + Type: 1, + }}, {{ + CommitTs: 1, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table a", + Type: 1, + }, { + CommitTs: 2, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table b", + Type: 2, + }, { + CommitTs: 3, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table c", + Type: 3, + }}, {}} + + codecResolvedTSCases = [][]uint64{{1}, {1, 2, 3}, {}} +) + +func dumpJson(v interface{}) { + b, e := json.Marshal(v) + if e != nil { + panic(e) + } + fmt.Println(string(b)) +} + +var _ = check.Suite(&codecTestSuite{}) + +type codecTestSuite struct{} + +func (s *codecTestSuite) checkCompressedSize(messages []*MQMessage) (int, int) { + var buff bytes.Buffer + writer := zlib.NewWriter(&buff) + originalSize := 0 + for _, message := range messages { + originalSize += len(message.Key) + len(message.Value) + if len(message.Key) > 0 { + writer.Write(message.Key) + } + writer.Write(message.Value) + } + writer.Close() + return originalSize, buff.Len() +} + +func (s *codecTestSuite) encodeRowCase(c *check.C, encoder EventBatchEncoder, events []*model.RowChangedEvent) []*MQMessage { + err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + c.Assert(err, check.IsNil) + + for _, event := range events { + op, err := encoder.AppendRowChangedEvent(event) + c.Assert(err, check.IsNil) + c.Assert(op, check.Equals, EncoderNoOperation) + } + + if len(events) > 0 { + return encoder.Build() + } + return nil +} + +func (s *codecTestSuite) TestJsonVsCraft(c *check.C) { + defer testleak.AfterTest(c)() + fmt.Println("| index | craft size | json size | craft compressed | json compressed |") + fmt.Println("| ----- | ---------- | --------- | ---------------- | --------------- |") + for i, cs := range codecRowCases { + craftEncoder := NewCraftEventBatchEncoder() + jsonEncoder := NewJSONEventBatchEncoder() + craftMessages := s.encodeRowCase(c, craftEncoder, cs) + jsonMessages := s.encodeRowCase(c, jsonEncoder, cs) + craftOriginal, craftCompressed := s.checkCompressedSize(craftMessages) + jsonOriginal, jsonCompressed := s.checkCompressedSize(jsonMessages) + fmt.Printf("| %d | %d | %d | %d | %d |\n", i, craftOriginal, jsonOriginal, craftCompressed, jsonCompressed) + } +} diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go new file mode 100644 index 00000000000..5fb20ecb2f7 --- /dev/null +++ b/cdc/sink/codec/craft.go @@ -0,0 +1,301 @@ +package codec + +import ( + "math" + "strconv" + + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" +) + +// string/bytes array layout +// n bytes array of elements' size, format: uvarint array +// n bytes elements, format: bits +// +// varint/uvarint array layout +// n bytes elements. format: varint / uvarint +// +// delta varint/uvarint array layout +// n bytes base number +// n bytes offsets. format: varint/uvarint +// +// string/bytes layout +// n bytes varint length +// n bytes payload +// +// float layout, standard protobuf float +// double layout, standard protobuf double +// varint layout, standard protobuf varint +// uvarint layout, standard protobuf uvarint +// +// Message layout +// 2 bytes version +// 2 bytes number of pairs +// n bytes keys +// n bytes values +// n bytes size tables +// +// Keys layout +// n bytes array of commit ts, format: delta uvarint array +// n bytes array of type, format: uvarint array +// n bytes array of row id, format: uvarint array +// n bytes array of partition id, format: varint array, -1 means field is not set +// n bytes array of schema, format: string array +// n bytes array of table, format: string array +// +// Row changed layout +// n bytes multiple column groups +// +// Column group layout +// 1 byte column group type: 1 New Values, 2: Old Values, 3: Delete Values +// n bytes number of columns, format: uvarint +// n bytes array of name, format: string array +// n bytes array of type, format: uvarint array +// n bytes array of flag, format: uvarint array +// n bytes array of value, format: nullable bytes array +// +// DDL layout +// n bytes type, format: uvarint +// n bytes query, format: string +// +// Size tables layout +// n bytes table to store size of serialized keys +// n bytes table to store size of values +// n bytes tables to store of serialized column groups +// n bytes size of serialized size tables, format: reversed uvarint +// +// Size table layout +// n bytes number of elements, format: uvarint +// n bytes repeated elements, format: uvarint +// +const ( + // CraftVersion1 represents the version of craft format + CraftVersion1 uint64 = 1 + + // craftMaxMessageBytes sets the default value for max-message-bytes + craftMaxMessageBytes int = 64 * 1024 * 1024 // 64M + // craftMaxBatchSize sets the default value for max-batch-size + craftMaxBatchSize int = 4096 + // default buffer size + craftDefaultBufferCapacity = 512 + + // Column group types + craftColumnGroupTypeDelete = 0x3 + craftColumnGroupTypeOld = 0x2 + craftColumnGroupTypeNew = 0x1 + + // Size tables index + craftKeySizeTableIndex = 0 + craftValueSizeTableIndex = 1 + craftColumnGroupSizeTableStartIndex = 2 +) + +type CraftEventBatchEncoder struct { + rowChangedBuffer *craftRowChangedEventBuffer + messageBuf []*MQMessage + + // configs + maxMessageSize int + maxBatchSize int +} + +// EncodeCheckpointEvent implements the EventBatchEncoder interface +func (e *CraftEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { + return newResolvedMQMessage(ProtocolCraft, nil, newCraftResolvedEventEncoder(ts).encode(), ts), nil +} + +func (e *CraftEventBatchEncoder) flush() { + keys := e.rowChangedBuffer.getKeys() + ts := keys.getTs(0) + schema := keys.getSchema(0) + table := keys.getTable(0) + e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.encode(), ts, model.MqMessageTypeRow, &schema, &table)) +} + +func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent) (EncoderResult, error) { + rows, size := e.rowChangedBuffer.appendRowChangedEvent(ev) + if size > e.maxMessageSize || rows >= e.maxBatchSize { + e.flush() + } + return EncoderNoOperation, nil +} + +// AppendResolvedEvent is no-op +func (e *CraftEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { + return EncoderNoOperation, nil +} + +func (e *CraftEventBatchEncoder) EncodeDDLEvent(ev *model.DDLEvent) (*MQMessage, error) { + return newDDLMQMessage(ProtocolCraft, nil, newCraftDDLEventEncoder(ev).encode(), ev), nil +} + +func (e *CraftEventBatchEncoder) Build() []*MQMessage { + if e.rowChangedBuffer.size() > 0 { + // flush buffered data to message buffer + e.flush() + } + ret := e.messageBuf + e.messageBuf = make([]*MQMessage, 0) + return ret +} + +func (e *CraftEventBatchEncoder) MixedBuild(withVersion bool) []byte { + panic("Only JsonEncoder supports mixed build") +} + +func (e *CraftEventBatchEncoder) Size() int { + return e.rowChangedBuffer.size() +} + +func (e *CraftEventBatchEncoder) Reset() { + e.rowChangedBuffer.reset() +} + +func (e *CraftEventBatchEncoder) SetParams(params map[string]string) error { + var err error + if maxMessageBytes, ok := params["max-message-bytes"]; ok { + e.maxMessageSize, err = strconv.Atoi(maxMessageBytes) + if err != nil { + return cerror.ErrSinkInvalidConfig.Wrap(err) + } + } else { + e.maxMessageSize = DefaultMaxMessageBytes + } + + if e.maxMessageSize <= 0 || e.maxMessageSize > math.MaxInt32 { + return cerror.ErrSinkInvalidConfig.Wrap(errors.Errorf("invalid max-message-bytes %d", e.maxMessageSize)) + } + + if maxBatchSize, ok := params["max-batch-size"]; ok { + e.maxBatchSize, err = strconv.Atoi(maxBatchSize) + if err != nil { + return cerror.ErrSinkInvalidConfig.Wrap(err) + } + } else { + e.maxBatchSize = DefaultMaxBatchSize + } + + if e.maxBatchSize <= 0 || e.maxBatchSize > math.MaxUint16 { + return cerror.ErrSinkInvalidConfig.Wrap(errors.Errorf("invalid max-batch-size %d", e.maxBatchSize)) + } + return nil +} + +func NewCraftEventBatchEncoder() EventBatchEncoder { + return &CraftEventBatchEncoder{ + rowChangedBuffer: &craftRowChangedEventBuffer{ + keys: &craftColumnarKeys{}, + }, + } +} + +// CraftEventBatchDecoder decodes the byte of a batch into the original messages. +type CraftEventBatchDecoder struct { + keys *craftColumnarKeys + decoder *craftMessageDecoder + index int +} + +// HasNext implements the EventBatchDecoder interface +func (b *CraftEventBatchDecoder) HasNext() (model.MqMessageType, bool, error) { + if b.index >= b.keys.count { + return model.MqMessageTypeUnknown, false, nil + } + return b.keys.getType(b.index), true, nil +} + +// NextResolvedEvent implements the EventBatchDecoder interface +func (b *CraftEventBatchDecoder) NextResolvedEvent() (uint64, error) { + ty, hasNext, err := b.HasNext() + if err != nil { + return 0, errors.Trace(err) + } + if !hasNext || ty != model.MqMessageTypeResolved { + return 0, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") + } + ts := b.keys.getTs(b.index) + b.index++ + return ts, nil +} + +// NextRowChangedEvent implements the EventBatchDecoder interface +func (b *CraftEventBatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) { + ty, hasNext, err := b.HasNext() + if err != nil { + return nil, errors.Trace(err) + } + if !hasNext || ty != model.MqMessageTypeRow { + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") + } + old, new, err := b.decoder.decodeRowChangedEvent(b.index) + if err != nil { + return nil, errors.Trace(err) + } + ev := &model.RowChangedEvent{} + if old != nil { + if ev.PreColumns, err = old.toModel(); err != nil { + return nil, errors.Trace(err) + } + } + if new != nil { + if ev.Columns, err = new.toModel(); err != nil { + return nil, errors.Trace(err) + } + } + ev.CommitTs = b.keys.getTs(b.index) + ev.Table = &model.TableName{ + Schema: b.keys.getSchema(b.index), + Table: b.keys.getTable(b.index), + } + partition := b.keys.getPartition(b.index) + if partition >= 0 { + ev.Table.TableID = partition + ev.Table.IsPartition = true + } + b.index++ + return ev, nil +} + +// NextDDLEvent implements the EventBatchDecoder interface +func (b *CraftEventBatchDecoder) NextDDLEvent() (*model.DDLEvent, error) { + ty, hasNext, err := b.HasNext() + if err != nil { + return nil, errors.Trace(err) + } + if !hasNext || ty != model.MqMessageTypeDDL { + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") + } + ddlType, query, err := b.decoder.decodeDDLEvent(b.index) + if err != nil { + return nil, errors.Trace(err) + } + event := &model.DDLEvent{ + CommitTs: b.keys.getTs(b.index), + Query: query, + Type: ddlType, + TableInfo: &model.SimpleTableInfo{ + Schema: b.keys.getSchema(b.index), + Table: b.keys.getTable(b.index), + }, + } + b.index++ + return event, nil +} + +// NewCraftEventBatchDecoder creates a new CraftEventBatchDecoder. +func NewCraftEventBatchDecoder(bits []byte) (EventBatchDecoder, error) { + decoder, err := newCraftMessageDecoder(bits) + if err != nil { + return nil, errors.Trace(err) + } + keys, err := decoder.decodeKeys() + if err != nil { + return nil, errors.Trace(err) + } + + return &CraftEventBatchDecoder{ + keys: keys, + decoder: decoder, + }, nil +} diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go new file mode 100644 index 00000000000..71c8083d48f --- /dev/null +++ b/cdc/sink/codec/craft_decoder.go @@ -0,0 +1,475 @@ +package codec + +import ( + "encoding/binary" + "math" + + "github.com/pingcap/errors" + pmodel "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" +) + +/// Primitive type decoders +func decodeUint8(bits []byte) ([]byte, byte, error) { + if len(bits) < 1 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("buffer underflow") + } + return bits[1:], bits[0], nil +} + +func decodeVarint(bits []byte) ([]byte, int64, error) { + x, rd := binary.Varint(bits) + if rd < 0 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid varint data") + } + return bits[rd:], x, nil +} + +func decodeUvarint(bits []byte) ([]byte, uint64, error) { + x, rd := binary.Uvarint(bits) + if rd < 0 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid uvarint data") + } + return bits[rd:], x, nil +} + +func decodeUvarintReversed(bits []byte) (int, uint64, error) { + l := len(bits) - 1 + var x uint64 + var s uint + i := 0 + for l >= 0 { + b := bits[l] + if b < 0x80 { + if i >= binary.MaxVarintLen64 || i == binary.MaxVarintLen64-1 && b > 1 { + return 0, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid reversed uvarint data") + } + return i + 1, x | uint64(b)< math.MaxInt32 { + return 0, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + } + return nb, int(x), err +} + +func decodeUvarint32(bits []byte) ([]byte, int32, error) { + newBits, x, err := decodeUvarint(bits) + if err != nil { + return bits, 0, errors.Trace(err) + } + if x > math.MaxInt32 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + } + return newBits, int32(x), nil +} + +func decodeVarint32(bits []byte) ([]byte, int32, error) { + newBits, x, err := decodeVarint(bits) + if err != nil { + return bits, 0, errors.Trace(err) + } + if x > math.MaxInt32 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + } + return newBits, int32(x), nil +} + +func decodeUvarintLength(bits []byte) ([]byte, int, error) { + bits, x, err := decodeUvarint32(bits) + return bits, int(x), err +} + +func decodeVarintLength(bits []byte) ([]byte, int, error) { + bits, x, err := decodeVarint32(bits) + return bits, int(x), err +} + +func decodeFloat64(bits []byte) ([]byte, float64, error) { + if len(bits) < 8 { + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("buffer underflow") + } + x := binary.BigEndian.Uint64(bits) + return bits[8:], math.Float64frombits(x), nil +} + +func decodeBytes(bits []byte) ([]byte, []byte, error) { + newBits, l, err := decodeUvarintLength(bits) + if err != nil { + return bits, nil, errors.Trace(err) + } + if len(newBits) < l { + return bits, nil, cerror.ErrCraftCodecInvalidData.GenWithStack("buffer underflow") + } + return newBits[l:], newBits[:l], nil +} + +func decodeString(bits []byte) ([]byte, string, error) { + bits, bytes, err := decodeBytes(bits) + if err == nil { + return bits, string(bytes), nil + } else { + return bits, "", errors.Trace(err) + } +} + +/// Chunk decoders +func decodeStringChunk(bits []byte, size int) ([]byte, []string, error) { + newBits, data, err := decodeBytesChunk(bits, size) + if err != nil { + return bits, nil, errors.Trace(err) + } + result := make([]string, size) + for i, d := range data { + result[i] = string(d) + } + return newBits, result, nil +} + +func decodeNullableStringChunk(bits []byte, size int) ([]byte, []*string, error) { + newBits, data, err := decodeNullableBytesChunk(bits, size) + if err != nil { + return bits, nil, errors.Trace(err) + } + result := make([]*string, size) + for i, d := range data { + if d != nil { + s := string(d) + result[i] = &s + } + } + return newBits, result, nil +} + +func decodeBytesChunk(bits []byte, size int) ([]byte, [][]byte, error) { + return doDecodeBytesChunk(bits, size, decodeUvarintLength) +} + +func doDecodeBytesChunk(bits []byte, size int, lengthDecoder func([]byte) ([]byte, int, error)) ([]byte, [][]byte, error) { + larray := make([]int, size) + newBits := bits + var bl int + var err error + for i := 0; i < size; i++ { + newBits, bl, err = lengthDecoder(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + larray[i] = bl + } + + data := make([][]byte, size) + for i := 0; i < size; i++ { + if larray[i] != -1 { + data[i] = newBits[:larray[i]] + newBits = newBits[larray[i]:] + } + } + return newBits, data, nil +} + +func decodeNullableBytesChunk(bits []byte, size int) ([]byte, [][]byte, error) { + return doDecodeBytesChunk(bits, size, decodeVarintLength) +} + +func decodeVarintChunk(bits []byte, size int) ([]byte, []int64, error) { + array := make([]int64, size) + newBits := bits + var i64 int64 + var err error + for i := 0; i < size; i++ { + newBits, i64, err = decodeVarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + array[i] = i64 + } + return newBits, array, nil +} + +func decodeUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { + array := make([]uint64, size) + newBits := bits + var u64 uint64 + var err error + for i := 0; i < size; i++ { + newBits, u64, err = decodeUvarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + array[i] = u64 + } + return newBits, array, nil +} + +func decodeDeltaVarintChunk(bits []byte, size int) ([]byte, []int64, error) { + array := make([]int64, size) + newBits := bits + var err error + newBits, array[0], err = decodeVarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + for i := 1; i < size; i++ { + newBits, array[i], err = decodeVarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + array[i] = array[i-1] + array[i] + } + return newBits, array, nil +} + +func decodeDeltaUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { + array := make([]uint64, size) + newBits := bits + var err error + newBits, array[0], err = decodeUvarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + for i := 1; i < size; i++ { + newBits, array[i], err = decodeUvarint(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + array[i] = array[i-1] + array[i] + } + return newBits, array, nil +} + +// size tables are always at end of serialized data, there is no unread bytes to return +func decodeSizeTables(bits []byte) (int, [][]uint64, error) { + nb, size, _ := decodeUvarintReversedLength(bits) + sizeOffset := len(bits) - nb + tablesOffset := sizeOffset - size + tables := bits[tablesOffset:sizeOffset] + + tableSize := size + nb + var err error + var table []uint64 + result := make([][]uint64, 0, 1) + for len(tables) > 0 { + tables, size, err = decodeUvarintLength(tables) + if err != nil { + return 0, nil, errors.Trace(err) + } + tables, table, err = decodeDeltaUvarintChunk(tables, size) + if err != nil { + return 0, nil, errors.Trace(err) + } + result = append(result, table) + } + + return tableSize, result, nil +} + +/// TiDB types decoder +func decodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{}, error) { + if bits == nil { + return nil, nil + } + switch ty { + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: + // value type for these mysql types are string + return string(bits), nil + case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: + // value type for thest mysql types are uint64 + _, u64, err := decodeUvarint(bits) + return u64, err + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, + mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + // value type for these mysql types are []byte + return bits, nil + case mysql.TypeFloat, mysql.TypeDouble: + // value type for these mysql types are float64 + _, f64, err := decodeFloat64(bits) + return f64, err + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: + // value type for these mysql types are int64 or uint64 depends on flags + if flag.IsUnsigned() { + _, u64, err := decodeUvarint(bits) + return u64, err + } else { + _, i64, err := decodeUvarint(bits) + return i64, err + } + case mysql.TypeUnspecified: + fallthrough + case mysql.TypeNull: + fallthrough + case mysql.TypeGeometry: + return nil, nil + } + return nil, nil +} + +// Message decoder +type craftMessageDecoder struct { + bits []byte + sizeTables [][]uint64 + valuesSizeTable []uint64 +} + +func newCraftMessageDecoder(bits []byte) (*craftMessageDecoder, error) { + bits, version, err := decodeUvarint(bits) + if err != nil { + return nil, errors.Trace(err) + } + if version < CraftVersion1 { + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("unexpected craft version") + } + sizeTablesSize, sizeTables, err := decodeSizeTables(bits) + if err != nil { + return nil, errors.Trace(err) + } + return &craftMessageDecoder{ + bits: bits[:len(bits)-sizeTablesSize], + sizeTables: sizeTables, + valuesSizeTable: sizeTables[craftValueSizeTableIndex], + }, nil +} + +func (d *craftMessageDecoder) decodeKeys() (*craftColumnarKeys, error) { + var pairs, keysSize int + var err error + d.bits, pairs, err = decodeUvarintLength(d.bits) + if err != nil { + return nil, errors.Trace(err) + } + keysSize = int(d.sizeTables[craftKeySizeTableIndex][0]) + var keys *craftColumnarKeys + keys, err = decodeCraftColumnarKeys(d.bits[:keysSize], pairs) + if err != nil { + return nil, errors.Trace(err) + } + // skip keys + d.bits = d.bits[keysSize:] + return keys, nil +} + +func (d *craftMessageDecoder) valueBits(index int) []byte { + start := 0 + if index > 0 { + start = int(d.valuesSizeTable[index-1]) + } + return d.bits[start:int(d.valuesSizeTable[index])] +} + +func (d *craftMessageDecoder) decodeDDLEvent(index int) (pmodel.ActionType, string, error) { + bits, ty, err := decodeUvarint(d.valueBits(index)) + if err != nil { + return pmodel.ActionNone, "", errors.Trace(err) + } + _, query, err := decodeString(bits) + return pmodel.ActionType(ty), query, err +} + +func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, error) { + var numColumns int + bits, ty, err := decodeUint8(bits) + if err != nil { + return nil, errors.Trace(err) + } + bits, numColumns, err = decodeUvarintLength(bits) + if err != nil { + return nil, errors.Trace(err) + } + var names, values [][]byte + var types, flags []uint64 + bits, names, err = decodeBytesChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + bits, types, err = decodeUvarintChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + bits, flags, err = decodeUvarintChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + bits, values, err = decodeBytesChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + return &craftColumnarColumnGroup{ + ty: ty, + names: names, + types: types, + flags: flags, + values: values, + }, nil +} + +func (d *craftMessageDecoder) decodeRowChangedEvent(index int) (preColumns, columns *craftColumnarColumnGroup, err error) { + bits := d.valueBits(index) + columnGroupSizeTable := d.sizeTables[craftValueSizeTableIndex+index] + columnGroupIndex := 0 + for len(bits) > 0 { + columnGroupSize := columnGroupSizeTable[columnGroupIndex] + columnGroup, err := decodeCraftColumnarColumnGroup(bits[:columnGroupSize]) + bits = bits[columnGroupSize:] + columnGroupIndex++ + if err != nil { + return nil, nil, errors.Trace(err) + } + switch columnGroup.ty { + case craftColumnGroupTypeDelete: + fallthrough + case craftColumnGroupTypeOld: + preColumns = columnGroup + case craftColumnGroupTypeNew: + columns = columnGroup + } + } + return preColumns, columns, nil +} + +// Events decoder +func decodeRowChangedEvent(keys *craftColumnarKeys, keyIndex int, sizeTables [][]uint64, bits []byte) (*model.RowChangedEvent, error) { + ev := &model.RowChangedEvent{} + ev.CommitTs = keys.getTs(keyIndex) + ev.Table = &model.TableName{ + Schema: keys.getSchema(keyIndex), + Table: keys.getTable(keyIndex), + } + partition := keys.getPartition(keyIndex) + if partition >= 0 { + ev.Table.TableID = partition + ev.Table.IsPartition = true + } + var columnGroup *craftColumnarColumnGroup + var err error + columnGroupSizeTable := sizeTables[keyIndex] + columnGroupIndex := 0 + for len(bits) > 0 { + columnGroupSize := columnGroupSizeTable[columnGroupIndex] + columnGroup, err = decodeCraftColumnarColumnGroup(bits[:columnGroupSize]) + bits = bits[columnGroupSize:] + columnGroupIndex++ + if err != nil { + return nil, errors.Trace(err) + } + switch columnGroup.ty { + case craftColumnGroupTypeDelete: + fallthrough + case craftColumnGroupTypeOld: + ev.PreColumns, err = columnGroup.toModel() + case craftColumnGroupTypeNew: + ev.Columns, err = columnGroup.toModel() + } + } + return ev, nil +} diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft_encoder.go new file mode 100644 index 00000000000..e1a376ac084 --- /dev/null +++ b/cdc/sink/codec/craft_encoder.go @@ -0,0 +1,286 @@ +package codec + +import ( + "encoding/binary" + "math" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" +) + +/// Primitive type encoders +func encodeFloat64(bits []byte, data float64) []byte { + buf := make([]byte, 8) + binary.BigEndian.PutUint64(buf, math.Float64bits(data)) + return buf +} + +func encodeVarint(bits []byte, data int64) []byte { + buf := make([]byte, binary.MaxVarintLen64) + l := binary.PutVarint(buf, data) + buf = buf[:l] + if bits == nil { + return buf + } else { + return append(bits, buf...) + } +} + +func encodeUvarint(bits []byte, data uint64) []byte { + buf := make([]byte, binary.MaxVarintLen64) + l := binary.PutUvarint(buf, data) + buf = buf[:l] + if bits == nil { + return buf + } else { + return append(bits, buf...) + } +} + +func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { + buf := make([]byte, binary.MaxVarintLen64) + i := 0 + for data >= 0x80 { + buf[i] = byte(data) | 0x80 + data >>= 7 + i++ + } + buf[i] = byte(data) + for bi := i; bi >= 0; bi-- { + bits = append(bits, buf[bi]) + } + return bits, i + 1 +} + +func encodeBytes(bits []byte, data []byte) []byte { + l := len(data) + if bits == nil { + bits = make([]byte, 0, binary.MaxVarintLen64+len(data)) + } + bits = encodeUvarint(bits, uint64(l)) + return append(bits, data...) +} + +func encodeString(bits []byte, data string) []byte { + return encodeBytes(bits, []byte(data)) +} + +/// Chunk encoders +func encodeStringChunk(bits []byte, data []string) []byte { + for _, s := range data { + bits = encodeUvarint(bits, uint64(len(s))) + } + for _, s := range data { + bits = append(bits, []byte(s)...) + } + return bits +} + +func encodeNullableStringChunk(bits []byte, data []*string) []byte { + for _, s := range data { + var l int64 = -1 + if s != nil { + l = int64(len(*s)) + } + bits = encodeVarint(bits, l) + } + for _, s := range data { + if s != nil { + bits = append(bits, []byte(*s)...) + } + } + return bits +} + +func encodeBytesChunk(bits []byte, data [][]byte) []byte { + for _, b := range data { + bits = encodeUvarint(bits, uint64(len(b))) + } + for _, b := range data { + bits = append(bits, b...) + } + return bits +} + +func encodeNullableBytesChunk(bits []byte, data [][]byte) []byte { + for _, b := range data { + var l int64 = -1 + if b != nil { + l = int64(len(b)) + } + bits = encodeVarint(bits, l) + } + for _, b := range data { + if b != nil { + bits = append(bits, b...) + } + } + return bits +} + +func encodeVarintChunk(bits []byte, data []int64) []byte { + for _, v := range data { + bits = encodeVarint(bits, v) + } + return bits +} + +func encodeUvarintChunk(bits []byte, data []uint64) []byte { + for _, v := range data { + bits = encodeUvarint(bits, v) + } + return bits +} + +func encodeDeltaVarintChunk(bits []byte, data []int64) []byte { + last := data[0] + bits = encodeVarint(bits, last) + for _, v := range data[1:] { + bits = encodeVarint(bits, v-last) + last = v + } + return bits +} + +func encodeDeltaUvarintChunk(bits []byte, data []uint64) []byte { + last := data[0] + bits = encodeUvarint(bits, last) + for _, v := range data[1:] { + bits = encodeUvarint(bits, v-last) + last = v + } + return bits +} + +func encodeSizeTables(bits []byte, tables [][]uint64) []byte { + size := len(bits) + for _, table := range tables { + bits = encodeUvarint(bits, uint64(len(table))) + bits = encodeDeltaUvarintChunk(bits, table) + } + bits, _ = encodeUvarintReversed(bits, uint64(len(bits)-size)) + return bits +} + +/// TiDB types encoder +func encodeTiDBType(ty byte, flag model.ColumnFlagType, value interface{}) []byte { + if value == nil { + return nil + } + switch ty { + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: + // value type for these mysql types are string + return []byte(value.(string)) + case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: + // value type for thest mysql types are uint64 + return encodeUvarint(nil, value.(uint64)) + case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, + mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + // value type for these mysql types are []byte + return value.([]byte) + case mysql.TypeFloat, mysql.TypeDouble: + // value type for these mysql types are float64 + return encodeFloat64(nil, value.(float64)) + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: + // value type for these mysql types are int64 or uint64 depends on flags + if flag.IsUnsigned() { + return encodeUvarint(nil, value.(uint64)) + } else { + return encodeVarint(nil, value.(int64)) + } + case mysql.TypeUnspecified: + fallthrough + case mysql.TypeNull: + fallthrough + case mysql.TypeGeometry: + return nil + } + return nil +} + +/// Message encoder +type craftMessageEncoder struct { + bits []byte + sizeTables [][]uint64 + valuesStartOffset int + valuesSizes []uint64 + valuesSizesIndex int +} + +func newCraftMessageEncoder() *craftMessageEncoder { + return &craftMessageEncoder{ + bits: encodeUvarint(make([]byte, 0, craftDefaultBufferCapacity), CraftVersion1), + } +} + +func (e *craftMessageEncoder) encodeValueSize() *craftMessageEncoder { + e.valuesSizes[e.valuesSizesIndex] = uint64(len(e.bits) - e.valuesStartOffset) + e.valuesSizesIndex++ + return e +} + +func (e *craftMessageEncoder) encodeUvarint(u64 uint64) *craftMessageEncoder { + e.bits = encodeUvarint(e.bits, u64) + return e +} + +func (e *craftMessageEncoder) encodeString(s string) *craftMessageEncoder { + e.bits = encodeString(e.bits, s) + return e +} + +func (e *craftMessageEncoder) encodeKeys(keys *craftColumnarKeys) *craftMessageEncoder { + e.bits = encodeUvarint(e.bits, uint64(keys.count)) + oldSize := len(e.bits) + e.valuesSizes = make([]uint64, keys.count) + e.bits = keys.encode(e.bits) + e.valuesStartOffset = len(e.bits) + e.sizeTables = append(e.sizeTables, []uint64{uint64(len(e.bits) - oldSize)}, e.valuesSizes) + return e +} + +func (e *craftMessageEncoder) encode() []byte { + return encodeSizeTables(e.bits, e.sizeTables) +} + +func (e *craftMessageEncoder) encodeRowChangeEvents(events []craftRowChangedEvent) *craftMessageEncoder { + sizeTables := e.sizeTables + for _, event := range events { + columnGroupSizeTable := make([]uint64, len(event)) + for gi, group := range event { + oldSize := len(e.bits) + e.bits = group.encode(e.bits) + columnGroupSizeTable[gi] = uint64(len(e.bits) - oldSize) + } + sizeTables = append(sizeTables, columnGroupSizeTable) + e.encodeValueSize() + } + e.sizeTables = sizeTables + return e +} + +func newCraftResolvedEventEncoder(ts uint64) *craftMessageEncoder { + return newCraftMessageEncoder().encodeKeys(&craftColumnarKeys{ + ts: []uint64{uint64(ts)}, + ty: []uint64{uint64(model.MqMessageTypeResolved)}, + rowID: []int64{int64(-1)}, + partition: []int64{int64(-1)}, + schema: [][]byte{nil}, + table: [][]byte{nil}, + count: 1, + }).encodeValueSize() +} + +func newCraftDDLEventEncoder(ev *model.DDLEvent) *craftMessageEncoder { + ty := uint64(ev.Type) + query := ev.Query + return newCraftMessageEncoder().encodeKeys(&craftColumnarKeys{ + ts: []uint64{uint64(ev.CommitTs)}, + ty: []uint64{uint64(model.MqMessageTypeDDL)}, + rowID: []int64{int64(-1)}, + partition: []int64{int64(-1)}, + schema: [][]byte{[]byte(ev.TableInfo.Schema)}, + table: [][]byte{[]byte(ev.TableInfo.Table)}, + count: 1, + }).encodeUvarint(ty).encodeString(query).encodeValueSize() +} diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go new file mode 100644 index 00000000000..50e48f9832b --- /dev/null +++ b/cdc/sink/codec/craft_model.go @@ -0,0 +1,303 @@ +package codec + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" +) + +/// Utility functions for buffer allocation +func newBufferSize(oldSize int) int { + var newSize int + if oldSize > 128 { + newSize = oldSize + 128 + } else { + if oldSize > 0 { + newSize = oldSize * 2 + } else { + newSize = 8 + } + } + return newSize +} + +func newUint64Buffers(eachSize int) ([]uint64, []uint64) { + buffer := make([]uint64, eachSize*2) + return buffer[:eachSize], buffer[eachSize:] +} + +func resizeUint64Buffer(buffer1, buffer2 []uint64) ([]uint64, []uint64) { + newBuffer1, newBuffer2 := newUint64Buffers(newBufferSize(len(buffer1))) + copy(newBuffer1, buffer1) + copy(newBuffer2, buffer2) + return newBuffer1, newBuffer2 +} + +func newInt64Buffers(eachSize int) ([]int64, []int64) { + buffer := make([]int64, eachSize*2) + return buffer[:eachSize], buffer[eachSize:] +} + +func resizeInt64Buffer(buffer1, buffer2 []int64) ([]int64, []int64) { + newBuffer1, newBuffer2 := newInt64Buffers(newBufferSize(len(buffer1))) + copy(newBuffer1, buffer1) + copy(newBuffer2, buffer2) + return newBuffer1, newBuffer2 +} + +func newBytesBuffers(eachSize int) ([][]byte, [][]byte) { + buffer := make([][]byte, eachSize*2) + return buffer[:eachSize], buffer[eachSize:] +} + +func resizeBytesBuffer(buffer1, buffer2 [][]byte) ([][]byte, [][]byte) { + newBuffer1, newBuffer2 := newBytesBuffers(newBufferSize(len(buffer1))) + copy(newBuffer1, buffer1) + copy(newBuffer2, buffer2) + return newBuffer1, newBuffer2 +} + +/// Keys in columnar layout +type craftColumnarKeys struct { + ts []uint64 + ty []uint64 + rowID []int64 + partition []int64 + schema [][]byte + table [][]byte + + count int +} + +func (k *craftColumnarKeys) encode(bits []byte) []byte { + bits = encodeDeltaUvarintChunk(bits, k.ts[:k.count]) + bits = encodeDeltaUvarintChunk(bits, k.ty[:k.count]) + bits = encodeDeltaVarintChunk(bits, k.rowID[:k.count]) + bits = encodeDeltaVarintChunk(bits, k.partition[:k.count]) + bits = encodeNullableBytesChunk(bits, k.schema[:k.count]) + bits = encodeNullableBytesChunk(bits, k.table[:k.count]) + return bits +} + +func (k *craftColumnarKeys) appendKey(ts, ty uint64, rowID, partition int64, schema, table []byte) int { + idx := k.count + if idx+1 > len(k.ty) { + k.ts, k.ty = resizeUint64Buffer(k.ts, k.ty) + k.rowID, k.partition = resizeInt64Buffer(k.rowID, k.partition) + k.schema, k.table = resizeBytesBuffer(k.schema, k.table) + } + k.ts[idx] = ts + k.ty[idx] = ty + k.rowID[idx] = rowID + k.partition[idx] = partition + k.schema[idx] = schema + k.table[idx] = table + k.count++ + + return 32 + len(schema) + len(table) /* 4 64-bits integers and two bytes array */ +} + +func (k *craftColumnarKeys) reset() { + k.count = 0 +} + +func (k *craftColumnarKeys) getType(index int) model.MqMessageType { + return model.MqMessageType(k.ty[index]) +} + +func (k *craftColumnarKeys) getTs(index int) uint64 { + return k.ts[index] +} + +func (k *craftColumnarKeys) getPartition(index int) int64 { + return k.partition[index] +} + +func (k *craftColumnarKeys) getSchema(index int) string { + return string(k.schema[index]) +} + +func (k *craftColumnarKeys) getTable(index int) string { + return string(k.table[index]) +} + +func decodeCraftColumnarKeys(bits []byte, numKeys int) (*craftColumnarKeys, error) { + var ts, ty []uint64 + var rowID, partition []int64 + var schema, table [][]byte + var err error + if bits, ts, err = decodeDeltaUvarintChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + if bits, ty, err = decodeDeltaUvarintChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + if bits, rowID, err = decodeDeltaVarintChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + if bits, partition, err = decodeDeltaVarintChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + if bits, schema, err = decodeNullableBytesChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + if bits, table, err = decodeNullableBytesChunk(bits, numKeys); err != nil { + return nil, errors.Trace(err) + } + return &craftColumnarKeys{ + ts: ts, + ty: ty, + rowID: rowID, + partition: partition, + schema: schema, + table: table, + count: numKeys, + }, nil +} + +/// Column group in columnar layout +type craftColumnarColumnGroup struct { + ty byte + names [][]byte + types []uint64 + flags []uint64 + values [][]byte +} + +func (g *craftColumnarColumnGroup) encode(bits []byte) []byte { + bits = append(bits, g.ty) + bits = encodeUvarint(bits, uint64(len(g.names))) + bits = encodeBytesChunk(bits, g.names) + bits = encodeUvarintChunk(bits, g.types) + bits = encodeUvarintChunk(bits, g.flags) + bits = encodeBytesChunk(bits, g.values) + return bits +} + +func (g *craftColumnarColumnGroup) toModel() ([]*model.Column, error) { + columns := make([]*model.Column, len(g.names)) + for i, name := range g.names { + ty := byte(g.types[i]) + flag := model.ColumnFlagType(g.flags[i]) + value, err := decodeTiDBType(ty, flag, g.values[i]) + if err != nil { + return nil, errors.Trace(err) + } + columns[i] = &model.Column{ + Name: string(name), + Type: ty, + Flag: flag, + Value: value, + } + } + return columns, nil +} + +func newCraftColumnarColumnGroup(ty byte, columns []*model.Column) (int, *craftColumnarColumnGroup) { + var names [][]byte + var values [][]byte + var types []uint64 + var flags []uint64 + estimatedSize := 0 + for _, col := range columns { + if col == nil { + continue + } + name := []byte(col.Name) + names = append(names, name) + types = append(types, uint64(col.Type)) + flags = append(flags, uint64(col.Flag)) + value := encodeTiDBType(col.Type, col.Flag, col.Value) + values = append(values, value) + estimatedSize += len(name) + len(value) + 8 /* two bytes array and two 64-bits integers */ + } + if len(names) > 0 { + return estimatedSize, &craftColumnarColumnGroup{ + ty: ty, + names: names, + types: types, + flags: flags, + values: values, + } + } + return estimatedSize, nil +} + +/// Row changed message is basically an array of column groups +type craftRowChangedEvent = []*craftColumnarColumnGroup + +func newCraftRowChangedMessage(ev *model.RowChangedEvent) (int, craftRowChangedEvent) { + var groups []*craftColumnarColumnGroup + estimatedSize := 0 + if ev.IsDelete() { + if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeDelete, ev.PreColumns); group != nil { + groups = append(groups, group) + estimatedSize += size + } + } else { + if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeNew, ev.Columns); group != nil { + groups = append(groups, group) + estimatedSize += size + } + if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeOld, ev.PreColumns); group != nil { + groups = append(groups, group) + estimatedSize += size + } + } + return estimatedSize, craftRowChangedEvent(groups) +} + +/// A buffer to save row changed events in batch +type craftRowChangedEventBuffer struct { + keys *craftColumnarKeys + + events []craftRowChangedEvent + eventsCount int + estimatedSize int +} + +func (b *craftRowChangedEventBuffer) encode() []byte { + bits := newCraftMessageEncoder().encodeKeys(b.keys).encodeRowChangeEvents(b.events[:b.eventsCount]).encode() + b.reset() + return bits +} + +func (b *craftRowChangedEventBuffer) appendRowChangedEvent(ev *model.RowChangedEvent) (rows, size int) { + var partition int64 = -1 + if ev.Table.IsPartition { + partition = ev.Table.TableID + } + + b.estimatedSize += b.keys.appendKey( + ev.CommitTs, + uint64(model.MqMessageTypeRow), + ev.RowID, + partition, + []byte(ev.Table.Schema), + []byte(ev.Table.Table), + ) + if b.eventsCount+1 > len(b.events) { + newSize := newBufferSize(b.eventsCount) + events := make([]craftRowChangedEvent, newSize) + copy(events, b.events) + b.events = events + } + size, message := newCraftRowChangedMessage(ev) + b.events[b.eventsCount] = message + b.eventsCount++ + b.estimatedSize += size + return b.eventsCount, b.estimatedSize +} + +func (b *craftRowChangedEventBuffer) reset() { + b.keys.reset() + b.eventsCount = 0 + b.estimatedSize = 0 +} + +func (b *craftRowChangedEventBuffer) size() int { + return b.estimatedSize +} + +func (b *craftRowChangedEventBuffer) getKeys() *craftColumnarKeys { + return b.keys +} diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go new file mode 100644 index 00000000000..e511d89b6a4 --- /dev/null +++ b/cdc/sink/codec/craft_test.go @@ -0,0 +1,297 @@ +// Copyright 2021 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "math" + "math/rand" + "strconv" + + "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +type craftBatchSuite struct { + rowCases [][]*model.RowChangedEvent + ddlCases [][]*model.DDLEvent + resolvedTsCases [][]uint64 +} + +var _ = check.Suite(&craftBatchSuite{ + rowCases: codecRowCases, + ddlCases: codecDDLCases, + resolvedTsCases: codecResolvedTSCases, +}) + +func (s *craftBatchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEncoder, newDecoder func(value []byte) (EventBatchDecoder, error)) { + checkRowDecoder := func(decoder EventBatchDecoder, cs []*model.RowChangedEvent) { + index := 0 + for { + tp, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + if !hasNext { + break + } + c.Assert(tp, check.Equals, model.MqMessageTypeRow) + row, err := decoder.NextRowChangedEvent() + c.Assert(err, check.IsNil) + c.Assert(row, check.DeepEquals, cs[index]) + index++ + } + } + checkDDLDecoder := func(decoder EventBatchDecoder, cs []*model.DDLEvent) { + index := 0 + for { + tp, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + if !hasNext { + break + } + c.Assert(tp, check.Equals, model.MqMessageTypeDDL) + ddl, err := decoder.NextDDLEvent() + c.Assert(err, check.IsNil) + c.Assert(ddl, check.DeepEquals, cs[index]) + index++ + } + } + checkTSDecoder := func(decoder EventBatchDecoder, cs []uint64) { + index := 0 + for { + tp, hasNext, err := decoder.HasNext() + c.Assert(err, check.IsNil) + if !hasNext { + break + } + c.Assert(tp, check.Equals, model.MqMessageTypeResolved) + ts, err := decoder.NextResolvedEvent() + c.Assert(err, check.IsNil) + c.Assert(ts, check.DeepEquals, cs[index]) + index++ + } + } + + for _, cs := range s.rowCases { + encoder := newEncoder() + err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + c.Assert(err, check.IsNil) + + events := 0 + for _, row := range cs { + op, err := encoder.AppendRowChangedEvent(row) + events++ + c.Assert(err, check.IsNil) + c.Assert(op, check.Equals, EncoderNoOperation) + } + // test normal decode + if len(cs) > 0 { + res := encoder.Build() + c.Assert(res, check.HasLen, 1) + decoder, err := newDecoder(res[0].Value) + c.Assert(err, check.IsNil) + checkRowDecoder(decoder, cs) + } + } + + for _, cs := range s.ddlCases { + encoder := newEncoder() + err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + c.Assert(err, check.IsNil) + + for i, ddl := range cs { + msg, err := encoder.EncodeDDLEvent(ddl) + c.Assert(err, check.IsNil) + c.Assert(msg, check.NotNil) + decoder, err := newDecoder(msg.Value) + c.Assert(err, check.IsNil) + checkDDLDecoder(decoder, cs[i:i+1]) + } + } + + for _, cs := range s.resolvedTsCases { + encoder := newEncoder() + err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + c.Assert(err, check.IsNil) + + for i, ts := range cs { + msg, err := encoder.EncodeCheckpointEvent(ts) + c.Assert(err, check.IsNil) + c.Assert(msg, check.NotNil) + decoder, err := newDecoder(msg.Value) + c.Assert(err, check.IsNil) + checkTSDecoder(decoder, cs[i:i+1]) + } + } +} + +func (s *craftBatchSuite) TestParamsEdgeCases(c *check.C) { + defer testleak.AfterTest(c)() + encoder := NewCraftEventBatchEncoder().(*CraftEventBatchEncoder) + err := encoder.SetParams(map[string]string{}) + c.Assert(err, check.IsNil) + c.Assert(encoder.maxBatchSize, check.Equals, 4096) + c.Assert(encoder.maxMessageSize, check.Equals, 64*1024*1024) + + err = encoder.SetParams(map[string]string{"max-message-bytes": "0"}) + c.Assert(err, check.ErrorMatches, ".*invalid.*") + + err = encoder.SetParams(map[string]string{"max-message-bytes": "-1"}) + c.Assert(err, check.ErrorMatches, ".*invalid.*") + + err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxInt32)}) + c.Assert(err, check.IsNil) + c.Assert(encoder.maxBatchSize, check.Equals, 4096) + c.Assert(encoder.maxMessageSize, check.Equals, math.MaxInt32) + + err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxUint32)}) + c.Assert(err, check.NotNil) + + err = encoder.SetParams(map[string]string{"max-batch-size": "0"}) + c.Assert(err, check.ErrorMatches, ".*invalid.*") + + err = encoder.SetParams(map[string]string{"max-batch-size": "-1"}) + c.Assert(err, check.ErrorMatches, ".*invalid.*") + + err = encoder.SetParams(map[string]string{"max-batch-size": strconv.Itoa(math.MaxUint16)}) + c.Assert(err, check.IsNil) + c.Assert(encoder.maxBatchSize, check.Equals, int(math.MaxUint16)) + c.Assert(encoder.maxMessageSize, check.Equals, 64*1024*1024) + + err = encoder.SetParams(map[string]string{"max-batch-size": strconv.Itoa(math.MaxInt32)}) + c.Assert(err, check.NotNil) + + err = encoder.SetParams(map[string]string{"max-batch-size": strconv.Itoa(math.MaxUint32)}) + c.Assert(err, check.NotNil) +} + +func (s *craftBatchSuite) TestMaxMessageBytes(c *check.C) { + defer testleak.AfterTest(c)() + encoder := NewCraftEventBatchEncoder() + err := encoder.SetParams(map[string]string{"max-message-bytes": "256"}) + c.Check(err, check.IsNil) + + testEvent := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + } + + for i := 0; i < 10000; i++ { + r, err := encoder.AppendRowChangedEvent(testEvent) + c.Check(r, check.Equals, EncoderNoOperation) + c.Check(err, check.IsNil) + } + + messages := encoder.Build() + for _, msg := range messages { + c.Assert(msg.Length(), check.LessEqual, 256) + } +} + +func (s *craftBatchSuite) TestMaxBatchSize(c *check.C) { + defer testleak.AfterTest(c)() + encoder := NewCraftEventBatchEncoder() + err := encoder.SetParams(map[string]string{"max-batch-size": "64"}) + c.Check(err, check.IsNil) + + testEvent := &model.RowChangedEvent{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + } + + for i := 0; i < 10000; i++ { + r, err := encoder.AppendRowChangedEvent(testEvent) + c.Check(r, check.Equals, EncoderNoOperation) + c.Check(err, check.IsNil) + } + + messages := encoder.Build() + sum := 0 + for _, msg := range messages { + decoder, err := NewCraftEventBatchDecoder(msg.Value) + c.Check(err, check.IsNil) + count := 0 + for { + t, hasNext, err := decoder.HasNext() + c.Check(err, check.IsNil) + if !hasNext { + break + } + + c.Check(t, check.Equals, model.MqMessageTypeRow) + _, err = decoder.NextRowChangedEvent() + c.Check(err, check.IsNil) + count++ + } + c.Check(count, check.LessEqual, 64) + sum += count + } + c.Check(sum, check.Equals, 10000) +} + +func (s *craftBatchSuite) TestDefaultEventBatchCodec(c *check.C) { + defer testleak.AfterTest(c)() + s.testBatchCodec(c, func() EventBatchEncoder { + encoder := NewCraftEventBatchEncoder() + return encoder + }, NewCraftEventBatchDecoder) +} + +var _ = check.Suite(&craftCodecSuite{}) + +type craftCodecSuite struct{} + +func (s *craftCodecSuite) TestSizeTable(c *check.C) { + defer testleak.AfterTest(c)() + tables := [][]uint64{ + []uint64{ + 1, 3, 5, 7, 9, + }, + []uint64{ + 2, 4, 6, 8, 10, + }, + } + bits := make([]byte, 16) + rand.Read(bits) + bits = encodeSizeTables(bits, tables) + + size, decoded, err := decodeSizeTables(bits) + c.Check(err, check.IsNil) + c.Check(decoded, check.DeepEquals, tables) + c.Check(size, check.Equals, len(bits)-16) +} + +func (s *craftCodecSuite) TestUvarintReverse(c *check.C) { + defer testleak.AfterTest(c)() + + var i uint64 = 0 + + for i < 0x8000000000000000 { + bits := make([]byte, 16) + rand.Read(bits) + bits, bytes1 := encodeUvarintReversed(bits, i) + bytes2, u64, err := decodeUvarintReversed(bits) + c.Check(err, check.IsNil) + c.Check(u64, check.Equals, i) + c.Check(bytes1, check.Equals, len(bits)-16) + c.Check(bytes1, check.Equals, bytes2) + if i == 0 { + i = 1 + } else { + i <<= 1 + } + } +} diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index f63b8e82ca3..b70397264f2 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -143,6 +143,7 @@ const ( ProtocolAvro ProtocolMaxwell ProtocolCanalJSON + ProtocolCraft ) // FromString converts the protocol from string to Protocol enum type @@ -158,6 +159,8 @@ func (p *Protocol) FromString(protocol string) { *p = ProtocolMaxwell case "canal-json": *p = ProtocolCanalJSON + case "craft": + *p = ProtocolCraft default: *p = ProtocolDefault log.Warn("can't support codec protocol, using default protocol", zap.String("protocol", protocol)) @@ -177,6 +180,8 @@ func NewEventBatchEncoder(p Protocol) func() EventBatchEncoder { return NewMaxwellEventBatchEncoder case ProtocolCanalJSON: return NewCanalFlatEventBatchEncoder + case ProtocolCraft: + return NewCraftEventBatchEncoder default: log.Warn("unknown codec protocol value of EventBatchEncoder", zap.Int("protocol_value", int(p))) return NewJSONEventBatchEncoder diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index e7a851609d3..ba5e6330b61 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -535,27 +535,27 @@ func (d *JSONEventBatchEncoder) SetParams(params map[string]string) error { if maxMessageBytes, ok := params["max-message-bytes"]; ok { d.maxKafkaMessageSize, err = strconv.Atoi(maxMessageBytes) if err != nil { - return cerror.ErrKafkaInvalidConfig.Wrap(err) + return cerror.ErrSinkInvalidConfig.Wrap(err) } } else { d.maxKafkaMessageSize = DefaultMaxMessageBytes } if d.maxKafkaMessageSize <= 0 { - return cerror.ErrKafkaInvalidConfig.Wrap(errors.Errorf("invalid max-message-bytes %d", d.maxKafkaMessageSize)) + return cerror.ErrSinkInvalidConfig.Wrap(errors.Errorf("invalid max-message-bytes %d", d.maxKafkaMessageSize)) } if maxBatchSize, ok := params["max-batch-size"]; ok { d.maxBatchSize, err = strconv.Atoi(maxBatchSize) if err != nil { - return cerror.ErrKafkaInvalidConfig.Wrap(err) + return cerror.ErrSinkInvalidConfig.Wrap(err) } } else { d.maxBatchSize = DefaultMaxBatchSize } if d.maxBatchSize <= 0 { - return cerror.ErrKafkaInvalidConfig.Wrap(errors.Errorf("invalid max-batch-size %d", d.maxBatchSize)) + return cerror.ErrSinkInvalidConfig.Wrap(errors.Errorf("invalid max-batch-size %d", d.maxBatchSize)) } return nil } diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 6b3d19f49c5..f2d6db245d8 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -33,57 +33,9 @@ type batchSuite struct { } var _ = check.Suite(&batchSuite{ - rowCases: [][]*model.RowChangedEvent{{{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, - }}, {{ - CommitTs: 1, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "aa"}}, - }, { - CommitTs: 2, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, - }, { - CommitTs: 3, - Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "bb"}}, - }, { - CommitTs: 4, - Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, - Columns: []*model.Column{{Name: "col1", Type: 1, Value: "cc"}}, - }}, {}}, - ddlCases: [][]*model.DDLEvent{{{ - CommitTs: 1, - TableInfo: &model.SimpleTableInfo{ - Schema: "a", Table: "b", - }, - Query: "create table a", - Type: 1, - }}, {{ - CommitTs: 1, - TableInfo: &model.SimpleTableInfo{ - Schema: "a", Table: "b", - }, - Query: "create table a", - Type: 1, - }, { - CommitTs: 2, - TableInfo: &model.SimpleTableInfo{ - Schema: "a", Table: "b", - }, - Query: "create table b", - Type: 2, - }, { - CommitTs: 3, - TableInfo: &model.SimpleTableInfo{ - Schema: "a", Table: "b", - }, - Query: "create table c", - Type: 3, - }}, {}}, - resolvedTsCases: [][]uint64{{1}, {1, 2, 3}, {}}, + rowCases: codecRowCases, + ddlCases: codecDDLCases, + resolvedTsCases: codecResolvedTSCases, }) func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEncoder, newDecoder func(key []byte, value []byte) (EventBatchDecoder, error)) { diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index acdb6da8661..881842b047e 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -105,6 +105,8 @@ var ( ErrCanalDecodeFailed = errors.Normalize("canal decode failed", errors.RFCCodeText("CDC:ErrCanalDecodeFailed")) ErrCanalEncodeFailed = errors.Normalize("canal encode failed", errors.RFCCodeText("CDC:ErrCanalEncodeFailed")) ErrOldValueNotEnabled = errors.Normalize("old value is not enabled", errors.RFCCodeText("CDC:ErrOldValueNotEnabled")) + ErrSinkInvalidConfig = errors.Normalize("sink config invalid", errors.RFCCodeText("CDC:ErrSinkInvalidConfig")) + ErrCraftCodecInvalidData = errors.Normalize("craft codec invalid data", errors.RFCCodeText("CDC:ErrCraftCodecInvalidData")) // utilities related errors ErrToTLSConfigFailed = errors.Normalize("generate tls config failed", errors.RFCCodeText("CDC:ErrToTLSConfigFailed")) From 96ee865669b6ee5b20e04affea6fa998eb1afaed Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:04:17 +0800 Subject: [PATCH 02/39] Add license Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 13 +++++++++++++ cdc/sink/codec/craft_decoder.go | 13 +++++++++++++ cdc/sink/codec/craft_encoder.go | 13 +++++++++++++ cdc/sink/codec/craft_model.go | 13 +++++++++++++ 4 files changed, 52 insertions(+) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 5fb20ecb2f7..45fe936db0f 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -1,3 +1,16 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package codec import ( diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 71c8083d48f..1bfb3628c03 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -1,3 +1,16 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package codec import ( diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft_encoder.go index e1a376ac084..1bc0262d566 100644 --- a/cdc/sink/codec/craft_encoder.go +++ b/cdc/sink/codec/craft_encoder.go @@ -1,3 +1,16 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package codec import ( diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go index 50e48f9832b..796c933732f 100644 --- a/cdc/sink/codec/craft_model.go +++ b/cdc/sink/codec/craft_model.go @@ -1,3 +1,16 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package codec import ( From 7347d7c9f5e4899239005a333560ab4c1cbdac3f Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:39:09 +0800 Subject: [PATCH 03/39] Fix some lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 13 ++----------- cdc/sink/codec/craft.go | 4 ---- cdc/sink/codec/craft_decoder.go | 10 +++++++--- cdc/sink/codec/craft_model.go | 2 +- cdc/sink/codec/craft_test.go | 4 ++-- 5 files changed, 12 insertions(+), 21 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index e9a80a0722d..6568f56f5b0 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -16,7 +16,6 @@ package codec import ( "bytes" "compress/zlib" - "encoding/json" "fmt" "github.com/pingcap/check" @@ -81,14 +80,6 @@ var ( codecResolvedTSCases = [][]uint64{{1}, {1, 2, 3}, {}} ) -func dumpJson(v interface{}) { - b, e := json.Marshal(v) - if e != nil { - panic(e) - } - fmt.Println(string(b)) -} - var _ = check.Suite(&codecTestSuite{}) type codecTestSuite struct{} @@ -100,9 +91,9 @@ func (s *codecTestSuite) checkCompressedSize(messages []*MQMessage) (int, int) { for _, message := range messages { originalSize += len(message.Key) + len(message.Value) if len(message.Key) > 0 { - writer.Write(message.Key) + _, _ = writer.Write(message.Key) } - writer.Write(message.Value) + _, _ = writer.Write(message.Value) } writer.Close() return originalSize, buff.Len() diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 45fe936db0f..a0959383fa7 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -86,10 +86,6 @@ const ( // CraftVersion1 represents the version of craft format CraftVersion1 uint64 = 1 - // craftMaxMessageBytes sets the default value for max-message-bytes - craftMaxMessageBytes int = 64 * 1024 * 1024 // 64M - // craftMaxBatchSize sets the default value for max-batch-size - craftMaxBatchSize int = 4096 // default buffer size craftDefaultBufferCapacity = 512 diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 1bfb3628c03..ced8057d15e 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -413,7 +413,7 @@ func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, err if err != nil { return nil, errors.Trace(err) } - bits, values, err = decodeBytesChunk(bits, numColumns) + _, values, err = decodeBytesChunk(bits, numColumns) if err != nil { return nil, errors.Trace(err) } @@ -479,9 +479,13 @@ func decodeRowChangedEvent(keys *craftColumnarKeys, keyIndex int, sizeTables [][ case craftColumnGroupTypeDelete: fallthrough case craftColumnGroupTypeOld: - ev.PreColumns, err = columnGroup.toModel() + if ev.PreColumns, err = columnGroup.toModel(); err != nil { + return nil, errors.Trace(err) + } case craftColumnGroupTypeNew: - ev.Columns, err = columnGroup.toModel() + if ev.Columns, err = columnGroup.toModel(); err != nil { + return nil, errors.Trace(err) + } } } return ev, nil diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go index 796c933732f..e3e524afd75 100644 --- a/cdc/sink/codec/craft_model.go +++ b/cdc/sink/codec/craft_model.go @@ -153,7 +153,7 @@ func decodeCraftColumnarKeys(bits []byte, numKeys int) (*craftColumnarKeys, erro if bits, schema, err = decodeNullableBytesChunk(bits, numKeys); err != nil { return nil, errors.Trace(err) } - if bits, table, err = decodeNullableBytesChunk(bits, numKeys); err != nil { + if _, table, err = decodeNullableBytesChunk(bits, numKeys); err != nil { return nil, errors.Trace(err) } return &craftColumnarKeys{ diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index e511d89b6a4..6f2df4aafce 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -257,10 +257,10 @@ type craftCodecSuite struct{} func (s *craftCodecSuite) TestSizeTable(c *check.C) { defer testleak.AfterTest(c)() tables := [][]uint64{ - []uint64{ + { 1, 3, 5, 7, 9, }, - []uint64{ + { 2, 4, 6, 8, 10, }, } From 86f461261e95d28d1298e2fbf10a0a051b7fefd2 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:48:23 +0800 Subject: [PATCH 04/39] Add more test to supress not used function lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_test.go | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index 6f2df4aafce..66f056f6898 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -295,3 +295,32 @@ func (s *craftCodecSuite) TestUvarintReverse(c *check.C) { } } } + +func newNullableString(a string) *string { + return &a +} + +func (s *craftCodecSuite) TestEncodeChunk(c *check.C) { + defer testleak.AfterTest(c)() + stringChunk := []string{"a", "b", "c"} + nullableStringChunk := []*string{newNullableString("a"), newNullableString("b"), newNullableString("c")} + int64Chunk := []int64{1, 2, 3} + + bits := encodeStringChunk(nil, stringChunk) + bits, decodedStringChunk, err := decodeStringChunk(bits, 3) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedStringChunk, check.DeepEquals, stringChunk) + + bits = encodeNullableStringChunk(nil, nullableStringChunk) + bits, decodedNullableStringChunk, err := decodeNullableStringChunk(bits, 3) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedNullableStringChunk, check.DeepEquals, nullableStringChunk) + + bits = encodeVarintChunk(nil, int64Chunk) + bits, decodedVarintChunk, err := decodeVarintChunk(bits, 3) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedVarintChunk, check.DeepEquals, int64Chunk) +} From 1cb0a2ddae5fda063c07fe8ef3cb1a3713385f0b Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:55:22 +0800 Subject: [PATCH 05/39] Remove not useful decodeRowChangedEvent function Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_decoder.go | 41 --------------------------------- 1 file changed, 41 deletions(-) diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index ced8057d15e..594115c3bae 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -449,44 +449,3 @@ func (d *craftMessageDecoder) decodeRowChangedEvent(index int) (preColumns, colu } return preColumns, columns, nil } - -// Events decoder -func decodeRowChangedEvent(keys *craftColumnarKeys, keyIndex int, sizeTables [][]uint64, bits []byte) (*model.RowChangedEvent, error) { - ev := &model.RowChangedEvent{} - ev.CommitTs = keys.getTs(keyIndex) - ev.Table = &model.TableName{ - Schema: keys.getSchema(keyIndex), - Table: keys.getTable(keyIndex), - } - partition := keys.getPartition(keyIndex) - if partition >= 0 { - ev.Table.TableID = partition - ev.Table.IsPartition = true - } - var columnGroup *craftColumnarColumnGroup - var err error - columnGroupSizeTable := sizeTables[keyIndex] - columnGroupIndex := 0 - for len(bits) > 0 { - columnGroupSize := columnGroupSizeTable[columnGroupIndex] - columnGroup, err = decodeCraftColumnarColumnGroup(bits[:columnGroupSize]) - bits = bits[columnGroupSize:] - columnGroupIndex++ - if err != nil { - return nil, errors.Trace(err) - } - switch columnGroup.ty { - case craftColumnGroupTypeDelete: - fallthrough - case craftColumnGroupTypeOld: - if ev.PreColumns, err = columnGroup.toModel(); err != nil { - return nil, errors.Trace(err) - } - case craftColumnGroupTypeNew: - if ev.Columns, err = columnGroup.toModel(); err != nil { - return nil, errors.Trace(err) - } - } - } - return ev, nil -} From b570de23d3571f8f91d7498d4693cc59fae88309 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 22:56:53 +0800 Subject: [PATCH 06/39] Use the correct size table to decode column groups Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_decoder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 594115c3bae..137fb9e2c56 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -428,7 +428,7 @@ func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, err func (d *craftMessageDecoder) decodeRowChangedEvent(index int) (preColumns, columns *craftColumnarColumnGroup, err error) { bits := d.valueBits(index) - columnGroupSizeTable := d.sizeTables[craftValueSizeTableIndex+index] + columnGroupSizeTable := d.sizeTables[craftColumnGroupSizeTableStartIndex+index] columnGroupIndex := 0 for len(bits) > 0 { columnGroupSize := columnGroupSizeTable[columnGroupIndex] From d1908d56e369585c6486dd3ea4a23280415dc2a1 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 23:02:23 +0800 Subject: [PATCH 07/39] Fix more lints Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 9 +++++++++ cdc/sink/codec/craft_decoder.go | 8 +++----- cdc/sink/codec/craft_encoder.go | 9 +++------ 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index a0959383fa7..73c8cb34e57 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -100,6 +100,7 @@ const ( craftColumnGroupSizeTableStartIndex = 2 ) +// CraftEventBatchEncoder encodes the events into the byte of a batch into craft binary format. type CraftEventBatchEncoder struct { rowChangedBuffer *craftRowChangedEventBuffer messageBuf []*MQMessage @@ -122,6 +123,7 @@ func (e *CraftEventBatchEncoder) flush() { e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.encode(), ts, model.MqMessageTypeRow, &schema, &table)) } +// AppendResolvedEvent is no-op func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent) (EncoderResult, error) { rows, size := e.rowChangedBuffer.appendRowChangedEvent(ev) if size > e.maxMessageSize || rows >= e.maxBatchSize { @@ -135,10 +137,12 @@ func (e *CraftEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, return EncoderNoOperation, nil } +// EncodeDDLEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) EncodeDDLEvent(ev *model.DDLEvent) (*MQMessage, error) { return newDDLMQMessage(ProtocolCraft, nil, newCraftDDLEventEncoder(ev).encode(), ev), nil } +// Build implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Build() []*MQMessage { if e.rowChangedBuffer.size() > 0 { // flush buffered data to message buffer @@ -149,18 +153,22 @@ func (e *CraftEventBatchEncoder) Build() []*MQMessage { return ret } +// MixedBuild implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) MixedBuild(withVersion bool) []byte { panic("Only JsonEncoder supports mixed build") } +// Size implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Size() int { return e.rowChangedBuffer.size() } +// Reset implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Reset() { e.rowChangedBuffer.reset() } +// SetParams reads relevant parameters for craft protocol func (e *CraftEventBatchEncoder) SetParams(params map[string]string) error { var err error if maxMessageBytes, ok := params["max-message-bytes"]; ok { @@ -191,6 +199,7 @@ func (e *CraftEventBatchEncoder) SetParams(params map[string]string) error { return nil } +// NewCraftEventBatchEncoder creates a new CraftEventBatchEncoder. func NewCraftEventBatchEncoder() EventBatchEncoder { return &CraftEventBatchEncoder{ rowChangedBuffer: &craftRowChangedEventBuffer{ diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 137fb9e2c56..6aba12ee721 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -132,9 +132,8 @@ func decodeString(bits []byte) ([]byte, string, error) { bits, bytes, err := decodeBytes(bits) if err == nil { return bits, string(bytes), nil - } else { - return bits, "", errors.Trace(err) } + return bits, "", errors.Trace(err) } /// Chunk decoders @@ -314,10 +313,9 @@ func decodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ if flag.IsUnsigned() { _, u64, err := decodeUvarint(bits) return u64, err - } else { - _, i64, err := decodeUvarint(bits) - return i64, err } + _, i64, err := decodeUvarint(bits) + return i64, err case mysql.TypeUnspecified: fallthrough case mysql.TypeNull: diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft_encoder.go index 1bc0262d566..791b8133b0e 100644 --- a/cdc/sink/codec/craft_encoder.go +++ b/cdc/sink/codec/craft_encoder.go @@ -34,9 +34,8 @@ func encodeVarint(bits []byte, data int64) []byte { buf = buf[:l] if bits == nil { return buf - } else { - return append(bits, buf...) } + return append(bits, buf...) } func encodeUvarint(bits []byte, data uint64) []byte { @@ -45,9 +44,8 @@ func encodeUvarint(bits []byte, data uint64) []byte { buf = buf[:l] if bits == nil { return buf - } else { - return append(bits, buf...) } + return append(bits, buf...) } func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { @@ -198,9 +196,8 @@ func encodeTiDBType(ty byte, flag model.ColumnFlagType, value interface{}) []byt // value type for these mysql types are int64 or uint64 depends on flags if flag.IsUnsigned() { return encodeUvarint(nil, value.(uint64)) - } else { - return encodeVarint(nil, value.(int64)) } + return encodeVarint(nil, value.(int64)) case mysql.TypeUnspecified: fallthrough case mysql.TypeNull: From d2bdea488fea98ac081efbe05b55ef3899c2408a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 23:04:18 +0800 Subject: [PATCH 08/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 73c8cb34e57..f88b5b45ae1 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -123,7 +123,7 @@ func (e *CraftEventBatchEncoder) flush() { e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.encode(), ts, model.MqMessageTypeRow, &schema, &table)) } -// AppendResolvedEvent is no-op +// AppendRowChangedEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent) (EncoderResult, error) { rows, size := e.rowChangedBuffer.appendRowChangedEvent(ev) if size > e.maxMessageSize || rows >= e.maxBatchSize { From 0e2133184a2fa05a0c454df6315c7a07e06b92a3 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 8 Apr 2021 23:13:03 +0800 Subject: [PATCH 09/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_decoder.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 6aba12ee721..6d7eff5f046 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -72,7 +72,7 @@ func decodeUvarintReversed(bits []byte) (int, uint64, error) { func decodeUvarintReversedLength(bits []byte) (int, int, error) { nb, x, err := decodeUvarintReversed(bits) if x > math.MaxInt32 { - return 0, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + return 0, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length is greater than max int32") } return nb, int(x), err } @@ -83,7 +83,7 @@ func decodeUvarint32(bits []byte) ([]byte, int32, error) { return bits, 0, errors.Trace(err) } if x > math.MaxInt32 { - return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length is greater than max int32") } return newBits, int32(x), nil } @@ -94,7 +94,7 @@ func decodeVarint32(bits []byte) ([]byte, int32, error) { return bits, 0, errors.Trace(err) } if x > math.MaxInt32 { - return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length %d is greater than max limit of %d", x, math.MaxInt32) + return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("length is greater than max int32") } return newBits, int32(x), nil } From 97b9e1cafbba8efff58367318b34911bb880e307 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 12 Apr 2021 21:37:24 +0800 Subject: [PATCH 10/39] Change to little endian like what protobuf does. Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_decoder.go | 2 +- cdc/sink/codec/craft_encoder.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 6d7eff5f046..659cb0e94b7 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -113,7 +113,7 @@ func decodeFloat64(bits []byte) ([]byte, float64, error) { if len(bits) < 8 { return bits, 0, cerror.ErrCraftCodecInvalidData.GenWithStack("buffer underflow") } - x := binary.BigEndian.Uint64(bits) + x := binary.LittleEndian.Uint64(bits) return bits[8:], math.Float64frombits(x), nil } diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft_encoder.go index 791b8133b0e..c6d0b0c7366 100644 --- a/cdc/sink/codec/craft_encoder.go +++ b/cdc/sink/codec/craft_encoder.go @@ -24,7 +24,7 @@ import ( /// Primitive type encoders func encodeFloat64(bits []byte, data float64) []byte { buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, math.Float64bits(data)) + binary.LittleEndian.PutUint64(buf, math.Float64bits(data)) return buf } From dc77557d5fd2a2626ef9dd5dd394112875c9d2eb Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 13 Apr 2021 21:59:22 +0800 Subject: [PATCH 11/39] Remove comments and fix column group values encoding Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 60 --------------------------------- cdc/sink/codec/craft_decoder.go | 37 -------------------- cdc/sink/codec/craft_model.go | 39 ++++++++++++++++++++- 3 files changed, 38 insertions(+), 98 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index f88b5b45ae1..fc554e63a3e 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -22,66 +22,6 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" ) -// string/bytes array layout -// n bytes array of elements' size, format: uvarint array -// n bytes elements, format: bits -// -// varint/uvarint array layout -// n bytes elements. format: varint / uvarint -// -// delta varint/uvarint array layout -// n bytes base number -// n bytes offsets. format: varint/uvarint -// -// string/bytes layout -// n bytes varint length -// n bytes payload -// -// float layout, standard protobuf float -// double layout, standard protobuf double -// varint layout, standard protobuf varint -// uvarint layout, standard protobuf uvarint -// -// Message layout -// 2 bytes version -// 2 bytes number of pairs -// n bytes keys -// n bytes values -// n bytes size tables -// -// Keys layout -// n bytes array of commit ts, format: delta uvarint array -// n bytes array of type, format: uvarint array -// n bytes array of row id, format: uvarint array -// n bytes array of partition id, format: varint array, -1 means field is not set -// n bytes array of schema, format: string array -// n bytes array of table, format: string array -// -// Row changed layout -// n bytes multiple column groups -// -// Column group layout -// 1 byte column group type: 1 New Values, 2: Old Values, 3: Delete Values -// n bytes number of columns, format: uvarint -// n bytes array of name, format: string array -// n bytes array of type, format: uvarint array -// n bytes array of flag, format: uvarint array -// n bytes array of value, format: nullable bytes array -// -// DDL layout -// n bytes type, format: uvarint -// n bytes query, format: string -// -// Size tables layout -// n bytes table to store size of serialized keys -// n bytes table to store size of values -// n bytes tables to store of serialized column groups -// n bytes size of serialized size tables, format: reversed uvarint -// -// Size table layout -// n bytes number of elements, format: uvarint -// n bytes repeated elements, format: uvarint -// const ( // CraftVersion1 represents the version of craft format CraftVersion1 uint64 = 1 diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 659cb0e94b7..9b61eb8e4a0 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -387,43 +387,6 @@ func (d *craftMessageDecoder) decodeDDLEvent(index int) (pmodel.ActionType, stri return pmodel.ActionType(ty), query, err } -func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, error) { - var numColumns int - bits, ty, err := decodeUint8(bits) - if err != nil { - return nil, errors.Trace(err) - } - bits, numColumns, err = decodeUvarintLength(bits) - if err != nil { - return nil, errors.Trace(err) - } - var names, values [][]byte - var types, flags []uint64 - bits, names, err = decodeBytesChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - bits, types, err = decodeUvarintChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - bits, flags, err = decodeUvarintChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - _, values, err = decodeBytesChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - return &craftColumnarColumnGroup{ - ty: ty, - names: names, - types: types, - flags: flags, - values: values, - }, nil -} - func (d *craftMessageDecoder) decodeRowChangedEvent(index int) (preColumns, columns *craftColumnarColumnGroup, err error) { bits := d.valueBits(index) columnGroupSizeTable := d.sizeTables[craftColumnGroupSizeTableStartIndex+index] diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go index e3e524afd75..6c338eb45a2 100644 --- a/cdc/sink/codec/craft_model.go +++ b/cdc/sink/codec/craft_model.go @@ -182,7 +182,7 @@ func (g *craftColumnarColumnGroup) encode(bits []byte) []byte { bits = encodeBytesChunk(bits, g.names) bits = encodeUvarintChunk(bits, g.types) bits = encodeUvarintChunk(bits, g.flags) - bits = encodeBytesChunk(bits, g.values) + bits = encodeNullableBytesChunk(bits, g.values) return bits } @@ -205,6 +205,43 @@ func (g *craftColumnarColumnGroup) toModel() ([]*model.Column, error) { return columns, nil } +func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, error) { + var numColumns int + bits, ty, err := decodeUint8(bits) + if err != nil { + return nil, errors.Trace(err) + } + bits, numColumns, err = decodeUvarintLength(bits) + if err != nil { + return nil, errors.Trace(err) + } + var names, values [][]byte + var types, flags []uint64 + bits, names, err = decodeBytesChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + bits, types, err = decodeUvarintChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + bits, flags, err = decodeUvarintChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + _, values, err = decodeNullableBytesChunk(bits, numColumns) + if err != nil { + return nil, errors.Trace(err) + } + return &craftColumnarColumnGroup{ + ty: ty, + names: names, + types: types, + flags: flags, + values: values, + }, nil +} + func newCraftColumnarColumnGroup(ty byte, columns []*model.Column) (int, *craftColumnarColumnGroup) { var names [][]byte var values [][]byte From b3a5ff9efafa17da8d54899785f5b85c67191424 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 13 Apr 2021 23:27:09 +0800 Subject: [PATCH 12/39] Set schema and table to nil when they are empty Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft_encoder.go | 11 +++++++++-- cdc/sink/codec/craft_model.go | 12 ++++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft_encoder.go index c6d0b0c7366..63e3513c818 100644 --- a/cdc/sink/codec/craft_encoder.go +++ b/cdc/sink/codec/craft_encoder.go @@ -284,13 +284,20 @@ func newCraftResolvedEventEncoder(ts uint64) *craftMessageEncoder { func newCraftDDLEventEncoder(ev *model.DDLEvent) *craftMessageEncoder { ty := uint64(ev.Type) query := ev.Query + var schema, table []byte + if len(ev.TableInfo.Schema) > 0 { + schema = []byte(ev.TableInfo.Schema) + } + if len(ev.TableInfo.Table) > 0 { + table = []byte(ev.TableInfo.Table) + } return newCraftMessageEncoder().encodeKeys(&craftColumnarKeys{ ts: []uint64{uint64(ev.CommitTs)}, ty: []uint64{uint64(model.MqMessageTypeDDL)}, rowID: []int64{int64(-1)}, partition: []int64{int64(-1)}, - schema: [][]byte{[]byte(ev.TableInfo.Schema)}, - table: [][]byte{[]byte(ev.TableInfo.Table)}, + schema: [][]byte{schema}, + table: [][]byte{table}, count: 1, }).encodeUvarint(ty).encodeString(query).encodeValueSize() } diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go index 6c338eb45a2..6e2727ee942 100644 --- a/cdc/sink/codec/craft_model.go +++ b/cdc/sink/codec/craft_model.go @@ -317,13 +317,21 @@ func (b *craftRowChangedEventBuffer) appendRowChangedEvent(ev *model.RowChangedE partition = ev.Table.TableID } + var schema, table []byte + if len(ev.Table.Schema) > 0 { + schema = []byte(ev.Table.Schema) + } + if len(ev.Table.Table) > 0 { + table = []byte(ev.Table.Table) + } + b.estimatedSize += b.keys.appendKey( ev.CommitTs, uint64(model.MqMessageTypeRow), ev.RowID, partition, - []byte(ev.Table.Schema), - []byte(ev.Table.Table), + schema, + table, ) if b.eventsCount+1 > len(b.events) { newSize := newBufferSize(b.eventsCount) From f1c602f61d9cdea209b97909502a6bc91154a398 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 17 Apr 2021 10:18:41 +0800 Subject: [PATCH 13/39] Add more test cases and fix bugs about the fail tests Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 55 ++++++++++++++++++++++++++++++--- cdc/sink/codec/craft_decoder.go | 2 +- cdc/sink/codec/craft_test.go | 4 +-- cdc/sink/codec/json.go | 20 ++++++++++++ cdc/sink/codec/json_test.go | 24 ++++++++++++++ 5 files changed, 97 insertions(+), 8 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 6568f56f5b0..b03cec02777 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -28,23 +28,68 @@ var ( codecRowCases = [][]*model.RowChangedEvent{{{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + Columns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, }}, {{ CommitTs: 1, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("aa")}}, + Columns: []*model.Column{ + {Name: "varchar1", Type: mysql.TypeVarchar, Value: []byte("varchar")}, + {Name: "string1", Type: mysql.TypeString, Value: []byte("string")}, + {Name: "date1", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp1", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime1", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float1", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long1", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null1", Type: mysql.TypeNull, Value: nil}, + }, }, { CommitTs: 2, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("bb")}}, + Columns: []*model.Column{ + {Name: "varchar2", Type: mysql.TypeVarchar, Value: []byte("varchar")}, + {Name: "string2", Type: mysql.TypeString, Value: []byte("string")}, + {Name: "date2", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp2", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime2", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float2", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long2", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null2", Type: mysql.TypeNull, Value: nil}, + }, }, { CommitTs: 3, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("bb")}}, + Columns: []*model.Column{ + {Name: "varchar3", Type: mysql.TypeVarchar, Value: []byte("varchar")}, + {Name: "string3", Type: mysql.TypeString, Value: []byte("string")}, + {Name: "date3", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp3", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime3", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float3", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long3", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null3", Type: mysql.TypeNull, Value: nil}, + }, }, { CommitTs: 4, Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, - Columns: []*model.Column{{Name: "col1", Type: mysql.TypeVarchar, Value: []byte("cc")}}, + Columns: []*model.Column{ + {Name: "varchar4", Type: mysql.TypeVarchar, Value: []byte("varchar")}, + {Name: "string4", Type: mysql.TypeString, Value: []byte("string")}, + {Name: "date4", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp4", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime4", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float4", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long4", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null4", Type: mysql.TypeNull, Value: nil}, + }, }}, {}} codecDDLCases = [][]*model.DDLEvent{{{ diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft_decoder.go index 9b61eb8e4a0..462750ab39b 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft_decoder.go @@ -314,7 +314,7 @@ func decodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ _, u64, err := decodeUvarint(bits) return u64, err } - _, i64, err := decodeUvarint(bits) + _, i64, err := decodeVarint(bits) return i64, err case mysql.TypeUnspecified: fallthrough diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index 66f056f6898..8459e144a7f 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -141,7 +141,7 @@ func (s *craftBatchSuite) TestParamsEdgeCases(c *check.C) { encoder := NewCraftEventBatchEncoder().(*CraftEventBatchEncoder) err := encoder.SetParams(map[string]string{}) c.Assert(err, check.IsNil) - c.Assert(encoder.maxBatchSize, check.Equals, 4096) + c.Assert(encoder.maxBatchSize, check.Equals, DefaultMaxBatchSize) c.Assert(encoder.maxMessageSize, check.Equals, 64*1024*1024) err = encoder.SetParams(map[string]string{"max-message-bytes": "0"}) @@ -152,7 +152,7 @@ func (s *craftBatchSuite) TestParamsEdgeCases(c *check.C) { err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxInt32)}) c.Assert(err, check.IsNil) - c.Assert(encoder.maxBatchSize, check.Equals, 4096) + c.Assert(encoder.maxBatchSize, check.Equals, DefaultMaxBatchSize) c.Assert(encoder.maxMessageSize, check.Equals, math.MaxInt32) err = encoder.SetParams(map[string]string{"max-message-bytes": strconv.Itoa(math.MaxUint32)}) diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index fe618e877f7..ff439d3d8e5 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -112,6 +112,26 @@ func formatColumnVal(c column) column { log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) } } + case mysql.TypeFloat, mysql.TypeDouble: + if s, ok := c.Value.(json.Number); ok { + f64, err := s.Float64() + if err != nil { + log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) + } + c.Value = f64 + } + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: + 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)) + } + if c.Flag.IsUnsigned() { + c.Value = uint64(intNum) + } else { + c.Value = intNum + } + } case mysql.TypeBit: if s, ok := c.Value.(json.Number); ok { intNum, err := s.Int64() diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 4efef41f41b..5c898227a21 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -15,6 +15,7 @@ package codec import ( "math" + "sort" "strconv" "testing" @@ -38,6 +39,28 @@ var _ = check.Suite(&batchSuite{ resolvedTsCases: codecResolvedTSCases, }) +type columnsArray []*model.Column + +func (a columnsArray) Len() int { + return len(a) +} + +func (a columnsArray) Less(i, j int) bool { + return a[i].Name < a[j].Name +} + +func (a columnsArray) Swap(i, j int) { + a[i], a[j] = a[j], a[i] +} + +func sortColumnsArrays(arrays ...[]*model.Column) { + for _, array := range arrays { + if array != nil { + sort.Sort(columnsArray(array)) + } + } +} + func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEncoder, newDecoder func(key []byte, value []byte) (EventBatchDecoder, error)) { checkRowDecoder := func(decoder EventBatchDecoder, cs []*model.RowChangedEvent) { index := 0 @@ -50,6 +73,7 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco c.Assert(tp, check.Equals, model.MqMessageTypeRow) row, err := decoder.NextRowChangedEvent() c.Assert(err, check.IsNil) + sortColumnsArrays(row.Columns, row.PreColumns, cs[index].Columns, cs[index].PreColumns) c.Assert(row, check.DeepEquals, cs[index]) index++ } From 51d798078a2488a6f5b9a033d347fe6390a9e283 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 20 Apr 2021 13:20:07 +0800 Subject: [PATCH 14/39] Add protobuf vs craft benchmark related code Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 377 ++- cdc/sink/codec/craft.go | 103 +- cdc/sink/codec/craft/buffer.go | 488 ++++ cdc/sink/codec/craft/codec_test.go | 101 + .../{craft_decoder.go => craft/decoder.go} | 167 +- .../{craft_encoder.go => craft/encoder.go} | 154 +- cdc/sink/codec/craft/model.go | 351 +++ cdc/sink/codec/craft_model.go | 361 --- cdc/sink/codec/craft_test.go | 76 - proto/CraftBenchmark.proto | 62 + proto/benchmark/CraftBenchmark.pb.go | 2593 +++++++++++++++++ proto/generate-proto.sh | 4 +- 12 files changed, 4178 insertions(+), 659 deletions(-) create mode 100644 cdc/sink/codec/craft/buffer.go create mode 100644 cdc/sink/codec/craft/codec_test.go rename cdc/sink/codec/{craft_decoder.go => craft/decoder.go} (66%) rename cdc/sink/codec/{craft_encoder.go => craft/encoder.go} (58%) create mode 100644 cdc/sink/codec/craft/model.go delete mode 100644 cdc/sink/codec/craft_model.go create mode 100644 proto/CraftBenchmark.proto create mode 100644 proto/benchmark/CraftBenchmark.pb.go diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index b03cec02777..0eee4510d33 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -17,16 +17,19 @@ import ( "bytes" "compress/zlib" "fmt" + "testing" "github.com/pingcap/check" "github.com/pingcap/parser/mysql" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/codec/craft" "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/ticdc/proto/benchmark" ) var ( codecRowCases = [][]*model.RowChangedEvent{{{ - CommitTs: 1, + CommitTs: 424316552636792833, Table: &model.TableName{Schema: "a", Table: "b"}, Columns: []*model.Column{ {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar")}, @@ -39,7 +42,7 @@ var ( {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }}, {{ - CommitTs: 1, + CommitTs: 424316553934667777, Table: &model.TableName{Schema: "a", Table: "b"}, Columns: []*model.Column{ {Name: "varchar1", Type: mysql.TypeVarchar, Value: []byte("varchar")}, @@ -52,7 +55,7 @@ var ( {Name: "null1", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 2, + CommitTs: 424316554327097345, Table: &model.TableName{Schema: "a", Table: "b"}, Columns: []*model.Column{ {Name: "varchar2", Type: mysql.TypeVarchar, Value: []byte("varchar")}, @@ -65,7 +68,7 @@ var ( {Name: "null2", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 3, + CommitTs: 424316554746789889, Table: &model.TableName{Schema: "a", Table: "b"}, Columns: []*model.Column{ {Name: "varchar3", Type: mysql.TypeVarchar, Value: []byte("varchar")}, @@ -78,7 +81,7 @@ var ( {Name: "null3", Type: mysql.TypeNull, Value: nil}, }, }, { - CommitTs: 4, + CommitTs: 424316555073945601, Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, Columns: []*model.Column{ {Name: "varchar4", Type: mysql.TypeVarchar, Value: []byte("varchar")}, @@ -93,28 +96,28 @@ var ( }}, {}} codecDDLCases = [][]*model.DDLEvent{{{ - CommitTs: 1, + CommitTs: 424316555979653121, TableInfo: &model.SimpleTableInfo{ Schema: "a", Table: "b", }, Query: "create table a", Type: 1, }}, {{ - CommitTs: 1, + CommitTs: 424316583965360129, TableInfo: &model.SimpleTableInfo{ Schema: "a", Table: "b", }, Query: "create table a", Type: 1, }, { - CommitTs: 2, + CommitTs: 424316586087940097, TableInfo: &model.SimpleTableInfo{ Schema: "a", Table: "b", }, Query: "create table b", Type: 2, }, { - CommitTs: 3, + CommitTs: 424316588736118785, TableInfo: &model.SimpleTableInfo{ Schema: "a", Table: "b", }, @@ -122,7 +125,16 @@ var ( Type: 3, }}, {}} - codecResolvedTSCases = [][]uint64{{1}, {1, 2, 3}, {}} + codecResolvedTSCases = [][]uint64{{424316592563683329}, {424316594097225729, 424316594214141953, 424316594345213953}, {}} + + codecBenchmarkRowChanges = codecRowCases[1] + + codecCraftEncodedRowChanges = []*MQMessage{} + codecJSONEncodedRowChanges = []*MQMessage{} + codecPB1EncodedRowChanges = []*MQMessage{} + codecPB2EncodedRowChanges = []*MQMessage{} + + codecTestSliceAllocator = craft.NewSliceAllocator(512) ) var _ = check.Suite(&codecTestSuite{}) @@ -145,32 +157,345 @@ func (s *codecTestSuite) checkCompressedSize(messages []*MQMessage) (int, int) { } func (s *codecTestSuite) encodeRowCase(c *check.C, encoder EventBatchEncoder, events []*model.RowChangedEvent) []*MQMessage { - err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + msg, err := codecEncodeRowCase(encoder, events) c.Assert(err, check.IsNil) - - for _, event := range events { - op, err := encoder.AppendRowChangedEvent(event) - c.Assert(err, check.IsNil) - c.Assert(op, check.Equals, EncoderNoOperation) - } - - if len(events) > 0 { - return encoder.Build() - } - return nil + return msg } -func (s *codecTestSuite) TestJsonVsCraft(c *check.C) { +func (s *codecTestSuite) TestJsonVsCraftVsPB(c *check.C) { defer testleak.AfterTest(c)() - fmt.Println("| index | craft size | json size | craft compressed | json compressed |") - fmt.Println("| ----- | ---------- | --------- | ---------------- | --------------- |") + fmt.Println("| case | craft size | json size | protobuf 1 size | protobuf 2 size | craft compressed | json compressed | protobuf 1 compressed | protobuf 2 compressed |") + fmt.Println("| :---- | :--------- | :-------- | :-------------- | :-------------- | :--------------- | :-------------- | :-------------------- | :-------------------- |") for i, cs := range codecRowCases { + if len(cs) == 0 { + continue + } craftEncoder := NewCraftEventBatchEncoder() jsonEncoder := NewJSONEventBatchEncoder() craftMessages := s.encodeRowCase(c, craftEncoder, cs) jsonMessages := s.encodeRowCase(c, jsonEncoder, cs) + protobuf1Messages := codecEncodeRowChangedPB1ToMessage(cs) + protobuf2Messages := codecEncodeRowChangedPB2ToMessage(cs) craftOriginal, craftCompressed := s.checkCompressedSize(craftMessages) jsonOriginal, jsonCompressed := s.checkCompressedSize(jsonMessages) - fmt.Printf("| %d | %d | %d | %d | %d |\n", i, craftOriginal, jsonOriginal, craftCompressed, jsonCompressed) + protobuf1Original, protobuf1Compressed := s.checkCompressedSize(protobuf1Messages) + protobuf2Original, protobuf2Compressed := s.checkCompressedSize(protobuf2Messages) + fmt.Printf("| case %d | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ |\n", i, + craftOriginal, jsonOriginal, 100*jsonOriginal/craftOriginal-100, + protobuf1Original, 100*protobuf1Original/craftOriginal-100, + protobuf2Original, 100*protobuf2Original/craftOriginal-100, + craftCompressed, jsonCompressed, 100*jsonCompressed/craftCompressed-100, + protobuf1Compressed, 100*protobuf1Compressed/craftCompressed-100, + protobuf2Compressed, 100*protobuf2Compressed/craftCompressed-100) + } +} + +func codecEncodeKeyPB(event *model.RowChangedEvent) []byte { + key := &benchmark.Key{ + Ts: event.CommitTs, + Schema: event.Table.Schema, + Table: event.Table.Table, + RowId: event.RowID, + Partition: 0, + } + if b, err := key.Marshal(); err != nil { + panic(err) + } else { + return b + } +} + +func codecEncodeColumnPB(column *model.Column) *benchmark.Column { + return &benchmark.Column{ + Name: column.Name, + Type: uint32(column.Type), + Flag: uint32(column.Flag), + Value: craft.EncodeTiDBType(codecTestSliceAllocator, column.Type, column.Flag, column.Value), + } +} + +func codecEncodeColumnsPB(columns []*model.Column) []*benchmark.Column { + converted := make([]*benchmark.Column, len(columns)) + for i, column := range columns { + converted[i] = codecEncodeColumnPB(column) + } + return converted +} + +func codecEncodeRowChangedPB(event *model.RowChangedEvent) []byte { + rowChanged := &benchmark.RowChanged{ + OldValue: codecEncodeColumnsPB(event.PreColumns), + NewValue: codecEncodeColumnsPB(event.Columns), + } + if b, err := rowChanged.Marshal(); err != nil { + panic(err) + } else { + return b + } +} + +func codecEncodeRowChangedPB1ToMessage(events []*model.RowChangedEvent) []*MQMessage { + result := make([]*MQMessage, len(events)) + for i, event := range events { + result[i] = &MQMessage{ + Key: codecEncodeKeyPB(event), + Value: codecEncodeRowChangedPB(event), + } + } + return result +} + +func codecEncodeRowChangedPB2ToMessage(events []*model.RowChangedEvent) []*MQMessage { + return []*MQMessage{&MQMessage{ + Key: codecEncodeKeysPB2(events), + Value: codecEncodeRowChangedPB2(events), + }} +} + +func codecEncodeKeysPB2(events []*model.RowChangedEvent) []byte { + converted := &benchmark.KeysColumnar{} + + for _, event := range events { + converted.Ts = append(converted.Ts, event.CommitTs) + converted.Schema = append(converted.Schema, event.Table.Schema) + converted.Table = append(converted.Table, event.Table.Table) + converted.RowId = append(converted.RowId, event.RowID) + converted.Partition = append(converted.Partition, 0) + } + + if b, err := converted.Marshal(); err != nil { + panic(err) + } else { + return b + } +} + +func codecEncodeColumnsPB2(columns []*model.Column) *benchmark.ColumnsColumnar { + converted := &benchmark.ColumnsColumnar{ + Name: make([]string, len(columns)), + Type: make([]uint32, len(columns)), + Flag: make([]uint32, len(columns)), + Value: make([][]byte, len(columns)), + } + for i, column := range columns { + converted.Name[i] = column.Name + converted.Type[i] = uint32(column.Type) + converted.Flag[i] = uint32(column.Flag) + converted.Value[i] = craft.EncodeTiDBType(codecTestSliceAllocator, column.Type, column.Flag, column.Value) + } + return converted +} + +func codecEncodeRowChangedPB2(events []*model.RowChangedEvent) []byte { + rowChanged := &benchmark.RowChangedColumnar{} + for _, event := range events { + rowChanged.OldValue = append(rowChanged.OldValue, codecEncodeColumnsPB2(event.PreColumns)) + rowChanged.NewValue = append(rowChanged.NewValue, codecEncodeColumnsPB2(event.Columns)) + } + if b, err := rowChanged.Marshal(); err != nil { + panic(err) + } else { + return b + } +} + +func codecEncodeRowCase(encoder EventBatchEncoder, events []*model.RowChangedEvent) ([]*MQMessage, error) { + err := encoder.SetParams(map[string]string{"max-message-bytes": "8192", "max-batch-size": "64"}) + if err != nil { + return nil, err + } + + for _, event := range events { + _, err := encoder.AppendRowChangedEvent(event) + if err != nil { + return nil, err + } + } + + if len(events) > 0 { + return encoder.Build(), nil + } + return nil, nil +} + +func init() { + var err error + if codecCraftEncodedRowChanges, err = codecEncodeRowCase(NewCraftEventBatchEncoder(), codecBenchmarkRowChanges); err != nil { + panic(err) + } + if codecJSONEncodedRowChanges, err = codecEncodeRowCase(NewJSONEventBatchEncoder(), codecBenchmarkRowChanges); err != nil { + panic(err) + } + codecPB1EncodedRowChanges = codecEncodeRowChangedPB1ToMessage(codecBenchmarkRowChanges) + codecPB2EncodedRowChanges = codecEncodeRowChangedPB2ToMessage(codecBenchmarkRowChanges) +} + +func BenchmarkCraftEncoding(b *testing.B) { + allocator := craft.NewSliceAllocator(128) + for i := 0; i < b.N; i++ { + _, _ = codecEncodeRowCase(NewCraftEventBatchEncoderWithAllocator(allocator), codecBenchmarkRowChanges) + } +} + +func BenchmarkJsonEncoding(b *testing.B) { + for i := 0; i < b.N; i++ { + _, _ = codecEncodeRowCase(NewJSONEventBatchEncoder(), codecBenchmarkRowChanges) + } +} + +func BenchmarkProtobuf1Encoding(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = codecEncodeRowChangedPB1ToMessage(codecBenchmarkRowChanges) + } +} + +func BenchmarkProtobuf2Encoding(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = codecEncodeRowChangedPB2ToMessage(codecBenchmarkRowChanges) + } +} + +func BenchmarkCraftDecoding(b *testing.B) { + allocator := craft.NewSliceAllocator(128) + for i := 0; i < b.N; i++ { + for _, message := range codecCraftEncodedRowChanges { + if decoder, err := NewCraftEventBatchDecoderWithAllocator(message.Value, allocator); err != nil { + panic(err) + } else { + for { + if _, hasNext, err := decoder.HasNext(); err != nil { + panic(err) + } else if hasNext { + _, _ = decoder.NextRowChangedEvent() + } else { + break + } + } + } + } + } +} + +func BenchmarkJsonDecoding(b *testing.B) { + for i := 0; i < b.N; i++ { + for _, message := range codecJSONEncodedRowChanges { + if decoder, err := NewJSONEventBatchDecoder(message.Key, message.Value); err != nil { + panic(err) + } else { + for { + if _, hasNext, err := decoder.HasNext(); err != nil { + panic(err) + } else if hasNext { + _, _ = decoder.NextRowChangedEvent() + } else { + break + } + } + } + } + } +} + +func codecDecodeRowChangedPB1(columns []*benchmark.Column) []*model.Column { + if len(columns) == 0 { + return nil + } + + result := make([]*model.Column, len(columns)) + for i, column := range columns { + value, _ := craft.DecodeTiDBType(byte(column.Type), model.ColumnFlagType(column.Flag), column.Value) + result[i] = &model.Column{ + Name: column.Name, + Type: byte(column.Type), + Flag: model.ColumnFlagType(column.Flag), + Value: value, + } + } + + return result +} + +func benchmarkProtobuf1Decoding() []*model.RowChangedEvent { + result := make([]*model.RowChangedEvent, 0, 4) + for _, message := range codecPB1EncodedRowChanges { + key := &benchmark.Key{} + key.Unmarshal(message.Key) + value := &benchmark.RowChanged{} + value.Unmarshal(message.Value) + ev := &model.RowChangedEvent{} + ev.PreColumns = codecDecodeRowChangedPB1(value.OldValue) + ev.Columns = codecDecodeRowChangedPB1(value.NewValue) + ev.CommitTs = key.Ts + ev.Table = &model.TableName{ + Schema: key.Schema, + Table: key.Table, + } + if key.Partition >= 0 { + ev.Table.TableID = key.Partition + ev.Table.IsPartition = true + } + result = append(result, ev) + } + return result +} + +func BenchmarkProtobuf1Decoding(b *testing.B) { + for i := 0; i < b.N; i++ { + for _, row := range benchmarkProtobuf1Decoding() { + _ = row + } + } +} + +func codecDecodeRowChangedPB2(columns *benchmark.ColumnsColumnar) []*model.Column { + result := make([]*model.Column, len(columns.Value)) + for i, value := range columns.Value { + v, _ := craft.DecodeTiDBType(byte(columns.Type[i]), model.ColumnFlagType(columns.Flag[i]), value) + result[i] = &model.Column{ + Name: columns.Name[i], + Type: byte(columns.Type[i]), + Flag: model.ColumnFlagType(columns.Flag[i]), + Value: v, + } + } + return result +} + +func benchmarkProtobuf2Decoding() []*model.RowChangedEvent { + result := make([]*model.RowChangedEvent, 0, 4) + for _, message := range codecPB2EncodedRowChanges { + keys := &benchmark.KeysColumnar{} + keys.Unmarshal(message.Key) + values := &benchmark.RowChangedColumnar{} + values.Unmarshal(message.Value) + + for i, ts := range keys.Ts { + ev := &model.RowChangedEvent{} + if len(values.OldValue) > i { + ev.PreColumns = codecDecodeRowChangedPB2(values.OldValue[i]) + } + if len(values.NewValue) > i { + ev.Columns = codecDecodeRowChangedPB2(values.NewValue[i]) + } + ev.CommitTs = ts + ev.Table = &model.TableName{ + Schema: keys.Schema[i], + Table: keys.Table[i], + } + if keys.Partition[i] >= 0 { + ev.Table.TableID = keys.Partition[i] + ev.Table.IsPartition = true + } + result = append(result, ev) + } + } + return result +} + +func BenchmarkProtobuf2Decoding(b *testing.B) { + for i := 0; i < b.N; i++ { + for _, row := range benchmarkProtobuf2Decoding() { + _ = row + } } } diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index fc554e63a3e..8de60577d25 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -19,53 +19,43 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/ticdc/cdc/sink/codec/craft" cerror "github.com/pingcap/ticdc/pkg/errors" ) const ( // CraftVersion1 represents the version of craft format CraftVersion1 uint64 = 1 - - // default buffer size - craftDefaultBufferCapacity = 512 - - // Column group types - craftColumnGroupTypeDelete = 0x3 - craftColumnGroupTypeOld = 0x2 - craftColumnGroupTypeNew = 0x1 - - // Size tables index - craftKeySizeTableIndex = 0 - craftValueSizeTableIndex = 1 - craftColumnGroupSizeTableStartIndex = 2 ) // CraftEventBatchEncoder encodes the events into the byte of a batch into craft binary format. type CraftEventBatchEncoder struct { - rowChangedBuffer *craftRowChangedEventBuffer + rowChangedBuffer *craft.RowChangedEventBuffer messageBuf []*MQMessage // configs maxMessageSize int maxBatchSize int + + allocator *craft.SliceAllocator } // EncodeCheckpointEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { - return newResolvedMQMessage(ProtocolCraft, nil, newCraftResolvedEventEncoder(ts).encode(), ts), nil + return newResolvedMQMessage(ProtocolCraft, nil, craft.NewResolvedEventEncoder(e.allocator, ts).Encode(), ts), nil } func (e *CraftEventBatchEncoder) flush() { - keys := e.rowChangedBuffer.getKeys() - ts := keys.getTs(0) - schema := keys.getSchema(0) - table := keys.getTable(0) - e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.encode(), ts, model.MqMessageTypeRow, &schema, &table)) + headers := e.rowChangedBuffer.GetHeaders() + ts := headers.GetTs(0) + schema := headers.GetSchema(0) + table := headers.GetTable(0) + e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table)) } // AppendRowChangedEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) AppendRowChangedEvent(ev *model.RowChangedEvent) (EncoderResult, error) { - rows, size := e.rowChangedBuffer.appendRowChangedEvent(ev) + rows, size := e.rowChangedBuffer.AppendRowChangedEvent(ev) if size > e.maxMessageSize || rows >= e.maxBatchSize { e.flush() } @@ -79,17 +69,17 @@ func (e *CraftEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, // EncodeDDLEvent implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) EncodeDDLEvent(ev *model.DDLEvent) (*MQMessage, error) { - return newDDLMQMessage(ProtocolCraft, nil, newCraftDDLEventEncoder(ev).encode(), ev), nil + return newDDLMQMessage(ProtocolCraft, nil, craft.NewDDLEventEncoder(e.allocator, ev).Encode(), ev), nil } // Build implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Build() []*MQMessage { - if e.rowChangedBuffer.size() > 0 { + if e.rowChangedBuffer.Size() > 0 { // flush buffered data to message buffer e.flush() } ret := e.messageBuf - e.messageBuf = make([]*MQMessage, 0) + e.messageBuf = make([]*MQMessage, 0, 2) return ret } @@ -100,12 +90,12 @@ func (e *CraftEventBatchEncoder) MixedBuild(withVersion bool) []byte { // Size implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Size() int { - return e.rowChangedBuffer.size() + return e.rowChangedBuffer.Size() } // Reset implements the EventBatchEncoder interface func (e *CraftEventBatchEncoder) Reset() { - e.rowChangedBuffer.reset() + e.rowChangedBuffer.Reset() } // SetParams reads relevant parameters for craft protocol @@ -141,26 +131,32 @@ func (e *CraftEventBatchEncoder) SetParams(params map[string]string) error { // NewCraftEventBatchEncoder creates a new CraftEventBatchEncoder. func NewCraftEventBatchEncoder() EventBatchEncoder { + return NewCraftEventBatchEncoderWithAllocator(craft.NewSliceAllocator(64)) +} + +func NewCraftEventBatchEncoderWithAllocator(allocator *craft.SliceAllocator) EventBatchEncoder { return &CraftEventBatchEncoder{ - rowChangedBuffer: &craftRowChangedEventBuffer{ - keys: &craftColumnarKeys{}, - }, + allocator: allocator, + messageBuf: make([]*MQMessage, 0, 2), + rowChangedBuffer: craft.NewRowChangedEventBuffer(allocator), } } // CraftEventBatchDecoder decodes the byte of a batch into the original messages. type CraftEventBatchDecoder struct { - keys *craftColumnarKeys - decoder *craftMessageDecoder + headers *craft.Headers + decoder *craft.MessageDecoder index int + + allocator *craft.SliceAllocator } // HasNext implements the EventBatchDecoder interface func (b *CraftEventBatchDecoder) HasNext() (model.MqMessageType, bool, error) { - if b.index >= b.keys.count { + if b.index >= b.headers.Count() { return model.MqMessageTypeUnknown, false, nil } - return b.keys.getType(b.index), true, nil + return b.headers.GetType(b.index), true, nil } // NextResolvedEvent implements the EventBatchDecoder interface @@ -172,7 +168,7 @@ func (b *CraftEventBatchDecoder) NextResolvedEvent() (uint64, error) { if !hasNext || ty != model.MqMessageTypeResolved { return 0, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") } - ts := b.keys.getTs(b.index) + ts := b.headers.GetTs(b.index) b.index++ return ts, nil } @@ -184,29 +180,29 @@ func (b *CraftEventBatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, return nil, errors.Trace(err) } if !hasNext || ty != model.MqMessageTypeRow { - return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found row changed event message") } - old, new, err := b.decoder.decodeRowChangedEvent(b.index) + old, new, err := b.decoder.RowChangedEvent(b.index) if err != nil { return nil, errors.Trace(err) } ev := &model.RowChangedEvent{} if old != nil { - if ev.PreColumns, err = old.toModel(); err != nil { + if ev.PreColumns, err = old.ToModel(); err != nil { return nil, errors.Trace(err) } } if new != nil { - if ev.Columns, err = new.toModel(); err != nil { + if ev.Columns, err = new.ToModel(); err != nil { return nil, errors.Trace(err) } } - ev.CommitTs = b.keys.getTs(b.index) + ev.CommitTs = b.headers.GetTs(b.index) ev.Table = &model.TableName{ - Schema: b.keys.getSchema(b.index), - Table: b.keys.getTable(b.index), + Schema: b.headers.GetSchema(b.index), + Table: b.headers.GetTable(b.index), } - partition := b.keys.getPartition(b.index) + partition := b.headers.GetPartition(b.index) if partition >= 0 { ev.Table.TableID = partition ev.Table.IsPartition = true @@ -222,19 +218,19 @@ func (b *CraftEventBatchDecoder) NextDDLEvent() (*model.DDLEvent, error) { return nil, errors.Trace(err) } if !hasNext || ty != model.MqMessageTypeDDL { - return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found resolved event message") + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found ddl event message") } - ddlType, query, err := b.decoder.decodeDDLEvent(b.index) + ddlType, query, err := b.decoder.DDLEvent(b.index) if err != nil { return nil, errors.Trace(err) } event := &model.DDLEvent{ - CommitTs: b.keys.getTs(b.index), + CommitTs: b.headers.GetTs(b.index), Query: query, Type: ddlType, TableInfo: &model.SimpleTableInfo{ - Schema: b.keys.getSchema(b.index), - Table: b.keys.getTable(b.index), + Schema: b.headers.GetSchema(b.index), + Table: b.headers.GetTable(b.index), }, } b.index++ @@ -243,17 +239,22 @@ func (b *CraftEventBatchDecoder) NextDDLEvent() (*model.DDLEvent, error) { // NewCraftEventBatchDecoder creates a new CraftEventBatchDecoder. func NewCraftEventBatchDecoder(bits []byte) (EventBatchDecoder, error) { - decoder, err := newCraftMessageDecoder(bits) + return NewCraftEventBatchDecoderWithAllocator(bits, craft.NewSliceAllocator(64)) +} + +func NewCraftEventBatchDecoderWithAllocator(bits []byte, allocator *craft.SliceAllocator) (EventBatchDecoder, error) { + decoder, err := craft.NewMessageDecoder(bits, allocator) if err != nil { return nil, errors.Trace(err) } - keys, err := decoder.decodeKeys() + headers, err := decoder.Headers() if err != nil { return nil, errors.Trace(err) } return &CraftEventBatchDecoder{ - keys: keys, - decoder: decoder, + headers: headers, + decoder: decoder, + allocator: allocator, }, nil } diff --git a/cdc/sink/codec/craft/buffer.go b/cdc/sink/codec/craft/buffer.go new file mode 100644 index 00000000000..a90b8e1b8a2 --- /dev/null +++ b/cdc/sink/codec/craft/buffer.go @@ -0,0 +1,488 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package craft + +/// Utility functions for buffer allocation +func newBufferSize(oldSize int) int { + var newSize int + if oldSize > 128 { + newSize = oldSize + 128 + } else { + if oldSize > 0 { + newSize = oldSize * 2 + } else { + newSize = 8 + } + } + return newSize +} + +// int slice allocator +type intSliceAllocator struct { + buffer []int + offset int +} + +func (b *intSliceAllocator) realloc(old []int, newSize int) []int { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *intSliceAllocator) alloc(size int) []int { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]int, size) + } + b.buffer = make([]int, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *intSliceAllocator) one(x int) []int { + r := b.alloc(1) + r[0] = x + return r +} + +func newIntSliceAllocator(batchSize int) *intSliceAllocator { + return &intSliceAllocator{buffer: make([]int, batchSize)} +} + +// int64 slice allocator +type int64SliceAllocator struct { + buffer []int64 + offset int +} + +func (b *int64SliceAllocator) realloc(old []int64, newSize int) []int64 { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *int64SliceAllocator) alloc(size int) []int64 { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]int64, size) + } + b.buffer = make([]int64, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *int64SliceAllocator) one(x int64) []int64 { + r := b.alloc(1) + r[0] = x + return r +} + +func newInt64SliceAllocator(batchSize int) *int64SliceAllocator { + return &int64SliceAllocator{buffer: make([]int64, batchSize)} +} + +// uint64 slice allocator +type uint64SliceAllocator struct { + buffer []uint64 + offset int +} + +func (b *uint64SliceAllocator) realloc(old []uint64, newSize int) []uint64 { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *uint64SliceAllocator) alloc(size int) []uint64 { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]uint64, size) + } + b.buffer = make([]uint64, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *uint64SliceAllocator) one(x uint64) []uint64 { + r := b.alloc(1) + r[0] = x + return r +} + +func newUint64SliceAllocator(batchSize int) *uint64SliceAllocator { + return &uint64SliceAllocator{buffer: make([]uint64, batchSize)} +} + +// string slice allocator +type stringSliceAllocator struct { + buffer []string + offset int +} + +func (b *stringSliceAllocator) realloc(old []string, newSize int) []string { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *stringSliceAllocator) alloc(size int) []string { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]string, size) + } + b.buffer = make([]string, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *stringSliceAllocator) one(x string) []string { + r := b.alloc(1) + r[0] = x + return r +} + +func newStringSliceAllocator(batchSize int) *stringSliceAllocator { + return &stringSliceAllocator{buffer: make([]string, batchSize)} +} + +// nullable string slice allocator +type nullableStringSliceAllocator struct { + buffer []*string + offset int +} + +func (b *nullableStringSliceAllocator) realloc(old []*string, newSize int) []*string { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *nullableStringSliceAllocator) alloc(size int) []*string { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]*string, size) + } + b.buffer = make([]*string, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *nullableStringSliceAllocator) one(x *string) []*string { + r := b.alloc(1) + r[0] = x + return r +} + +func newNullableStringSliceAllocator(batchSize int) *nullableStringSliceAllocator { + return &nullableStringSliceAllocator{buffer: make([]*string, batchSize)} +} + +// byte slice allocator +type byteSliceAllocator struct { + buffer []byte + offset int +} + +func (b *byteSliceAllocator) realloc(old []byte, newSize int) []byte { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *byteSliceAllocator) alloc(size int) []byte { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]byte, size) + } + b.buffer = make([]byte, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *byteSliceAllocator) one(x byte) []byte { + r := b.alloc(1) + r[0] = x + return r +} + +func newByteSliceAllocator(batchSize int) *byteSliceAllocator { + return &byteSliceAllocator{buffer: make([]byte, batchSize)} +} + +// bytes slice allocator +type bytesSliceAllocator struct { + buffer [][]byte + offset int +} + +func (b *bytesSliceAllocator) realloc(old [][]byte, newSize int) [][]byte { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *bytesSliceAllocator) alloc(size int) [][]byte { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([][]byte, size) + } + b.buffer = make([][]byte, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *bytesSliceAllocator) one(x []byte) [][]byte { + r := b.alloc(1) + r[0] = x + return r +} + +func newBytesSliceAllocator(batchSize int) *bytesSliceAllocator { + return &bytesSliceAllocator{buffer: make([][]byte, batchSize)} +} + +// columnGroup slice allocator +type columnGroupSliceAllocator struct { + buffer []*columnGroup + offset int +} + +func (b *columnGroupSliceAllocator) realloc(old []*columnGroup, newSize int) []*columnGroup { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *columnGroupSliceAllocator) alloc(size int) []*columnGroup { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]*columnGroup, size) + } + b.buffer = make([]*columnGroup, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *columnGroupSliceAllocator) one(x *columnGroup) []*columnGroup { + r := b.alloc(1) + r[0] = x + return r +} + +func newColumnGroupSliceAllocator(batchSize int) *columnGroupSliceAllocator { + return &columnGroupSliceAllocator{buffer: make([]*columnGroup, batchSize)} +} + +// rowChangedEvent slice allocator +type rowChangedEventSliceAllocator struct { + buffer []rowChangedEvent + offset int +} + +func (b *rowChangedEventSliceAllocator) realloc(old []rowChangedEvent, newSize int) []rowChangedEvent { + n := b.alloc(newSize) + copy(n, old) + return n +} + +func (b *rowChangedEventSliceAllocator) alloc(size int) []rowChangedEvent { + if len(b.buffer)-b.offset < size { + if size > len(b.buffer)/4 { + // large allocation + return make([]rowChangedEvent, size) + } + b.buffer = make([]rowChangedEvent, len(b.buffer)) + b.offset = 0 + } + result := b.buffer[b.offset : b.offset+size] + b.offset += size + return result +} + +func (b *rowChangedEventSliceAllocator) one(x rowChangedEvent) []rowChangedEvent { + r := b.alloc(1) + r[0] = x + return r +} + +func newRowChangedEventSliceAllocator(batchSize int) *rowChangedEventSliceAllocator { + return &rowChangedEventSliceAllocator{buffer: make([]rowChangedEvent, batchSize)} +} + +// allocator for different slice types +type SliceAllocator struct { + intAllocator *intSliceAllocator + int64Allocator *int64SliceAllocator + uint64Allocator *uint64SliceAllocator + stringAllocator *stringSliceAllocator + nullableStringAllocator *nullableStringSliceAllocator + byteAllocator *byteSliceAllocator + bytesAllocator *bytesSliceAllocator + columnGroupAllocator *columnGroupSliceAllocator + rowChangedEventAllocator *rowChangedEventSliceAllocator +} + +func NewSliceAllocator(batchSize int) *SliceAllocator { + return &SliceAllocator{ + intAllocator: newIntSliceAllocator(batchSize), + int64Allocator: newInt64SliceAllocator(batchSize), + uint64Allocator: newUint64SliceAllocator(batchSize), + stringAllocator: newStringSliceAllocator(batchSize), + nullableStringAllocator: newNullableStringSliceAllocator(batchSize), + byteAllocator: newByteSliceAllocator(batchSize), + bytesAllocator: newBytesSliceAllocator(batchSize), + columnGroupAllocator: newColumnGroupSliceAllocator(batchSize), + rowChangedEventAllocator: newRowChangedEventSliceAllocator(batchSize), + } +} + +func (b *SliceAllocator) intSlice(size int) []int { + return b.intAllocator.alloc(size) +} + +func (b *SliceAllocator) oneIntSlice(x int) []int { + return b.intAllocator.one(x) +} + +func (b *SliceAllocator) resizeIntSlice(old []int, newSize int) []int { + return b.intAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) int64Slice(size int) []int64 { + return b.int64Allocator.alloc(size) +} + +func (b *SliceAllocator) oneInt64Slice(x int64) []int64 { + return b.int64Allocator.one(x) +} + +func (b *SliceAllocator) resizeInt64Slice(old []int64, newSize int) []int64 { + return b.int64Allocator.realloc(old, newSize) +} + +func (b *SliceAllocator) uint64Slice(size int) []uint64 { + return b.uint64Allocator.alloc(size) +} + +func (b *SliceAllocator) oneUint64Slice(x uint64) []uint64 { + return b.uint64Allocator.one(x) +} + +func (b *SliceAllocator) resizeUint64Slice(old []uint64, newSize int) []uint64 { + return b.uint64Allocator.realloc(old, newSize) +} + +func (b *SliceAllocator) stringSlice(size int) []string { + return b.stringAllocator.alloc(size) +} + +func (b *SliceAllocator) oneStringSlice(x string) []string { + return b.stringAllocator.one(x) +} + +func (b *SliceAllocator) resizeStringSlice(old []string, newSize int) []string { + return b.stringAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) nullableStringSlice(size int) []*string { + return b.nullableStringAllocator.alloc(size) +} + +func (b *SliceAllocator) oneNullableStringSlice(x *string) []*string { + return b.nullableStringAllocator.one(x) +} + +func (b *SliceAllocator) resizeNullableStringSlice(old []*string, newSize int) []*string { + return b.nullableStringAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) byteSlice(size int) []byte { + return b.byteAllocator.alloc(size) +} + +func (b *SliceAllocator) oneByteSlice(x byte) []byte { + return b.byteAllocator.one(x) +} + +func (b *SliceAllocator) resizeByteSlice(old []byte, newSize int) []byte { + return b.byteAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) bytesSlice(size int) [][]byte { + return b.bytesAllocator.alloc(size) +} + +func (b *SliceAllocator) oneBytesSlice(x []byte) [][]byte { + return b.bytesAllocator.one(x) +} + +func (b *SliceAllocator) resizeBytesSlice(old [][]byte, newSize int) [][]byte { + return b.bytesAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) columnGroupSlice(size int) []*columnGroup { + return b.columnGroupAllocator.alloc(size) +} + +func (b *SliceAllocator) oneColumnGroupSlice(x *columnGroup) []*columnGroup { + return b.columnGroupAllocator.one(x) +} + +func (b *SliceAllocator) resizeColumnGroupSlice(old []*columnGroup, newSize int) []*columnGroup { + return b.columnGroupAllocator.realloc(old, newSize) +} + +func (b *SliceAllocator) rowChangedEventSlice(size int) []rowChangedEvent { + return b.rowChangedEventAllocator.alloc(size) +} + +func (b *SliceAllocator) oneRowChangedEventSlice(x rowChangedEvent) []rowChangedEvent { + return b.rowChangedEventAllocator.one(x) +} + +func (b *SliceAllocator) resizeRowChangedEventSlice(old []rowChangedEvent, newSize int) []rowChangedEvent { + return b.rowChangedEventAllocator.realloc(old, newSize) +} diff --git a/cdc/sink/codec/craft/codec_test.go b/cdc/sink/codec/craft/codec_test.go new file mode 100644 index 00000000000..f0c1cf3528f --- /dev/null +++ b/cdc/sink/codec/craft/codec_test.go @@ -0,0 +1,101 @@ +// Copyright 2021 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package craft + +import ( + "math/rand" + "testing" + + "github.com/pingcap/check" + "github.com/pingcap/ticdc/pkg/util/testleak" +) + +var _ = check.Suite(&codecSuite{allocator: NewSliceAllocator(64)}) + +func Test(t *testing.T) { check.TestingT(t) } + +type codecSuite struct { + allocator *SliceAllocator +} + +func (s *codecSuite) TestSizeTable(c *check.C) { + defer testleak.AfterTest(c)() + tables := [][]uint64{ + { + 1, 3, 5, 7, 9, + }, + { + 2, 4, 6, 8, 10, + }, + } + bits := make([]byte, 16) + rand.Read(bits) + bits = encodeSizeTables(bits, tables) + + size, decoded, err := decodeSizeTables(bits, s.allocator) + c.Check(err, check.IsNil) + c.Check(decoded, check.DeepEquals, tables) + c.Check(size, check.Equals, len(bits)-16) +} + +func (s *codecSuite) TestUvarintReverse(c *check.C) { + defer testleak.AfterTest(c)() + + var i uint64 = 0 + + for i < 0x8000000000000000 { + bits := make([]byte, 16) + rand.Read(bits) + bits, bytes1 := encodeUvarintReversed(bits, i) + bytes2, u64, err := decodeUvarintReversed(bits) + c.Check(err, check.IsNil) + c.Check(u64, check.Equals, i) + c.Check(bytes1, check.Equals, len(bits)-16) + c.Check(bytes1, check.Equals, bytes2) + if i == 0 { + i = 1 + } else { + i <<= 1 + } + } +} + +func newNullableString(a string) *string { + return &a +} + +func (s *codecSuite) TestEncodeChunk(c *check.C) { + defer testleak.AfterTest(c)() + stringChunk := []string{"a", "b", "c"} + nullableStringChunk := []*string{newNullableString("a"), newNullableString("b"), newNullableString("c")} + int64Chunk := []int64{1, 2, 3} + + bits := encodeStringChunk(nil, stringChunk) + bits, decodedStringChunk, err := decodeStringChunk(bits, 3, s.allocator) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedStringChunk, check.DeepEquals, stringChunk) + + bits = encodeNullableStringChunk(nil, nullableStringChunk) + bits, decodedNullableStringChunk, err := decodeNullableStringChunk(bits, 3, s.allocator) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedNullableStringChunk, check.DeepEquals, nullableStringChunk) + + bits = encodeVarintChunk(nil, int64Chunk) + bits, decodedVarintChunk, err := decodeVarintChunk(bits, 3, s.allocator) + c.Check(err, check.IsNil) + c.Check(len(bits), check.Equals, 0) + c.Check(decodedVarintChunk, check.DeepEquals, int64Chunk) +} diff --git a/cdc/sink/codec/craft_decoder.go b/cdc/sink/codec/craft/decoder.go similarity index 66% rename from cdc/sink/codec/craft_decoder.go rename to cdc/sink/codec/craft/decoder.go index 462750ab39b..a1b3acc50e6 100644 --- a/cdc/sink/codec/craft_decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -11,11 +11,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package codec +package craft import ( "encoding/binary" "math" + "unsafe" "github.com/pingcap/errors" pmodel "github.com/pingcap/parser/model" @@ -24,6 +25,11 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" ) +/// create string from byte slice without copying +func unsafeBytesToString(b []byte) string { + return *(*string)(unsafe.Pointer(&b)) +} + /// Primitive type decoders func decodeUint8(bits []byte) ([]byte, byte, error) { if len(bits) < 1 { @@ -131,45 +137,64 @@ func decodeBytes(bits []byte) ([]byte, []byte, error) { func decodeString(bits []byte) ([]byte, string, error) { bits, bytes, err := decodeBytes(bits) if err == nil { - return bits, string(bytes), nil + return bits, unsafeBytesToString(bytes), nil } return bits, "", errors.Trace(err) } /// Chunk decoders -func decodeStringChunk(bits []byte, size int) ([]byte, []string, error) { - newBits, data, err := decodeBytesChunk(bits, size) - if err != nil { - return bits, nil, errors.Trace(err) +func decodeStringChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []string, error) { + larray := allocator.intSlice(size) + newBits := bits + var bl int + var err error + for i := 0; i < size; i++ { + newBits, bl, err = decodeUvarintLength(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + larray[i] = bl } - result := make([]string, size) - for i, d := range data { - result[i] = string(d) + + data := allocator.stringSlice(size) + for i := 0; i < size; i++ { + data[i] = unsafeBytesToString(newBits[:larray[i]]) + newBits = newBits[larray[i]:] } - return newBits, result, nil + return newBits, data, nil } -func decodeNullableStringChunk(bits []byte, size int) ([]byte, []*string, error) { - newBits, data, err := decodeNullableBytesChunk(bits, size) - if err != nil { - return bits, nil, errors.Trace(err) +func decodeNullableStringChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []*string, error) { + larray := allocator.intSlice(size) + newBits := bits + var bl int + var err error + for i := 0; i < size; i++ { + newBits, bl, err = decodeVarintLength(newBits) + if err != nil { + return bits, nil, errors.Trace(err) + } + larray[i] = bl } - result := make([]*string, size) - for i, d := range data { - if d != nil { - s := string(d) - result[i] = &s + + data := allocator.nullableStringSlice(size) + for i := 0; i < size; i++ { + if larray[i] == -1 { + continue } + s := unsafeBytesToString(newBits[:larray[i]]) + data[i] = &s + newBits = newBits[larray[i]:] } - return newBits, result, nil + return newBits, data, nil } -func decodeBytesChunk(bits []byte, size int) ([]byte, [][]byte, error) { - return doDecodeBytesChunk(bits, size, decodeUvarintLength) +func decodeBytesChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, [][]byte, error) { + return doDecodeBytesChunk(bits, size, decodeUvarintLength, allocator) } -func doDecodeBytesChunk(bits []byte, size int, lengthDecoder func([]byte) ([]byte, int, error)) ([]byte, [][]byte, error) { - larray := make([]int, size) +func doDecodeBytesChunk(bits []byte, size int, lengthDecoder func([]byte) ([]byte, int, error), allocator *SliceAllocator) ([]byte, [][]byte, error) { + larray := allocator.intSlice(size) newBits := bits var bl int var err error @@ -181,7 +206,7 @@ func doDecodeBytesChunk(bits []byte, size int, lengthDecoder func([]byte) ([]byt larray[i] = bl } - data := make([][]byte, size) + data := allocator.bytesSlice(size) for i := 0; i < size; i++ { if larray[i] != -1 { data[i] = newBits[:larray[i]] @@ -191,12 +216,12 @@ func doDecodeBytesChunk(bits []byte, size int, lengthDecoder func([]byte) ([]byt return newBits, data, nil } -func decodeNullableBytesChunk(bits []byte, size int) ([]byte, [][]byte, error) { - return doDecodeBytesChunk(bits, size, decodeVarintLength) +func decodeNullableBytesChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, [][]byte, error) { + return doDecodeBytesChunk(bits, size, decodeVarintLength, allocator) } -func decodeVarintChunk(bits []byte, size int) ([]byte, []int64, error) { - array := make([]int64, size) +func decodeVarintChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []int64, error) { + array := allocator.int64Slice(size) newBits := bits var i64 int64 var err error @@ -210,8 +235,8 @@ func decodeVarintChunk(bits []byte, size int) ([]byte, []int64, error) { return newBits, array, nil } -func decodeUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { - array := make([]uint64, size) +func decodeUvarintChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []uint64, error) { + array := allocator.uint64Slice(size) newBits := bits var u64 uint64 var err error @@ -225,8 +250,8 @@ func decodeUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { return newBits, array, nil } -func decodeDeltaVarintChunk(bits []byte, size int) ([]byte, []int64, error) { - array := make([]int64, size) +func decodeDeltaVarintChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []int64, error) { + array := allocator.int64Slice(size) newBits := bits var err error newBits, array[0], err = decodeVarint(newBits) @@ -243,8 +268,8 @@ func decodeDeltaVarintChunk(bits []byte, size int) ([]byte, []int64, error) { return newBits, array, nil } -func decodeDeltaUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { - array := make([]uint64, size) +func decodeDeltaUvarintChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []uint64, error) { + array := allocator.uint64Slice(size) newBits := bits var err error newBits, array[0], err = decodeUvarint(newBits) @@ -262,7 +287,7 @@ func decodeDeltaUvarintChunk(bits []byte, size int) ([]byte, []uint64, error) { } // size tables are always at end of serialized data, there is no unread bytes to return -func decodeSizeTables(bits []byte) (int, [][]uint64, error) { +func decodeSizeTables(bits []byte, allocator *SliceAllocator) (int, [][]uint64, error) { nb, size, _ := decodeUvarintReversedLength(bits) sizeOffset := len(bits) - nb tablesOffset := sizeOffset - size @@ -277,7 +302,7 @@ func decodeSizeTables(bits []byte) (int, [][]uint64, error) { if err != nil { return 0, nil, errors.Trace(err) } - tables, table, err = decodeDeltaUvarintChunk(tables, size) + tables, table, err = decodeDeltaUvarintChunk(tables, size, allocator) if err != nil { return 0, nil, errors.Trace(err) } @@ -288,14 +313,14 @@ func decodeSizeTables(bits []byte) (int, [][]uint64, error) { } /// TiDB types decoder -func decodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{}, error) { +func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{}, error) { if bits == nil { return nil, nil } switch ty { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: // value type for these mysql types are string - return string(bits), nil + return unsafeBytesToString(bits), nil case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: // value type for thest mysql types are uint64 _, u64, err := decodeUvarint(bits) @@ -327,59 +352,61 @@ func decodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ } // Message decoder -type craftMessageDecoder struct { - bits []byte - sizeTables [][]uint64 - valuesSizeTable []uint64 +type MessageDecoder struct { + bits []byte + sizeTables [][]uint64 + bodySizeTable []uint64 + allocator *SliceAllocator } -func newCraftMessageDecoder(bits []byte) (*craftMessageDecoder, error) { +func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, error) { bits, version, err := decodeUvarint(bits) if err != nil { return nil, errors.Trace(err) } - if version < CraftVersion1 { + if version < Version1 { return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("unexpected craft version") } - sizeTablesSize, sizeTables, err := decodeSizeTables(bits) + sizeTablesSize, sizeTables, err := decodeSizeTables(bits, allocator) if err != nil { return nil, errors.Trace(err) } - return &craftMessageDecoder{ - bits: bits[:len(bits)-sizeTablesSize], - sizeTables: sizeTables, - valuesSizeTable: sizeTables[craftValueSizeTableIndex], + return &MessageDecoder{ + bits: bits[:len(bits)-sizeTablesSize], + sizeTables: sizeTables, + bodySizeTable: sizeTables[bodySizeTableIndex], + allocator: allocator, }, nil } -func (d *craftMessageDecoder) decodeKeys() (*craftColumnarKeys, error) { - var pairs, keysSize int +func (d *MessageDecoder) Headers() (*Headers, error) { + var pairs, headersSize int var err error d.bits, pairs, err = decodeUvarintLength(d.bits) if err != nil { return nil, errors.Trace(err) } - keysSize = int(d.sizeTables[craftKeySizeTableIndex][0]) - var keys *craftColumnarKeys - keys, err = decodeCraftColumnarKeys(d.bits[:keysSize], pairs) + headersSize = int(d.sizeTables[headerSizeTableIndex][0]) + var headers *Headers + headers, err = decodeHeaders(d.bits[:headersSize], pairs, d.allocator) if err != nil { return nil, errors.Trace(err) } - // skip keys - d.bits = d.bits[keysSize:] - return keys, nil + // skip headers + d.bits = d.bits[headersSize:] + return headers, nil } -func (d *craftMessageDecoder) valueBits(index int) []byte { +func (d *MessageDecoder) bodyBits(index int) []byte { start := 0 if index > 0 { - start = int(d.valuesSizeTable[index-1]) + start = int(d.bodySizeTable[index-1]) } - return d.bits[start:int(d.valuesSizeTable[index])] + return d.bits[start:int(d.bodySizeTable[index])] } -func (d *craftMessageDecoder) decodeDDLEvent(index int) (pmodel.ActionType, string, error) { - bits, ty, err := decodeUvarint(d.valueBits(index)) +func (d *MessageDecoder) DDLEvent(index int) (pmodel.ActionType, string, error) { + bits, ty, err := decodeUvarint(d.bodyBits(index)) if err != nil { return pmodel.ActionNone, "", errors.Trace(err) } @@ -387,24 +414,22 @@ func (d *craftMessageDecoder) decodeDDLEvent(index int) (pmodel.ActionType, stri return pmodel.ActionType(ty), query, err } -func (d *craftMessageDecoder) decodeRowChangedEvent(index int) (preColumns, columns *craftColumnarColumnGroup, err error) { - bits := d.valueBits(index) - columnGroupSizeTable := d.sizeTables[craftColumnGroupSizeTableStartIndex+index] +func (d *MessageDecoder) RowChangedEvent(index int) (preColumns, columns *columnGroup, err error) { + bits := d.bodyBits(index) + columnGroupSizeTable := d.sizeTables[columnGroupSizeTableStartIndex+index] columnGroupIndex := 0 for len(bits) > 0 { columnGroupSize := columnGroupSizeTable[columnGroupIndex] - columnGroup, err := decodeCraftColumnarColumnGroup(bits[:columnGroupSize]) + columnGroup, err := decodeColumnGroup(bits[:columnGroupSize], d.allocator) bits = bits[columnGroupSize:] columnGroupIndex++ if err != nil { return nil, nil, errors.Trace(err) } switch columnGroup.ty { - case craftColumnGroupTypeDelete: - fallthrough - case craftColumnGroupTypeOld: + case columnGroupTypeOld: preColumns = columnGroup - case craftColumnGroupTypeNew: + case columnGroupTypeNew: columns = columnGroup } } diff --git a/cdc/sink/codec/craft_encoder.go b/cdc/sink/codec/craft/encoder.go similarity index 58% rename from cdc/sink/codec/craft_encoder.go rename to cdc/sink/codec/craft/encoder.go index 63e3513c818..efc53802e2f 100644 --- a/cdc/sink/codec/craft_encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -11,41 +11,47 @@ // See the License for the specific language governing permissions and // limitations under the License. -package codec +package craft import ( "encoding/binary" "math" + "unsafe" "github.com/pingcap/parser/mysql" "github.com/pingcap/ticdc/cdc/model" ) +/// create byte slice from string without copying +func unsafeStringToBytes(s string) []byte { + return *(*[]byte)(unsafe.Pointer( + &struct { + string + Cap int + }{s, len(s)}, + )) +} + /// Primitive type encoders func encodeFloat64(bits []byte, data float64) []byte { - buf := make([]byte, 8) - binary.LittleEndian.PutUint64(buf, math.Float64bits(data)) - return buf + v := math.Float64bits(data) + return append(bits, byte(v), byte(v>>8), byte(v>>16), byte(v>>24), byte(v>>32), byte(v>>40), byte(v>>48), byte(v>>56)) } func encodeVarint(bits []byte, data int64) []byte { - buf := make([]byte, binary.MaxVarintLen64) - l := binary.PutVarint(buf, data) - buf = buf[:l] - if bits == nil { - return buf + udata := uint64(data) << 1 + if data < 0 { + udata = ^udata } - return append(bits, buf...) + return encodeUvarint(bits, udata) } func encodeUvarint(bits []byte, data uint64) []byte { - buf := make([]byte, binary.MaxVarintLen64) - l := binary.PutUvarint(buf, data) - buf = buf[:l] - if bits == nil { - return buf + for data >= 0x80 { + bits = append(bits, byte(data)|0x80) + data >>= 7 } - return append(bits, buf...) + return append(bits, byte(data)) } func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { @@ -65,15 +71,14 @@ func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { func encodeBytes(bits []byte, data []byte) []byte { l := len(data) - if bits == nil { - bits = make([]byte, 0, binary.MaxVarintLen64+len(data)) - } bits = encodeUvarint(bits, uint64(l)) return append(bits, data...) } func encodeString(bits []byte, data string) []byte { - return encodeBytes(bits, []byte(data)) + l := len(data) + bits = encodeUvarint(bits, uint64(l)) + return append(bits, data...) } /// Chunk encoders @@ -82,7 +87,7 @@ func encodeStringChunk(bits []byte, data []string) []byte { bits = encodeUvarint(bits, uint64(len(s))) } for _, s := range data { - bits = append(bits, []byte(s)...) + bits = append(bits, s...) } return bits } @@ -97,7 +102,7 @@ func encodeNullableStringChunk(bits []byte, data []*string) []byte { } for _, s := range data { if s != nil { - bits = append(bits, []byte(*s)...) + bits = append(bits, *s...) } } return bits @@ -174,30 +179,30 @@ func encodeSizeTables(bits []byte, tables [][]uint64) []byte { } /// TiDB types encoder -func encodeTiDBType(ty byte, flag model.ColumnFlagType, value interface{}) []byte { +func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagType, value interface{}) []byte { if value == nil { return nil } switch ty { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp, mysql.TypeDuration, mysql.TypeJSON, mysql.TypeNewDecimal: // value type for these mysql types are string - return []byte(value.(string)) + return unsafeStringToBytes(value.(string)) case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: // value type for thest mysql types are uint64 - return encodeUvarint(nil, value.(uint64)) + return encodeUvarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(uint64)) case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: // value type for these mysql types are []byte return value.([]byte) case mysql.TypeFloat, mysql.TypeDouble: // value type for these mysql types are float64 - return encodeFloat64(nil, value.(float64)) + return encodeFloat64(allocator.byteSlice(8)[:0], value.(float64)) case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: // value type for these mysql types are int64 or uint64 depends on flags if flag.IsUnsigned() { - return encodeUvarint(nil, value.(uint64)) + return encodeUvarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(uint64)) } - return encodeVarint(nil, value.(int64)) + return encodeVarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(int64)) case mysql.TypeUnspecified: fallthrough case mysql.TypeNull: @@ -209,95 +214,98 @@ func encodeTiDBType(ty byte, flag model.ColumnFlagType, value interface{}) []byt } /// Message encoder -type craftMessageEncoder struct { - bits []byte - sizeTables [][]uint64 - valuesStartOffset int - valuesSizes []uint64 - valuesSizesIndex int +type MessageEncoder struct { + bits []byte + sizeTables [][]uint64 + bodyStartOffset int + bodySize []uint64 + bodySizeIndex int + + allocator *SliceAllocator } -func newCraftMessageEncoder() *craftMessageEncoder { - return &craftMessageEncoder{ - bits: encodeUvarint(make([]byte, 0, craftDefaultBufferCapacity), CraftVersion1), +func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { + return &MessageEncoder{ + bits: encodeUvarint(make([]byte, 0, DefaultBufferCapacity), Version1), + allocator: allocator, } } -func (e *craftMessageEncoder) encodeValueSize() *craftMessageEncoder { - e.valuesSizes[e.valuesSizesIndex] = uint64(len(e.bits) - e.valuesStartOffset) - e.valuesSizesIndex++ +func (e *MessageEncoder) encodeBodySize() *MessageEncoder { + e.bodySize[e.bodySizeIndex] = uint64(len(e.bits) - e.bodyStartOffset) + e.bodySizeIndex++ return e } -func (e *craftMessageEncoder) encodeUvarint(u64 uint64) *craftMessageEncoder { +func (e *MessageEncoder) encodeUvarint(u64 uint64) *MessageEncoder { e.bits = encodeUvarint(e.bits, u64) return e } -func (e *craftMessageEncoder) encodeString(s string) *craftMessageEncoder { +func (e *MessageEncoder) encodeString(s string) *MessageEncoder { e.bits = encodeString(e.bits, s) return e } -func (e *craftMessageEncoder) encodeKeys(keys *craftColumnarKeys) *craftMessageEncoder { - e.bits = encodeUvarint(e.bits, uint64(keys.count)) +func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { + e.bits = encodeUvarint(e.bits, uint64(headers.count)) oldSize := len(e.bits) - e.valuesSizes = make([]uint64, keys.count) - e.bits = keys.encode(e.bits) - e.valuesStartOffset = len(e.bits) - e.sizeTables = append(e.sizeTables, []uint64{uint64(len(e.bits) - oldSize)}, e.valuesSizes) + e.bodySize = e.allocator.uint64Slice(headers.count) + e.bits = headers.encode(e.bits) + e.bodyStartOffset = len(e.bits) + e.sizeTables = append(e.sizeTables, e.allocator.oneUint64Slice(uint64(len(e.bits)-oldSize)), e.bodySize) return e } -func (e *craftMessageEncoder) encode() []byte { +func (e *MessageEncoder) Encode() []byte { return encodeSizeTables(e.bits, e.sizeTables) } -func (e *craftMessageEncoder) encodeRowChangeEvents(events []craftRowChangedEvent) *craftMessageEncoder { +func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *MessageEncoder { sizeTables := e.sizeTables for _, event := range events { - columnGroupSizeTable := make([]uint64, len(event)) + columnGroupSizeTable := e.allocator.uint64Slice(len(event)) for gi, group := range event { oldSize := len(e.bits) e.bits = group.encode(e.bits) columnGroupSizeTable[gi] = uint64(len(e.bits) - oldSize) } sizeTables = append(sizeTables, columnGroupSizeTable) - e.encodeValueSize() + e.encodeBodySize() } e.sizeTables = sizeTables return e } -func newCraftResolvedEventEncoder(ts uint64) *craftMessageEncoder { - return newCraftMessageEncoder().encodeKeys(&craftColumnarKeys{ - ts: []uint64{uint64(ts)}, - ty: []uint64{uint64(model.MqMessageTypeResolved)}, - rowID: []int64{int64(-1)}, - partition: []int64{int64(-1)}, - schema: [][]byte{nil}, - table: [][]byte{nil}, +func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncoder { + return NewMessageEncoder(allocator).encodeHeaders(&Headers{ + ts: allocator.oneUint64Slice(ts), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), + rowID: allocator.oneInt64Slice(-1), + partition: allocator.oneInt64Slice(-1), + schema: allocator.oneNullableStringSlice(nil), + table: allocator.oneNullableStringSlice(nil), count: 1, - }).encodeValueSize() + }).encodeBodySize() } -func newCraftDDLEventEncoder(ev *model.DDLEvent) *craftMessageEncoder { +func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageEncoder { ty := uint64(ev.Type) query := ev.Query - var schema, table []byte + var schema, table *string if len(ev.TableInfo.Schema) > 0 { - schema = []byte(ev.TableInfo.Schema) + schema = &ev.TableInfo.Schema } if len(ev.TableInfo.Table) > 0 { - table = []byte(ev.TableInfo.Table) + table = &ev.TableInfo.Table } - return newCraftMessageEncoder().encodeKeys(&craftColumnarKeys{ - ts: []uint64{uint64(ev.CommitTs)}, - ty: []uint64{uint64(model.MqMessageTypeDDL)}, - rowID: []int64{int64(-1)}, - partition: []int64{int64(-1)}, - schema: [][]byte{schema}, - table: [][]byte{table}, + return NewMessageEncoder(allocator).encodeHeaders(&Headers{ + ts: allocator.oneUint64Slice(ev.CommitTs), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), + rowID: allocator.oneInt64Slice(-1), + partition: allocator.oneInt64Slice(-1), + schema: allocator.oneNullableStringSlice(schema), + table: allocator.oneNullableStringSlice(table), count: 1, - }).encodeUvarint(ty).encodeString(query).encodeValueSize() + }).encodeUvarint(ty).encodeString(query).encodeBodySize() } diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go new file mode 100644 index 00000000000..3645f75db6c --- /dev/null +++ b/cdc/sink/codec/craft/model.go @@ -0,0 +1,351 @@ +// Copyright 2021 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.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package craft + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/ticdc/cdc/model" +) + +const ( + // Version1 represents the version of craft format + Version1 uint64 = 1 + + // default buffer size + DefaultBufferCapacity = 1024 + + // Column group types + columnGroupTypeOld = 0x2 + columnGroupTypeNew = 0x1 + + // Size tables index + headerSizeTableIndex = 0 + bodySizeTableIndex = 1 + columnGroupSizeTableStartIndex = 2 +) + +/// Headers in columnar layout +type Headers struct { + ts []uint64 + ty []uint64 + rowID []int64 + partition []int64 + schema []*string + table []*string + + count int +} + +func (h *Headers) Count() int { + return h.count +} + +func (h *Headers) encode(bits []byte) []byte { + bits = encodeDeltaUvarintChunk(bits, h.ts[:h.count]) + bits = encodeDeltaUvarintChunk(bits, h.ty[:h.count]) + bits = encodeDeltaVarintChunk(bits, h.rowID[:h.count]) + bits = encodeDeltaVarintChunk(bits, h.partition[:h.count]) + bits = encodeNullableStringChunk(bits, h.schema[:h.count]) + bits = encodeNullableStringChunk(bits, h.table[:h.count]) + return bits +} + +func (h *Headers) appendHeader(allocator *SliceAllocator, ts, ty uint64, rowID, partition int64, schema, table *string) int { + idx := h.count + if idx+1 > len(h.ty) { + size := newBufferSize(idx) + h.ts = allocator.resizeUint64Slice(h.ts, size) + h.ty = allocator.resizeUint64Slice(h.ty, size) + h.rowID = allocator.resizeInt64Slice(h.rowID, size) + h.partition = allocator.resizeInt64Slice(h.partition, size) + h.schema = allocator.resizeNullableStringSlice(h.schema, size) + h.table = allocator.resizeNullableStringSlice(h.table, size) + } + h.ts[idx] = ts + h.ty[idx] = ty + h.rowID[idx] = rowID + h.partition[idx] = partition + h.schema[idx] = schema + h.table[idx] = table + h.count++ + + return 32 + len(*schema) + len(*table) /* 4 64-bits integers and two bytes array */ +} + +func (h *Headers) reset() { + h.count = 0 +} + +func (h *Headers) GetType(index int) model.MqMessageType { + return model.MqMessageType(h.ty[index]) +} + +func (h *Headers) GetTs(index int) uint64 { + return h.ts[index] +} + +func (h *Headers) GetPartition(index int) int64 { + return h.partition[index] +} + +func (h *Headers) GetSchema(index int) string { + if h.schema[index] != nil { + return *h.schema[index] + } + return "" +} + +func (h *Headers) GetTable(index int) string { + if h.table[index] != nil { + return *h.table[index] + } + return "" +} + +func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Headers, error) { + var ts, ty []uint64 + var rowID, partition []int64 + var schema, table []*string + var err error + if bits, ts, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if bits, ty, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if bits, rowID, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if bits, partition, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if bits, schema, err = decodeNullableStringChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if _, table, err = decodeNullableStringChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + return &Headers{ + ts: ts, + ty: ty, + rowID: rowID, + partition: partition, + schema: schema, + table: table, + count: numHeaders, + }, nil +} + +/// Column group in columnar layout +type columnGroup struct { + ty byte + names []string + types []uint64 + flags []uint64 + values [][]byte +} + +func (g *columnGroup) encode(bits []byte) []byte { + bits = append(bits, g.ty) + bits = encodeUvarint(bits, uint64(len(g.names))) + bits = encodeStringChunk(bits, g.names) + bits = encodeUvarintChunk(bits, g.types) + bits = encodeUvarintChunk(bits, g.flags) + bits = encodeNullableBytesChunk(bits, g.values) + return bits +} + +func (g *columnGroup) ToModel() ([]*model.Column, error) { + columns := make([]*model.Column, len(g.names)) + for i, name := range g.names { + ty := byte(g.types[i]) + flag := model.ColumnFlagType(g.flags[i]) + value, err := DecodeTiDBType(ty, flag, g.values[i]) + if err != nil { + return nil, errors.Trace(err) + } + columns[i] = &model.Column{ + Name: name, + Type: ty, + Flag: flag, + Value: value, + } + } + return columns, nil +} + +func decodeColumnGroup(bits []byte, allocator *SliceAllocator) (*columnGroup, error) { + var numColumns int + bits, ty, err := decodeUint8(bits) + if err != nil { + return nil, errors.Trace(err) + } + bits, numColumns, err = decodeUvarintLength(bits) + if err != nil { + return nil, errors.Trace(err) + } + var names []string + var values [][]byte + var types, flags []uint64 + bits, names, err = decodeStringChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + bits, types, err = decodeUvarintChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + bits, flags, err = decodeUvarintChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + _, values, err = decodeNullableBytesChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + return &columnGroup{ + ty: ty, + names: names, + types: types, + flags: flags, + values: values, + }, nil +} + +func newColumnGroup(allocator *SliceAllocator, ty byte, columns []*model.Column) (int, *columnGroup) { + l := len(columns) + values := allocator.bytesSlice(l) + names := allocator.stringSlice(l) + types := allocator.uint64Slice(l) + flags := allocator.uint64Slice(l) + estimatedSize := 0 + idx := 0 + for _, col := range columns { + if col == nil { + continue + } + names[idx] = col.Name + types[idx] = uint64(col.Type) + flags[idx] = uint64(col.Flag) + value := EncodeTiDBType(allocator, col.Type, col.Flag, col.Value) + values[idx] = value + estimatedSize += len(col.Name) + len(value) + 16 /* two 64-bits integers */ + idx++ + } + if idx > 0 { + return estimatedSize, &columnGroup{ + ty: ty, + names: names[:idx], + types: types[:idx], + flags: flags[:idx], + values: values[:idx], + } + } + return estimatedSize, nil +} + +/// Row changed message is basically an array of column groups +type rowChangedEvent = []*columnGroup + +func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) (int, rowChangedEvent) { + numGroups := 0 + if ev.PreColumns != nil { + numGroups++ + } + if ev.Columns != nil { + numGroups++ + } + groups := allocator.columnGroupSlice(numGroups) + estimatedSize := 0 + if size, group := newColumnGroup(allocator, columnGroupTypeNew, ev.Columns); group != nil { + groups[0] = group + estimatedSize += size + } + if size, group := newColumnGroup(allocator, columnGroupTypeOld, ev.PreColumns); group != nil { + groups[1] = group + estimatedSize += size + } + return estimatedSize, rowChangedEvent(groups) +} + +/// A buffer to save row changed events in batch +type RowChangedEventBuffer struct { + headers *Headers + + events []rowChangedEvent + eventsCount int + estimatedSize int + + allocator *SliceAllocator +} + +func NewRowChangedEventBuffer(allocator *SliceAllocator) *RowChangedEventBuffer { + return &RowChangedEventBuffer{ + headers: &Headers{}, + allocator: allocator, + } +} + +func (b *RowChangedEventBuffer) Encode() []byte { + bits := NewMessageEncoder(b.allocator).encodeHeaders(b.headers).encodeRowChangeEvents(b.events[:b.eventsCount]).Encode() + b.Reset() + return bits +} + +func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) (rows, size int) { + var partition int64 = -1 + if ev.Table.IsPartition { + partition = ev.Table.TableID + } + + var schema, table *string + if len(ev.Table.Schema) > 0 { + schema = &ev.Table.Schema + } + if len(ev.Table.Table) > 0 { + table = &ev.Table.Table + } + + b.estimatedSize += b.headers.appendHeader( + b.allocator, + ev.CommitTs, + uint64(model.MqMessageTypeRow), + ev.RowID, + partition, + schema, + table, + ) + if b.eventsCount+1 > len(b.events) { + b.events = b.allocator.resizeRowChangedEventSlice(b.events, newBufferSize(b.eventsCount)) + } + size, message := newRowChangedMessage(b.allocator, ev) + b.events[b.eventsCount] = message + b.eventsCount++ + b.estimatedSize += size + return b.eventsCount, b.estimatedSize +} + +func (b *RowChangedEventBuffer) Reset() { + b.headers.reset() + b.eventsCount = 0 + b.estimatedSize = 0 +} + +func (b *RowChangedEventBuffer) Size() int { + return b.estimatedSize +} + +func (b *RowChangedEventBuffer) GetHeaders() *Headers { + return b.headers +} diff --git a/cdc/sink/codec/craft_model.go b/cdc/sink/codec/craft_model.go deleted file mode 100644 index 6e2727ee942..00000000000 --- a/cdc/sink/codec/craft_model.go +++ /dev/null @@ -1,361 +0,0 @@ -// Copyright 2021 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.orglicensesLICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package codec - -import ( - "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" -) - -/// Utility functions for buffer allocation -func newBufferSize(oldSize int) int { - var newSize int - if oldSize > 128 { - newSize = oldSize + 128 - } else { - if oldSize > 0 { - newSize = oldSize * 2 - } else { - newSize = 8 - } - } - return newSize -} - -func newUint64Buffers(eachSize int) ([]uint64, []uint64) { - buffer := make([]uint64, eachSize*2) - return buffer[:eachSize], buffer[eachSize:] -} - -func resizeUint64Buffer(buffer1, buffer2 []uint64) ([]uint64, []uint64) { - newBuffer1, newBuffer2 := newUint64Buffers(newBufferSize(len(buffer1))) - copy(newBuffer1, buffer1) - copy(newBuffer2, buffer2) - return newBuffer1, newBuffer2 -} - -func newInt64Buffers(eachSize int) ([]int64, []int64) { - buffer := make([]int64, eachSize*2) - return buffer[:eachSize], buffer[eachSize:] -} - -func resizeInt64Buffer(buffer1, buffer2 []int64) ([]int64, []int64) { - newBuffer1, newBuffer2 := newInt64Buffers(newBufferSize(len(buffer1))) - copy(newBuffer1, buffer1) - copy(newBuffer2, buffer2) - return newBuffer1, newBuffer2 -} - -func newBytesBuffers(eachSize int) ([][]byte, [][]byte) { - buffer := make([][]byte, eachSize*2) - return buffer[:eachSize], buffer[eachSize:] -} - -func resizeBytesBuffer(buffer1, buffer2 [][]byte) ([][]byte, [][]byte) { - newBuffer1, newBuffer2 := newBytesBuffers(newBufferSize(len(buffer1))) - copy(newBuffer1, buffer1) - copy(newBuffer2, buffer2) - return newBuffer1, newBuffer2 -} - -/// Keys in columnar layout -type craftColumnarKeys struct { - ts []uint64 - ty []uint64 - rowID []int64 - partition []int64 - schema [][]byte - table [][]byte - - count int -} - -func (k *craftColumnarKeys) encode(bits []byte) []byte { - bits = encodeDeltaUvarintChunk(bits, k.ts[:k.count]) - bits = encodeDeltaUvarintChunk(bits, k.ty[:k.count]) - bits = encodeDeltaVarintChunk(bits, k.rowID[:k.count]) - bits = encodeDeltaVarintChunk(bits, k.partition[:k.count]) - bits = encodeNullableBytesChunk(bits, k.schema[:k.count]) - bits = encodeNullableBytesChunk(bits, k.table[:k.count]) - return bits -} - -func (k *craftColumnarKeys) appendKey(ts, ty uint64, rowID, partition int64, schema, table []byte) int { - idx := k.count - if idx+1 > len(k.ty) { - k.ts, k.ty = resizeUint64Buffer(k.ts, k.ty) - k.rowID, k.partition = resizeInt64Buffer(k.rowID, k.partition) - k.schema, k.table = resizeBytesBuffer(k.schema, k.table) - } - k.ts[idx] = ts - k.ty[idx] = ty - k.rowID[idx] = rowID - k.partition[idx] = partition - k.schema[idx] = schema - k.table[idx] = table - k.count++ - - return 32 + len(schema) + len(table) /* 4 64-bits integers and two bytes array */ -} - -func (k *craftColumnarKeys) reset() { - k.count = 0 -} - -func (k *craftColumnarKeys) getType(index int) model.MqMessageType { - return model.MqMessageType(k.ty[index]) -} - -func (k *craftColumnarKeys) getTs(index int) uint64 { - return k.ts[index] -} - -func (k *craftColumnarKeys) getPartition(index int) int64 { - return k.partition[index] -} - -func (k *craftColumnarKeys) getSchema(index int) string { - return string(k.schema[index]) -} - -func (k *craftColumnarKeys) getTable(index int) string { - return string(k.table[index]) -} - -func decodeCraftColumnarKeys(bits []byte, numKeys int) (*craftColumnarKeys, error) { - var ts, ty []uint64 - var rowID, partition []int64 - var schema, table [][]byte - var err error - if bits, ts, err = decodeDeltaUvarintChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - if bits, ty, err = decodeDeltaUvarintChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - if bits, rowID, err = decodeDeltaVarintChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - if bits, partition, err = decodeDeltaVarintChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - if bits, schema, err = decodeNullableBytesChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - if _, table, err = decodeNullableBytesChunk(bits, numKeys); err != nil { - return nil, errors.Trace(err) - } - return &craftColumnarKeys{ - ts: ts, - ty: ty, - rowID: rowID, - partition: partition, - schema: schema, - table: table, - count: numKeys, - }, nil -} - -/// Column group in columnar layout -type craftColumnarColumnGroup struct { - ty byte - names [][]byte - types []uint64 - flags []uint64 - values [][]byte -} - -func (g *craftColumnarColumnGroup) encode(bits []byte) []byte { - bits = append(bits, g.ty) - bits = encodeUvarint(bits, uint64(len(g.names))) - bits = encodeBytesChunk(bits, g.names) - bits = encodeUvarintChunk(bits, g.types) - bits = encodeUvarintChunk(bits, g.flags) - bits = encodeNullableBytesChunk(bits, g.values) - return bits -} - -func (g *craftColumnarColumnGroup) toModel() ([]*model.Column, error) { - columns := make([]*model.Column, len(g.names)) - for i, name := range g.names { - ty := byte(g.types[i]) - flag := model.ColumnFlagType(g.flags[i]) - value, err := decodeTiDBType(ty, flag, g.values[i]) - if err != nil { - return nil, errors.Trace(err) - } - columns[i] = &model.Column{ - Name: string(name), - Type: ty, - Flag: flag, - Value: value, - } - } - return columns, nil -} - -func decodeCraftColumnarColumnGroup(bits []byte) (*craftColumnarColumnGroup, error) { - var numColumns int - bits, ty, err := decodeUint8(bits) - if err != nil { - return nil, errors.Trace(err) - } - bits, numColumns, err = decodeUvarintLength(bits) - if err != nil { - return nil, errors.Trace(err) - } - var names, values [][]byte - var types, flags []uint64 - bits, names, err = decodeBytesChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - bits, types, err = decodeUvarintChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - bits, flags, err = decodeUvarintChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - _, values, err = decodeNullableBytesChunk(bits, numColumns) - if err != nil { - return nil, errors.Trace(err) - } - return &craftColumnarColumnGroup{ - ty: ty, - names: names, - types: types, - flags: flags, - values: values, - }, nil -} - -func newCraftColumnarColumnGroup(ty byte, columns []*model.Column) (int, *craftColumnarColumnGroup) { - var names [][]byte - var values [][]byte - var types []uint64 - var flags []uint64 - estimatedSize := 0 - for _, col := range columns { - if col == nil { - continue - } - name := []byte(col.Name) - names = append(names, name) - types = append(types, uint64(col.Type)) - flags = append(flags, uint64(col.Flag)) - value := encodeTiDBType(col.Type, col.Flag, col.Value) - values = append(values, value) - estimatedSize += len(name) + len(value) + 8 /* two bytes array and two 64-bits integers */ - } - if len(names) > 0 { - return estimatedSize, &craftColumnarColumnGroup{ - ty: ty, - names: names, - types: types, - flags: flags, - values: values, - } - } - return estimatedSize, nil -} - -/// Row changed message is basically an array of column groups -type craftRowChangedEvent = []*craftColumnarColumnGroup - -func newCraftRowChangedMessage(ev *model.RowChangedEvent) (int, craftRowChangedEvent) { - var groups []*craftColumnarColumnGroup - estimatedSize := 0 - if ev.IsDelete() { - if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeDelete, ev.PreColumns); group != nil { - groups = append(groups, group) - estimatedSize += size - } - } else { - if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeNew, ev.Columns); group != nil { - groups = append(groups, group) - estimatedSize += size - } - if size, group := newCraftColumnarColumnGroup(craftColumnGroupTypeOld, ev.PreColumns); group != nil { - groups = append(groups, group) - estimatedSize += size - } - } - return estimatedSize, craftRowChangedEvent(groups) -} - -/// A buffer to save row changed events in batch -type craftRowChangedEventBuffer struct { - keys *craftColumnarKeys - - events []craftRowChangedEvent - eventsCount int - estimatedSize int -} - -func (b *craftRowChangedEventBuffer) encode() []byte { - bits := newCraftMessageEncoder().encodeKeys(b.keys).encodeRowChangeEvents(b.events[:b.eventsCount]).encode() - b.reset() - return bits -} - -func (b *craftRowChangedEventBuffer) appendRowChangedEvent(ev *model.RowChangedEvent) (rows, size int) { - var partition int64 = -1 - if ev.Table.IsPartition { - partition = ev.Table.TableID - } - - var schema, table []byte - if len(ev.Table.Schema) > 0 { - schema = []byte(ev.Table.Schema) - } - if len(ev.Table.Table) > 0 { - table = []byte(ev.Table.Table) - } - - b.estimatedSize += b.keys.appendKey( - ev.CommitTs, - uint64(model.MqMessageTypeRow), - ev.RowID, - partition, - schema, - table, - ) - if b.eventsCount+1 > len(b.events) { - newSize := newBufferSize(b.eventsCount) - events := make([]craftRowChangedEvent, newSize) - copy(events, b.events) - b.events = events - } - size, message := newCraftRowChangedMessage(ev) - b.events[b.eventsCount] = message - b.eventsCount++ - b.estimatedSize += size - return b.eventsCount, b.estimatedSize -} - -func (b *craftRowChangedEventBuffer) reset() { - b.keys.reset() - b.eventsCount = 0 - b.estimatedSize = 0 -} - -func (b *craftRowChangedEventBuffer) size() int { - return b.estimatedSize -} - -func (b *craftRowChangedEventBuffer) getKeys() *craftColumnarKeys { - return b.keys -} diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index 8459e144a7f..8c6ec0fc936 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -15,7 +15,6 @@ package codec import ( "math" - "math/rand" "strconv" "github.com/pingcap/check" @@ -249,78 +248,3 @@ func (s *craftBatchSuite) TestDefaultEventBatchCodec(c *check.C) { return encoder }, NewCraftEventBatchDecoder) } - -var _ = check.Suite(&craftCodecSuite{}) - -type craftCodecSuite struct{} - -func (s *craftCodecSuite) TestSizeTable(c *check.C) { - defer testleak.AfterTest(c)() - tables := [][]uint64{ - { - 1, 3, 5, 7, 9, - }, - { - 2, 4, 6, 8, 10, - }, - } - bits := make([]byte, 16) - rand.Read(bits) - bits = encodeSizeTables(bits, tables) - - size, decoded, err := decodeSizeTables(bits) - c.Check(err, check.IsNil) - c.Check(decoded, check.DeepEquals, tables) - c.Check(size, check.Equals, len(bits)-16) -} - -func (s *craftCodecSuite) TestUvarintReverse(c *check.C) { - defer testleak.AfterTest(c)() - - var i uint64 = 0 - - for i < 0x8000000000000000 { - bits := make([]byte, 16) - rand.Read(bits) - bits, bytes1 := encodeUvarintReversed(bits, i) - bytes2, u64, err := decodeUvarintReversed(bits) - c.Check(err, check.IsNil) - c.Check(u64, check.Equals, i) - c.Check(bytes1, check.Equals, len(bits)-16) - c.Check(bytes1, check.Equals, bytes2) - if i == 0 { - i = 1 - } else { - i <<= 1 - } - } -} - -func newNullableString(a string) *string { - return &a -} - -func (s *craftCodecSuite) TestEncodeChunk(c *check.C) { - defer testleak.AfterTest(c)() - stringChunk := []string{"a", "b", "c"} - nullableStringChunk := []*string{newNullableString("a"), newNullableString("b"), newNullableString("c")} - int64Chunk := []int64{1, 2, 3} - - bits := encodeStringChunk(nil, stringChunk) - bits, decodedStringChunk, err := decodeStringChunk(bits, 3) - c.Check(err, check.IsNil) - c.Check(len(bits), check.Equals, 0) - c.Check(decodedStringChunk, check.DeepEquals, stringChunk) - - bits = encodeNullableStringChunk(nil, nullableStringChunk) - bits, decodedNullableStringChunk, err := decodeNullableStringChunk(bits, 3) - c.Check(err, check.IsNil) - c.Check(len(bits), check.Equals, 0) - c.Check(decodedNullableStringChunk, check.DeepEquals, nullableStringChunk) - - bits = encodeVarintChunk(nil, int64Chunk) - bits, decodedVarintChunk, err := decodeVarintChunk(bits, 3) - c.Check(err, check.IsNil) - c.Check(len(bits), check.Equals, 0) - c.Check(decodedVarintChunk, check.DeepEquals, int64Chunk) -} diff --git a/proto/CraftBenchmark.proto b/proto/CraftBenchmark.proto new file mode 100644 index 00000000000..ffb65efc1a9 --- /dev/null +++ b/proto/CraftBenchmark.proto @@ -0,0 +1,62 @@ +// Copyright 2021 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; +package benchmark; + +option java_package = "io.tidb.bigdata.cdc.craft"; +option java_outer_classname = "CraftBenchmark"; +option optimize_for = SPEED; + +message Key { + uint64 ts = 1; + string schema = 2; + string table = 3; + int64 row_id = 4; + uint32 type = 5; + int64 partition = 6; +} + +message Column { + string name = 1; + uint32 type = 2; + uint32 flag = 3; + bytes value = 4; +} + +message RowChanged { + repeated Column old_value = 1; + repeated Column new_value = 2; +} + +message KeysColumnar { + repeated uint64 ts = 1; + repeated string schema = 2; + repeated string table = 3; + repeated int64 row_id = 4; + repeated uint32 type = 5; + repeated int64 partition = 6; +} + +message ColumnsColumnar { + repeated string name = 1; + repeated uint32 type = 2; + repeated bool where_handle = 3; + repeated uint32 flag = 4; + repeated bytes value = 5; +} + +message RowChangedColumnar { + repeated ColumnsColumnar old_value = 1; + repeated ColumnsColumnar new_value = 2; +} diff --git a/proto/benchmark/CraftBenchmark.pb.go b/proto/benchmark/CraftBenchmark.pb.go new file mode 100644 index 00000000000..c993386711e --- /dev/null +++ b/proto/benchmark/CraftBenchmark.pb.go @@ -0,0 +1,2593 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: CraftBenchmark.proto + +package benchmark + +import ( + fmt "fmt" + proto "github.com/golang/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package + +type Key struct { + Ts uint64 `protobuf:"varint,1,opt,name=ts,proto3" json:"ts,omitempty"` + Schema string `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"` + Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` + RowId int64 `protobuf:"varint,4,opt,name=row_id,json=rowId,proto3" json:"row_id,omitempty"` + Type uint32 `protobuf:"varint,5,opt,name=type,proto3" json:"type,omitempty"` + Partition int64 `protobuf:"varint,6,opt,name=partition,proto3" json:"partition,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Key) Reset() { *m = Key{} } +func (m *Key) String() string { return proto.CompactTextString(m) } +func (*Key) ProtoMessage() {} +func (*Key) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{0} +} +func (m *Key) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Key) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Key.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Key) XXX_Merge(src proto.Message) { + xxx_messageInfo_Key.Merge(m, src) +} +func (m *Key) XXX_Size() int { + return m.Size() +} +func (m *Key) XXX_DiscardUnknown() { + xxx_messageInfo_Key.DiscardUnknown(m) +} + +var xxx_messageInfo_Key proto.InternalMessageInfo + +func (m *Key) GetTs() uint64 { + if m != nil { + return m.Ts + } + return 0 +} + +func (m *Key) GetSchema() string { + if m != nil { + return m.Schema + } + return "" +} + +func (m *Key) GetTable() string { + if m != nil { + return m.Table + } + return "" +} + +func (m *Key) GetRowId() int64 { + if m != nil { + return m.RowId + } + return 0 +} + +func (m *Key) GetType() uint32 { + if m != nil { + return m.Type + } + return 0 +} + +func (m *Key) GetPartition() int64 { + if m != nil { + return m.Partition + } + return 0 +} + +type Column struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Type uint32 `protobuf:"varint,2,opt,name=type,proto3" json:"type,omitempty"` + Flag uint32 `protobuf:"varint,3,opt,name=flag,proto3" json:"flag,omitempty"` + Value []byte `protobuf:"bytes,4,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Column) Reset() { *m = Column{} } +func (m *Column) String() string { return proto.CompactTextString(m) } +func (*Column) ProtoMessage() {} +func (*Column) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{1} +} +func (m *Column) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Column) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Column.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Column) XXX_Merge(src proto.Message) { + xxx_messageInfo_Column.Merge(m, src) +} +func (m *Column) XXX_Size() int { + return m.Size() +} +func (m *Column) XXX_DiscardUnknown() { + xxx_messageInfo_Column.DiscardUnknown(m) +} + +var xxx_messageInfo_Column proto.InternalMessageInfo + +func (m *Column) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *Column) GetType() uint32 { + if m != nil { + return m.Type + } + return 0 +} + +func (m *Column) GetFlag() uint32 { + if m != nil { + return m.Flag + } + return 0 +} + +func (m *Column) GetValue() []byte { + if m != nil { + return m.Value + } + return nil +} + +type RowChanged struct { + OldValue []*Column `protobuf:"bytes,1,rep,name=old_value,json=oldValue,proto3" json:"old_value,omitempty"` + NewValue []*Column `protobuf:"bytes,2,rep,name=new_value,json=newValue,proto3" json:"new_value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RowChanged) Reset() { *m = RowChanged{} } +func (m *RowChanged) String() string { return proto.CompactTextString(m) } +func (*RowChanged) ProtoMessage() {} +func (*RowChanged) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{2} +} +func (m *RowChanged) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RowChanged) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_RowChanged.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *RowChanged) XXX_Merge(src proto.Message) { + xxx_messageInfo_RowChanged.Merge(m, src) +} +func (m *RowChanged) XXX_Size() int { + return m.Size() +} +func (m *RowChanged) XXX_DiscardUnknown() { + xxx_messageInfo_RowChanged.DiscardUnknown(m) +} + +var xxx_messageInfo_RowChanged proto.InternalMessageInfo + +func (m *RowChanged) GetOldValue() []*Column { + if m != nil { + return m.OldValue + } + return nil +} + +func (m *RowChanged) GetNewValue() []*Column { + if m != nil { + return m.NewValue + } + return nil +} + +type KeysColumnar struct { + Ts []uint64 `protobuf:"varint,1,rep,packed,name=ts,proto3" json:"ts,omitempty"` + Schema []string `protobuf:"bytes,2,rep,name=schema,proto3" json:"schema,omitempty"` + Table []string `protobuf:"bytes,3,rep,name=table,proto3" json:"table,omitempty"` + RowId []int64 `protobuf:"varint,4,rep,packed,name=row_id,json=rowId,proto3" json:"row_id,omitempty"` + Type []uint32 `protobuf:"varint,5,rep,packed,name=type,proto3" json:"type,omitempty"` + Partition []int64 `protobuf:"varint,6,rep,packed,name=partition,proto3" json:"partition,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *KeysColumnar) Reset() { *m = KeysColumnar{} } +func (m *KeysColumnar) String() string { return proto.CompactTextString(m) } +func (*KeysColumnar) ProtoMessage() {} +func (*KeysColumnar) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{3} +} +func (m *KeysColumnar) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *KeysColumnar) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_KeysColumnar.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *KeysColumnar) XXX_Merge(src proto.Message) { + xxx_messageInfo_KeysColumnar.Merge(m, src) +} +func (m *KeysColumnar) XXX_Size() int { + return m.Size() +} +func (m *KeysColumnar) XXX_DiscardUnknown() { + xxx_messageInfo_KeysColumnar.DiscardUnknown(m) +} + +var xxx_messageInfo_KeysColumnar proto.InternalMessageInfo + +func (m *KeysColumnar) GetTs() []uint64 { + if m != nil { + return m.Ts + } + return nil +} + +func (m *KeysColumnar) GetSchema() []string { + if m != nil { + return m.Schema + } + return nil +} + +func (m *KeysColumnar) GetTable() []string { + if m != nil { + return m.Table + } + return nil +} + +func (m *KeysColumnar) GetRowId() []int64 { + if m != nil { + return m.RowId + } + return nil +} + +func (m *KeysColumnar) GetType() []uint32 { + if m != nil { + return m.Type + } + return nil +} + +func (m *KeysColumnar) GetPartition() []int64 { + if m != nil { + return m.Partition + } + return nil +} + +type ColumnsColumnar struct { + Name []string `protobuf:"bytes,1,rep,name=name,proto3" json:"name,omitempty"` + Type []uint32 `protobuf:"varint,2,rep,packed,name=type,proto3" json:"type,omitempty"` + WhereHandle []bool `protobuf:"varint,3,rep,packed,name=where_handle,json=whereHandle,proto3" json:"where_handle,omitempty"` + Flag []uint32 `protobuf:"varint,4,rep,packed,name=flag,proto3" json:"flag,omitempty"` + Value [][]byte `protobuf:"bytes,5,rep,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ColumnsColumnar) Reset() { *m = ColumnsColumnar{} } +func (m *ColumnsColumnar) String() string { return proto.CompactTextString(m) } +func (*ColumnsColumnar) ProtoMessage() {} +func (*ColumnsColumnar) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{4} +} +func (m *ColumnsColumnar) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnsColumnar) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ColumnsColumnar.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ColumnsColumnar) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnsColumnar.Merge(m, src) +} +func (m *ColumnsColumnar) XXX_Size() int { + return m.Size() +} +func (m *ColumnsColumnar) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnsColumnar.DiscardUnknown(m) +} + +var xxx_messageInfo_ColumnsColumnar proto.InternalMessageInfo + +func (m *ColumnsColumnar) GetName() []string { + if m != nil { + return m.Name + } + return nil +} + +func (m *ColumnsColumnar) GetType() []uint32 { + if m != nil { + return m.Type + } + return nil +} + +func (m *ColumnsColumnar) GetWhereHandle() []bool { + if m != nil { + return m.WhereHandle + } + return nil +} + +func (m *ColumnsColumnar) GetFlag() []uint32 { + if m != nil { + return m.Flag + } + return nil +} + +func (m *ColumnsColumnar) GetValue() [][]byte { + if m != nil { + return m.Value + } + return nil +} + +type RowChangedColumnar struct { + OldValue []*ColumnsColumnar `protobuf:"bytes,1,rep,name=old_value,json=oldValue,proto3" json:"old_value,omitempty"` + NewValue []*ColumnsColumnar `protobuf:"bytes,2,rep,name=new_value,json=newValue,proto3" json:"new_value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RowChangedColumnar) Reset() { *m = RowChangedColumnar{} } +func (m *RowChangedColumnar) String() string { return proto.CompactTextString(m) } +func (*RowChangedColumnar) ProtoMessage() {} +func (*RowChangedColumnar) Descriptor() ([]byte, []int) { + return fileDescriptor_14a6b0a2c8322bed, []int{5} +} +func (m *RowChangedColumnar) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RowChangedColumnar) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_RowChangedColumnar.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *RowChangedColumnar) XXX_Merge(src proto.Message) { + xxx_messageInfo_RowChangedColumnar.Merge(m, src) +} +func (m *RowChangedColumnar) XXX_Size() int { + return m.Size() +} +func (m *RowChangedColumnar) XXX_DiscardUnknown() { + xxx_messageInfo_RowChangedColumnar.DiscardUnknown(m) +} + +var xxx_messageInfo_RowChangedColumnar proto.InternalMessageInfo + +func (m *RowChangedColumnar) GetOldValue() []*ColumnsColumnar { + if m != nil { + return m.OldValue + } + return nil +} + +func (m *RowChangedColumnar) GetNewValue() []*ColumnsColumnar { + if m != nil { + return m.NewValue + } + return nil +} + +func init() { + proto.RegisterType((*Key)(nil), "benchmark.Key") + proto.RegisterType((*Column)(nil), "benchmark.Column") + proto.RegisterType((*RowChanged)(nil), "benchmark.RowChanged") + proto.RegisterType((*KeysColumnar)(nil), "benchmark.KeysColumnar") + proto.RegisterType((*ColumnsColumnar)(nil), "benchmark.ColumnsColumnar") + proto.RegisterType((*RowChangedColumnar)(nil), "benchmark.RowChangedColumnar") +} + +func init() { proto.RegisterFile("CraftBenchmark.proto", fileDescriptor_14a6b0a2c8322bed) } + +var fileDescriptor_14a6b0a2c8322bed = []byte{ + // 430 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x93, 0x31, 0x6f, 0xd4, 0x30, + 0x14, 0xc7, 0x71, 0x9c, 0x8b, 0x9a, 0xd7, 0x6b, 0x11, 0x56, 0x41, 0x01, 0xa1, 0x53, 0xc8, 0x94, + 0xc9, 0x03, 0x0c, 0x9d, 0x58, 0x7a, 0x4b, 0x51, 0x37, 0x0f, 0x0c, 0x2c, 0x27, 0x27, 0x76, 0x2f, + 0x11, 0x89, 0x7d, 0x72, 0x5c, 0xa2, 0xfb, 0x02, 0x9d, 0x58, 0x91, 0xf8, 0x48, 0x8c, 0x7c, 0x04, + 0x74, 0x7c, 0x11, 0x14, 0xe7, 0x9a, 0xf4, 0x74, 0x29, 0x9b, 0xdf, 0xf3, 0xff, 0x3d, 0xfd, 0xf5, + 0xfb, 0xeb, 0xc1, 0xc5, 0xd2, 0xf0, 0x5b, 0x7b, 0x25, 0x55, 0x5e, 0xd4, 0xdc, 0x7c, 0xa5, 0x1b, + 0xa3, 0xad, 0x26, 0x61, 0xf6, 0xd0, 0x48, 0xbe, 0x23, 0xc0, 0x37, 0x72, 0x4b, 0xce, 0xc1, 0xb3, + 0x4d, 0x84, 0x62, 0x94, 0xfa, 0xcc, 0xb3, 0x0d, 0x79, 0x05, 0x41, 0x93, 0x17, 0xb2, 0xe6, 0x91, + 0x17, 0xa3, 0x34, 0x64, 0xfb, 0x8a, 0x5c, 0xc0, 0xcc, 0xf2, 0xac, 0x92, 0x11, 0x76, 0xed, 0xbe, + 0x20, 0x2f, 0x21, 0x30, 0xba, 0x5d, 0x95, 0x22, 0xf2, 0x63, 0x94, 0x62, 0x36, 0x33, 0xba, 0xfd, + 0x24, 0x08, 0x01, 0xdf, 0x6e, 0x37, 0x32, 0x9a, 0xc5, 0x28, 0x3d, 0x63, 0xee, 0x4d, 0xde, 0x42, + 0xb8, 0xe1, 0xc6, 0x96, 0xb6, 0xd4, 0x2a, 0x0a, 0x9c, 0x7a, 0x6c, 0x24, 0x5f, 0x20, 0x58, 0xea, + 0xea, 0xae, 0x56, 0xdd, 0xac, 0xe2, 0xb5, 0x74, 0x96, 0x42, 0xe6, 0xde, 0xc3, 0x3e, 0xef, 0xd1, + 0x3e, 0x02, 0xfe, 0x6d, 0xc5, 0xd7, 0xce, 0xcf, 0x19, 0x73, 0xef, 0xce, 0xe4, 0x37, 0x5e, 0xdd, + 0x49, 0xe7, 0x66, 0xce, 0xfa, 0x22, 0xa9, 0x00, 0x98, 0x6e, 0x97, 0x05, 0x57, 0x6b, 0x29, 0x08, + 0x85, 0x50, 0x57, 0x62, 0xd5, 0xeb, 0x50, 0x8c, 0xd3, 0xd3, 0xf7, 0x2f, 0xe8, 0xc0, 0x85, 0xf6, + 0x2e, 0xd8, 0x89, 0xae, 0xc4, 0xe7, 0x4e, 0xd2, 0xe9, 0x95, 0x6c, 0xf7, 0x7a, 0xef, 0x49, 0xbd, + 0x92, 0xad, 0xd3, 0x27, 0x3f, 0x10, 0xcc, 0x6f, 0xe4, 0xb6, 0xe9, 0x3f, 0xb8, 0x19, 0x08, 0xe3, + 0x09, 0xc2, 0x78, 0x9a, 0x30, 0x9e, 0x26, 0x8c, 0xa7, 0x08, 0xe3, 0xa7, 0x08, 0xe3, 0x43, 0xc2, + 0xf7, 0x08, 0x9e, 0xf7, 0x9e, 0x46, 0x6b, 0x23, 0x6b, 0x3c, 0xc1, 0x7a, 0xdc, 0xfc, 0x0e, 0xe6, + 0x6d, 0x21, 0x8d, 0x5c, 0x15, 0x5c, 0x89, 0xbd, 0xc3, 0x13, 0x76, 0xea, 0x7a, 0xd7, 0xae, 0x35, + 0xc4, 0xe1, 0xf7, 0x63, 0x87, 0x71, 0x74, 0x2e, 0x87, 0x38, 0xee, 0x11, 0x90, 0x31, 0x8f, 0xc1, + 0xcb, 0xe5, 0x71, 0x2e, 0x6f, 0x8e, 0x38, 0x0f, 0xd6, 0x1f, 0x05, 0x74, 0x79, 0x1c, 0xd0, 0x7f, + 0x07, 0x1f, 0x92, 0xba, 0xfa, 0xf8, 0x6b, 0xb7, 0x40, 0xbf, 0x77, 0x0b, 0xf4, 0x67, 0xb7, 0x40, + 0x3f, 0xff, 0x2e, 0x9e, 0xc1, 0xeb, 0x52, 0x53, 0x5b, 0x8a, 0x8c, 0x66, 0xe5, 0x5a, 0x70, 0xcb, + 0x69, 0x2e, 0x72, 0x9a, 0xbb, 0x4b, 0x3a, 0x3f, 0x3c, 0xa8, 0x6b, 0x94, 0x05, 0xee, 0xa6, 0x3e, + 0xfc, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x8e, 0xf5, 0x0b, 0xd6, 0x6b, 0x03, 0x00, 0x00, +} + +func (m *Key) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Key) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Key) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Partition != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.Partition)) + i-- + dAtA[i] = 0x30 + } + if m.Type != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x28 + } + if m.RowId != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.RowId)) + i-- + dAtA[i] = 0x20 + } + if len(m.Table) > 0 { + i -= len(m.Table) + copy(dAtA[i:], m.Table) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Table))) + i-- + dAtA[i] = 0x1a + } + if len(m.Schema) > 0 { + i -= len(m.Schema) + copy(dAtA[i:], m.Schema) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Schema))) + i-- + dAtA[i] = 0x12 + } + if m.Ts != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.Ts)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Column) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Column) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Column) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x22 + } + if m.Flag != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.Flag)) + i-- + dAtA[i] = 0x18 + } + if m.Type != 0 { + i = encodeVarintCraftBenchmark(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x10 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *RowChanged) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RowChanged) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowChanged) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.NewValue) > 0 { + for iNdEx := len(m.NewValue) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NewValue[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintCraftBenchmark(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.OldValue) > 0 { + for iNdEx := len(m.OldValue) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.OldValue[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintCraftBenchmark(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *KeysColumnar) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *KeysColumnar) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *KeysColumnar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Partition) > 0 { + dAtA2 := make([]byte, len(m.Partition)*10) + var j1 int + for _, num1 := range m.Partition { + num := uint64(num1) + for num >= 1<<7 { + dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA2[j1] = uint8(num) + j1++ + } + i -= j1 + copy(dAtA[i:], dAtA2[:j1]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j1)) + i-- + dAtA[i] = 0x32 + } + if len(m.Type) > 0 { + dAtA4 := make([]byte, len(m.Type)*10) + var j3 int + for _, num := range m.Type { + for num >= 1<<7 { + dAtA4[j3] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j3++ + } + dAtA4[j3] = uint8(num) + j3++ + } + i -= j3 + copy(dAtA[i:], dAtA4[:j3]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j3)) + i-- + dAtA[i] = 0x2a + } + if len(m.RowId) > 0 { + dAtA6 := make([]byte, len(m.RowId)*10) + var j5 int + for _, num1 := range m.RowId { + num := uint64(num1) + for num >= 1<<7 { + dAtA6[j5] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j5++ + } + dAtA6[j5] = uint8(num) + j5++ + } + i -= j5 + copy(dAtA[i:], dAtA6[:j5]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j5)) + i-- + dAtA[i] = 0x22 + } + if len(m.Table) > 0 { + for iNdEx := len(m.Table) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Table[iNdEx]) + copy(dAtA[i:], m.Table[iNdEx]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Table[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.Schema) > 0 { + for iNdEx := len(m.Schema) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Schema[iNdEx]) + copy(dAtA[i:], m.Schema[iNdEx]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Schema[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.Ts) > 0 { + dAtA8 := make([]byte, len(m.Ts)*10) + var j7 int + for _, num := range m.Ts { + for num >= 1<<7 { + dAtA8[j7] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j7++ + } + dAtA8[j7] = uint8(num) + j7++ + } + i -= j7 + copy(dAtA[i:], dAtA8[:j7]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j7)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ColumnsColumnar) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ColumnsColumnar) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ColumnsColumnar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Value) > 0 { + for iNdEx := len(m.Value) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Value[iNdEx]) + copy(dAtA[i:], m.Value[iNdEx]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Value[iNdEx]))) + i-- + dAtA[i] = 0x2a + } + } + if len(m.Flag) > 0 { + dAtA10 := make([]byte, len(m.Flag)*10) + var j9 int + for _, num := range m.Flag { + for num >= 1<<7 { + dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j9++ + } + dAtA10[j9] = uint8(num) + j9++ + } + i -= j9 + copy(dAtA[i:], dAtA10[:j9]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j9)) + i-- + dAtA[i] = 0x22 + } + if len(m.WhereHandle) > 0 { + for iNdEx := len(m.WhereHandle) - 1; iNdEx >= 0; iNdEx-- { + i-- + if m.WhereHandle[iNdEx] { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + } + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.WhereHandle))) + i-- + dAtA[i] = 0x1a + } + if len(m.Type) > 0 { + dAtA12 := make([]byte, len(m.Type)*10) + var j11 int + for _, num := range m.Type { + for num >= 1<<7 { + dAtA12[j11] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j11++ + } + dAtA12[j11] = uint8(num) + j11++ + } + i -= j11 + copy(dAtA[i:], dAtA12[:j11]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(j11)) + i-- + dAtA[i] = 0x12 + } + if len(m.Name) > 0 { + for iNdEx := len(m.Name) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Name[iNdEx]) + copy(dAtA[i:], m.Name[iNdEx]) + i = encodeVarintCraftBenchmark(dAtA, i, uint64(len(m.Name[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *RowChangedColumnar) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RowChangedColumnar) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowChangedColumnar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.NewValue) > 0 { + for iNdEx := len(m.NewValue) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NewValue[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintCraftBenchmark(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.OldValue) > 0 { + for iNdEx := len(m.OldValue) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.OldValue[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintCraftBenchmark(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func encodeVarintCraftBenchmark(dAtA []byte, offset int, v uint64) int { + offset -= sovCraftBenchmark(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Key) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Ts != 0 { + n += 1 + sovCraftBenchmark(uint64(m.Ts)) + } + l = len(m.Schema) + if l > 0 { + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + l = len(m.Table) + if l > 0 { + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + if m.RowId != 0 { + n += 1 + sovCraftBenchmark(uint64(m.RowId)) + } + if m.Type != 0 { + n += 1 + sovCraftBenchmark(uint64(m.Type)) + } + if m.Partition != 0 { + n += 1 + sovCraftBenchmark(uint64(m.Partition)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Column) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + if m.Type != 0 { + n += 1 + sovCraftBenchmark(uint64(m.Type)) + } + if m.Flag != 0 { + n += 1 + sovCraftBenchmark(uint64(m.Flag)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *RowChanged) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.OldValue) > 0 { + for _, e := range m.OldValue { + l = e.Size() + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if len(m.NewValue) > 0 { + for _, e := range m.NewValue { + l = e.Size() + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *KeysColumnar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Ts) > 0 { + l = 0 + for _, e := range m.Ts { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if len(m.Schema) > 0 { + for _, s := range m.Schema { + l = len(s) + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if len(m.Table) > 0 { + for _, s := range m.Table { + l = len(s) + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if len(m.RowId) > 0 { + l = 0 + for _, e := range m.RowId { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if len(m.Type) > 0 { + l = 0 + for _, e := range m.Type { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if len(m.Partition) > 0 { + l = 0 + for _, e := range m.Partition { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ColumnsColumnar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Name) > 0 { + for _, s := range m.Name { + l = len(s) + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if len(m.Type) > 0 { + l = 0 + for _, e := range m.Type { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if len(m.WhereHandle) > 0 { + n += 1 + sovCraftBenchmark(uint64(len(m.WhereHandle))) + len(m.WhereHandle)*1 + } + if len(m.Flag) > 0 { + l = 0 + for _, e := range m.Flag { + l += sovCraftBenchmark(uint64(e)) + } + n += 1 + sovCraftBenchmark(uint64(l)) + l + } + if len(m.Value) > 0 { + for _, b := range m.Value { + l = len(b) + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *RowChangedColumnar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.OldValue) > 0 { + for _, e := range m.OldValue { + l = e.Size() + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if len(m.NewValue) > 0 { + for _, e := range m.NewValue { + l = e.Size() + n += 1 + l + sovCraftBenchmark(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovCraftBenchmark(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozCraftBenchmark(x uint64) (n int) { + return sovCraftBenchmark(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Key) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Key: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Key: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Ts", wireType) + } + m.Ts = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Ts |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Schema = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Table", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Table = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowId", wireType) + } + m.RowId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RowId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Partition", wireType) + } + m.Partition = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Partition |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Column) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Column: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Column: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Flag", wireType) + } + m.Flag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Flag |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RowChanged) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RowChanged: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RowChanged: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldValue", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldValue = append(m.OldValue, &Column{}) + if err := m.OldValue[len(m.OldValue)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewValue", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewValue = append(m.NewValue, &Column{}) + if err := m.NewValue[len(m.NewValue)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *KeysColumnar) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: KeysColumnar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: KeysColumnar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Ts = append(m.Ts, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Ts) == 0 { + m.Ts = make([]uint64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Ts = append(m.Ts, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Ts", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Schema = append(m.Schema, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Table", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Table = append(m.Table, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType == 0 { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RowId = append(m.RowId, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RowId) == 0 { + m.RowId = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.RowId = append(m.RowId, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field RowId", wireType) + } + case 5: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Type = append(m.Type, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Type) == 0 { + m.Type = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Type = append(m.Type, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + case 6: + if wireType == 0 { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Partition = append(m.Partition, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Partition) == 0 { + m.Partition = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Partition = append(m.Partition, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Partition", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ColumnsColumnar) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ColumnsColumnar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ColumnsColumnar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = append(m.Name, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 2: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Type = append(m.Type, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Type) == 0 { + m.Type = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Type = append(m.Type, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + case 3: + if wireType == 0 { + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.WhereHandle = append(m.WhereHandle, bool(v != 0)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen + if elementCount != 0 && len(m.WhereHandle) == 0 { + m.WhereHandle = make([]bool, 0, elementCount) + } + for iNdEx < postIndex { + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.WhereHandle = append(m.WhereHandle, bool(v != 0)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field WhereHandle", wireType) + } + case 4: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Flag = append(m.Flag, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Flag) == 0 { + m.Flag = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Flag = append(m.Flag, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Flag", wireType) + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value, make([]byte, postIndex-iNdEx)) + copy(m.Value[len(m.Value)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *RowChangedColumnar) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RowChangedColumnar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RowChangedColumnar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OldValue", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.OldValue = append(m.OldValue, &ColumnsColumnar{}) + if err := m.OldValue[len(m.OldValue)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NewValue", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthCraftBenchmark + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthCraftBenchmark + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NewValue = append(m.NewValue, &ColumnsColumnar{}) + if err := m.NewValue[len(m.NewValue)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCraftBenchmark(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCraftBenchmark + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipCraftBenchmark(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowCraftBenchmark + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthCraftBenchmark + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupCraftBenchmark + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthCraftBenchmark + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthCraftBenchmark = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowCraftBenchmark = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupCraftBenchmark = fmt.Errorf("proto: unexpected end of group") +) diff --git a/proto/generate-proto.sh b/proto/generate-proto.sh index 6dbd935be72..ad2ebc21f09 100755 --- a/proto/generate-proto.sh +++ b/proto/generate-proto.sh @@ -1,9 +1,11 @@ #!/usr/bin/env bash -echo "generate canal protocol code..." +echo "generate canal & craft benchmark protocol code..." [ ! -d ./canal ] && mkdir ./canal [ ! -d ./cdclog ] && mkdir ./cdclog +[ ! -d ./benchmark ] && mkdir ./benchmark protoc --gofast_out=./canal EntryProtocol.proto protoc --gofast_out=./canal CanalProtocol.proto +protoc --gofast_out=./benchmark CraftBenchmark.proto From 5cce40c566b373ef6f600259b9cac42429e9b3ab Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Thu, 22 Apr 2021 08:48:31 +0800 Subject: [PATCH 15/39] Fix column group & year type seralization bug Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 5 ++++- cdc/sink/codec/craft/model.go | 10 ++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index efc53802e2f..8060e1ea140 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -197,7 +197,10 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp case mysql.TypeFloat, mysql.TypeDouble: // value type for these mysql types are float64 return encodeFloat64(allocator.byteSlice(8)[:0], value.(float64)) - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: + case mysql.TypeYear: + // year is encoded as int64 + return encodeVarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(int64)) + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24: // value type for these mysql types are int64 or uint64 depends on flags if flag.IsUnsigned() { return encodeUvarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(uint64)) diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 3645f75db6c..3d04b2f4822 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -225,6 +225,9 @@ func decodeColumnGroup(bits []byte, allocator *SliceAllocator) (*columnGroup, er func newColumnGroup(allocator *SliceAllocator, ty byte, columns []*model.Column) (int, *columnGroup) { l := len(columns) + if l == 0 { + return 0, nil + } values := allocator.bytesSlice(l) names := allocator.stringSlice(l) types := allocator.uint64Slice(l) @@ -268,12 +271,15 @@ func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) } groups := allocator.columnGroupSlice(numGroups) estimatedSize := 0 + idx := 0 if size, group := newColumnGroup(allocator, columnGroupTypeNew, ev.Columns); group != nil { - groups[0] = group + groups[idx] = group + idx++ estimatedSize += size } if size, group := newColumnGroup(allocator, columnGroupTypeOld, ev.PreColumns); group != nil { - groups[1] = group + groups[idx] = group + idx++ estimatedSize += size } return estimatedSize, rowChangedEvent(groups) From 3f210646887ae0820a990eb5a58f17f17601bb9a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 24 Apr 2021 19:50:32 +0800 Subject: [PATCH 16/39] Force encoding year type to varint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/decoder.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index a1b3acc50e6..3cbdd223e28 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -333,7 +333,7 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ // value type for these mysql types are float64 _, f64, err := decodeFloat64(bits) return f64, err - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24: // value type for these mysql types are int64 or uint64 depends on flags if flag.IsUnsigned() { _, u64, err := decodeUvarint(bits) @@ -341,6 +341,9 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ } _, i64, err := decodeVarint(bits) return i64, err + case mysql.TypeYear: + _, i64, err := decodeVarint(bits) + return i64, err case mysql.TypeUnspecified: fallthrough case mysql.TypeNull: From 495ffb2b0b8356fc2818f2c2a023c9685bbf2c20 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Fri, 30 Apr 2021 12:15:40 +0800 Subject: [PATCH 17/39] Fix body size table Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 8060e1ea140..f59504f3365 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -218,11 +218,11 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp /// Message encoder type MessageEncoder struct { - bits []byte - sizeTables [][]uint64 - bodyStartOffset int - bodySize []uint64 - bodySizeIndex int + bits []byte + sizeTables [][]uint64 + bodyLastOffset int + bodySize []uint64 + bodySizeIndex int allocator *SliceAllocator } @@ -235,7 +235,8 @@ func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { } func (e *MessageEncoder) encodeBodySize() *MessageEncoder { - e.bodySize[e.bodySizeIndex] = uint64(len(e.bits) - e.bodyStartOffset) + e.bodySize[e.bodySizeIndex] = uint64(len(e.bits) - e.bodyLastOffset) + e.bodyLastOffset = len(e.bits) e.bodySizeIndex++ return e } @@ -255,7 +256,7 @@ func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { oldSize := len(e.bits) e.bodySize = e.allocator.uint64Slice(headers.count) e.bits = headers.encode(e.bits) - e.bodyStartOffset = len(e.bits) + e.bodyLastOffset = len(e.bits) e.sizeTables = append(e.sizeTables, e.allocator.oneUint64Slice(uint64(len(e.bits)-oldSize)), e.bodySize) return e } From aec1ee42b62464b421d2df2d4ffe271c3eece3af Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 22:18:39 +0800 Subject: [PATCH 18/39] Disable lint for not yet used utility functions Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/buffer.go | 23 +++++++++++++++++++++++ cdc/sink/codec/craft/decoder.go | 1 + cdc/sink/codec/craft/encoder.go | 2 ++ 3 files changed, 26 insertions(+) diff --git a/cdc/sink/codec/craft/buffer.go b/cdc/sink/codec/craft/buffer.go index a90b8e1b8a2..c2c59567252 100644 --- a/cdc/sink/codec/craft/buffer.go +++ b/cdc/sink/codec/craft/buffer.go @@ -34,6 +34,7 @@ type intSliceAllocator struct { offset int } +//nolint:unused func (b *intSliceAllocator) realloc(old []int, newSize int) []int { n := b.alloc(newSize) copy(n, old) @@ -54,6 +55,7 @@ func (b *intSliceAllocator) alloc(size int) []int { return result } +//nolint:unused func (b *intSliceAllocator) one(x int) []int { r := b.alloc(1) r[0] = x @@ -142,6 +144,7 @@ type stringSliceAllocator struct { offset int } +//nolint:unused func (b *stringSliceAllocator) realloc(old []string, newSize int) []string { n := b.alloc(newSize) copy(n, old) @@ -162,6 +165,7 @@ func (b *stringSliceAllocator) alloc(size int) []string { return result } +//nolint:unused func (b *stringSliceAllocator) one(x string) []string { r := b.alloc(1) r[0] = x @@ -214,6 +218,7 @@ type byteSliceAllocator struct { offset int } +//nolint:unused func (b *byteSliceAllocator) realloc(old []byte, newSize int) []byte { n := b.alloc(newSize) copy(n, old) @@ -234,6 +239,7 @@ func (b *byteSliceAllocator) alloc(size int) []byte { return result } +//nolint:unused func (b *byteSliceAllocator) one(x byte) []byte { r := b.alloc(1) r[0] = x @@ -250,6 +256,7 @@ type bytesSliceAllocator struct { offset int } +//nolint:unused func (b *bytesSliceAllocator) realloc(old [][]byte, newSize int) [][]byte { n := b.alloc(newSize) copy(n, old) @@ -270,6 +277,7 @@ func (b *bytesSliceAllocator) alloc(size int) [][]byte { return result } +//nolint:unused func (b *bytesSliceAllocator) one(x []byte) [][]byte { r := b.alloc(1) r[0] = x @@ -286,6 +294,7 @@ type columnGroupSliceAllocator struct { offset int } +//nolint:unused func (b *columnGroupSliceAllocator) realloc(old []*columnGroup, newSize int) []*columnGroup { n := b.alloc(newSize) copy(n, old) @@ -306,6 +315,7 @@ func (b *columnGroupSliceAllocator) alloc(size int) []*columnGroup { return result } +//nolint:unused func (b *columnGroupSliceAllocator) one(x *columnGroup) []*columnGroup { r := b.alloc(1) r[0] = x @@ -342,6 +352,7 @@ func (b *rowChangedEventSliceAllocator) alloc(size int) []rowChangedEvent { return result } +//nolint:unused func (b *rowChangedEventSliceAllocator) one(x rowChangedEvent) []rowChangedEvent { r := b.alloc(1) r[0] = x @@ -383,10 +394,12 @@ func (b *SliceAllocator) intSlice(size int) []int { return b.intAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneIntSlice(x int) []int { return b.intAllocator.one(x) } +//nolint:unused func (b *SliceAllocator) resizeIntSlice(old []int, newSize int) []int { return b.intAllocator.realloc(old, newSize) } @@ -419,10 +432,12 @@ func (b *SliceAllocator) stringSlice(size int) []string { return b.stringAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneStringSlice(x string) []string { return b.stringAllocator.one(x) } +//nolint:unused func (b *SliceAllocator) resizeStringSlice(old []string, newSize int) []string { return b.stringAllocator.realloc(old, newSize) } @@ -443,10 +458,12 @@ func (b *SliceAllocator) byteSlice(size int) []byte { return b.byteAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneByteSlice(x byte) []byte { return b.byteAllocator.one(x) } +//nolint:unused func (b *SliceAllocator) resizeByteSlice(old []byte, newSize int) []byte { return b.byteAllocator.realloc(old, newSize) } @@ -455,10 +472,12 @@ func (b *SliceAllocator) bytesSlice(size int) [][]byte { return b.bytesAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneBytesSlice(x []byte) [][]byte { return b.bytesAllocator.one(x) } +//nolint:unused func (b *SliceAllocator) resizeBytesSlice(old [][]byte, newSize int) [][]byte { return b.bytesAllocator.realloc(old, newSize) } @@ -467,18 +486,22 @@ func (b *SliceAllocator) columnGroupSlice(size int) []*columnGroup { return b.columnGroupAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneColumnGroupSlice(x *columnGroup) []*columnGroup { return b.columnGroupAllocator.one(x) } +//nolint:unused func (b *SliceAllocator) resizeColumnGroupSlice(old []*columnGroup, newSize int) []*columnGroup { return b.columnGroupAllocator.realloc(old, newSize) } +//nolint:unused func (b *SliceAllocator) rowChangedEventSlice(size int) []rowChangedEvent { return b.rowChangedEventAllocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneRowChangedEventSlice(x rowChangedEvent) []rowChangedEvent { return b.rowChangedEventAllocator.one(x) } diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 3cbdd223e28..87e16799290 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -189,6 +189,7 @@ func decodeNullableStringChunk(bits []byte, size int, allocator *SliceAllocator) return newBits, data, nil } +//nolint:unused func decodeBytesChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, [][]byte, error) { return doDecodeBytesChunk(bits, size, decodeUvarintLength, allocator) } diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index f59504f3365..8c40d1fa2b0 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -69,6 +69,7 @@ func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { return bits, i + 1 } +//nolint:unused func encodeBytes(bits []byte, data []byte) []byte { l := len(data) bits = encodeUvarint(bits, uint64(l)) @@ -108,6 +109,7 @@ func encodeNullableStringChunk(bits []byte, data []*string) []byte { return bits } +//nolint:unused func encodeBytesChunk(bits []byte, data [][]byte) []byte { for _, b := range data { bits = encodeUvarint(bits, uint64(len(b))) From c1af30485ef6397163efe39f4eab97fb99122156 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 22:29:29 +0800 Subject: [PATCH 19/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 0eee4510d33..615f06194bb 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -419,9 +419,13 @@ func benchmarkProtobuf1Decoding() []*model.RowChangedEvent { result := make([]*model.RowChangedEvent, 0, 4) for _, message := range codecPB1EncodedRowChanges { key := &benchmark.Key{} - key.Unmarshal(message.Key) + if err := key.Unmarshal(message.Key); err != nil { + panic(err) + } value := &benchmark.RowChanged{} - value.Unmarshal(message.Value) + if err := value.Unmarshal(message.Value); err != nil { + panic(err) + } ev := &model.RowChangedEvent{} ev.PreColumns = codecDecodeRowChangedPB1(value.OldValue) ev.Columns = codecDecodeRowChangedPB1(value.NewValue) @@ -465,9 +469,13 @@ func benchmarkProtobuf2Decoding() []*model.RowChangedEvent { result := make([]*model.RowChangedEvent, 0, 4) for _, message := range codecPB2EncodedRowChanges { keys := &benchmark.KeysColumnar{} - keys.Unmarshal(message.Key) + if err := keys.Unmarshal(message.Key); err != nil { + panic(err) + } values := &benchmark.RowChangedColumnar{} - values.Unmarshal(message.Value) + if err := values.Unmarshal(message.Value); err != nil { + panic(err) + } for i, ts := range keys.Ts { ev := &model.RowChangedEvent{} From b9c3e564a0ff4734bbf1a4cc2582a393d9ae261a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 22:34:49 +0800 Subject: [PATCH 20/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 615f06194bb..f4f2972d393 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -246,7 +246,7 @@ func codecEncodeRowChangedPB1ToMessage(events []*model.RowChangedEvent) []*MQMes } func codecEncodeRowChangedPB2ToMessage(events []*model.RowChangedEvent) []*MQMessage { - return []*MQMessage{&MQMessage{ + return []*MQMessage{{ Key: codecEncodeKeysPB2(events), Value: codecEncodeRowChangedPB2(events), }} From 8a14395faa048c3e1ca702a5fd01161f0c88fcec Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 22:53:52 +0800 Subject: [PATCH 21/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 2 ++ cdc/sink/codec/craft/buffer.go | 5 +++-- cdc/sink/codec/craft/decoder.go | 10 +++++++--- cdc/sink/codec/craft/encoder.go | 12 ++++++++---- cdc/sink/codec/craft/model.go | 21 +++++++++++++++++---- 5 files changed, 37 insertions(+), 13 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 8de60577d25..68e2c96a531 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -134,6 +134,7 @@ func NewCraftEventBatchEncoder() EventBatchEncoder { return NewCraftEventBatchEncoderWithAllocator(craft.NewSliceAllocator(64)) } +// NewCraftEventBatchEncoderWithAllocator creates a new CraftEventBatchEncoder with given allocator. func NewCraftEventBatchEncoderWithAllocator(allocator *craft.SliceAllocator) EventBatchEncoder { return &CraftEventBatchEncoder{ allocator: allocator, @@ -242,6 +243,7 @@ func NewCraftEventBatchDecoder(bits []byte) (EventBatchDecoder, error) { return NewCraftEventBatchDecoderWithAllocator(bits, craft.NewSliceAllocator(64)) } +// NewCraftEventBatchDecoderWithAllocator creates a new CraftEventBatchDecoder with given allocator. func NewCraftEventBatchDecoderWithAllocator(bits []byte, allocator *craft.SliceAllocator) (EventBatchDecoder, error) { decoder, err := craft.NewMessageDecoder(bits, allocator) if err != nil { diff --git a/cdc/sink/codec/craft/buffer.go b/cdc/sink/codec/craft/buffer.go index c2c59567252..b6fe098a473 100644 --- a/cdc/sink/codec/craft/buffer.go +++ b/cdc/sink/codec/craft/buffer.go @@ -13,7 +13,7 @@ package craft -/// Utility functions for buffer allocation +// Utility functions for buffer allocation func newBufferSize(oldSize int) int { var newSize int if oldSize > 128 { @@ -363,7 +363,7 @@ func newRowChangedEventSliceAllocator(batchSize int) *rowChangedEventSliceAlloca return &rowChangedEventSliceAllocator{buffer: make([]rowChangedEvent, batchSize)} } -// allocator for different slice types +// SliceAllocator for different slice types type SliceAllocator struct { intAllocator *intSliceAllocator int64Allocator *int64SliceAllocator @@ -376,6 +376,7 @@ type SliceAllocator struct { rowChangedEventAllocator *rowChangedEventSliceAllocator } +// NewSliceAllocator creates a new slice allocator with given batch allocation size. func NewSliceAllocator(batchSize int) *SliceAllocator { return &SliceAllocator{ intAllocator: newIntSliceAllocator(batchSize), diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 87e16799290..ef5178c9af9 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -142,7 +142,7 @@ func decodeString(bits []byte) ([]byte, string, error) { return bits, "", errors.Trace(err) } -/// Chunk decoders +// Chunk decoders func decodeStringChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []string, error) { larray := allocator.intSlice(size) newBits := bits @@ -313,7 +313,7 @@ func decodeSizeTables(bits []byte, allocator *SliceAllocator) (int, [][]uint64, return tableSize, result, nil } -/// TiDB types decoder +// DecodeTiDBType decodes TiDB types. func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{}, error) { if bits == nil { return nil, nil @@ -355,7 +355,7 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ return nil, nil } -// Message decoder +// MessageDecoder decoder type MessageDecoder struct { bits []byte sizeTables [][]uint64 @@ -363,6 +363,7 @@ type MessageDecoder struct { allocator *SliceAllocator } +// NewMessageDecoder create a new message decode with bits and allocator func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, error) { bits, version, err := decodeUvarint(bits) if err != nil { @@ -383,6 +384,7 @@ func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, }, nil } +// Headers decode headers of message func (d *MessageDecoder) Headers() (*Headers, error) { var pairs, headersSize int var err error @@ -409,6 +411,7 @@ func (d *MessageDecoder) bodyBits(index int) []byte { return d.bits[start:int(d.bodySizeTable[index])] } +// DDLEvent decode a DDL event func (d *MessageDecoder) DDLEvent(index int) (pmodel.ActionType, string, error) { bits, ty, err := decodeUvarint(d.bodyBits(index)) if err != nil { @@ -418,6 +421,7 @@ func (d *MessageDecoder) DDLEvent(index int) (pmodel.ActionType, string, error) return pmodel.ActionType(ty), query, err } +// RowChangedEvent decode a row changeded event func (d *MessageDecoder) RowChangedEvent(index int) (preColumns, columns *columnGroup, err error) { bits := d.bodyBits(index) columnGroupSizeTable := d.sizeTables[columnGroupSizeTableStartIndex+index] diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 8c40d1fa2b0..965a993811b 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/ticdc/cdc/model" ) -/// create byte slice from string without copying +// create byte slice from string without copying func unsafeStringToBytes(s string) []byte { return *(*[]byte)(unsafe.Pointer( &struct { @@ -32,7 +32,7 @@ func unsafeStringToBytes(s string) []byte { )) } -/// Primitive type encoders +// Primitive type encoders func encodeFloat64(bits []byte, data float64) []byte { v := math.Float64bits(data) return append(bits, byte(v), byte(v>>8), byte(v>>16), byte(v>>24), byte(v>>32), byte(v>>40), byte(v>>48), byte(v>>56)) @@ -180,7 +180,7 @@ func encodeSizeTables(bits []byte, tables [][]uint64) []byte { return bits } -/// TiDB types encoder +// EncodeTiDBType encodes TiDB types func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagType, value interface{}) []byte { if value == nil { return nil @@ -218,7 +218,7 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp return nil } -/// Message encoder +// MessageEncoder type MessageEncoder struct { bits []byte sizeTables [][]uint64 @@ -229,6 +229,7 @@ type MessageEncoder struct { allocator *SliceAllocator } +// NewMessageEncoder creates a new encoder with given allocator func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { return &MessageEncoder{ bits: encodeUvarint(make([]byte, 0, DefaultBufferCapacity), Version1), @@ -263,6 +264,7 @@ func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { return e } +// Encode message into bits func (e *MessageEncoder) Encode() []byte { return encodeSizeTables(e.bits, e.sizeTables) } @@ -283,6 +285,7 @@ func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *Messag return e } +// NewResolvedEventEncoder creates a new encoder with given allocator and timestsamp func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncoder { return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), @@ -295,6 +298,7 @@ func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncod }).encodeBodySize() } +// NewDDLEventEncoder creates a new encoder with given allocator and timestsamp func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageEncoder { ty := uint64(ev.Type) query := ev.Query diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 3d04b2f4822..9c421f192c5 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -22,7 +22,7 @@ const ( // Version1 represents the version of craft format Version1 uint64 = 1 - // default buffer size + // DefaultBufferCapacity is default buffer size DefaultBufferCapacity = 1024 // Column group types @@ -35,7 +35,7 @@ const ( columnGroupSizeTableStartIndex = 2 ) -/// Headers in columnar layout +// Headers in columnar layout type Headers struct { ts []uint64 ty []uint64 @@ -47,6 +47,7 @@ type Headers struct { count int } +// Count returns number of headers func (h *Headers) Count() int { return h.count } @@ -87,18 +88,22 @@ func (h *Headers) reset() { h.count = 0 } +// GetType returns type of event at given index func (h *Headers) GetType(index int) model.MqMessageType { return model.MqMessageType(h.ty[index]) } +// GetTs returns timestamp of event at given index func (h *Headers) GetTs(index int) uint64 { return h.ts[index] } +// GetPartition returns partition of event at given index func (h *Headers) GetPartition(index int) int64 { return h.partition[index] } +// GetSchema returns schema of event at given index func (h *Headers) GetSchema(index int) string { if h.schema[index] != nil { return *h.schema[index] @@ -106,6 +111,7 @@ func (h *Headers) GetSchema(index int) string { return "" } +// GetTable returns table of event at given index func (h *Headers) GetTable(index int) string { if h.table[index] != nil { return *h.table[index] @@ -166,6 +172,7 @@ func (g *columnGroup) encode(bits []byte) []byte { return bits } +// ToModel converts column group into model func (g *columnGroup) ToModel() ([]*model.Column, error) { columns := make([]*model.Column, len(g.names)) for i, name := range g.names { @@ -258,7 +265,7 @@ func newColumnGroup(allocator *SliceAllocator, ty byte, columns []*model.Column) return estimatedSize, nil } -/// Row changed message is basically an array of column groups +// Row changed message is basically an array of column groups type rowChangedEvent = []*columnGroup func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) (int, rowChangedEvent) { @@ -285,7 +292,7 @@ func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) return estimatedSize, rowChangedEvent(groups) } -/// A buffer to save row changed events in batch +// RowChangedEventBuffer is a buffer to save row changed events in batch type RowChangedEventBuffer struct { headers *Headers @@ -296,6 +303,7 @@ type RowChangedEventBuffer struct { allocator *SliceAllocator } +// NewRowChangedEventBuffer creates new row changed event buffer with given allocator func NewRowChangedEventBuffer(allocator *SliceAllocator) *RowChangedEventBuffer { return &RowChangedEventBuffer{ headers: &Headers{}, @@ -303,12 +311,14 @@ func NewRowChangedEventBuffer(allocator *SliceAllocator) *RowChangedEventBuffer } } +// Encode row changed event buffer into bits func (b *RowChangedEventBuffer) Encode() []byte { bits := NewMessageEncoder(b.allocator).encodeHeaders(b.headers).encodeRowChangeEvents(b.events[:b.eventsCount]).Encode() b.Reset() return bits } +// AppendRowChangedEvent append a new event to buffer func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) (rows, size int) { var partition int64 = -1 if ev.Table.IsPartition { @@ -342,16 +352,19 @@ func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) return b.eventsCount, b.estimatedSize } +// Reset buffer func (b *RowChangedEventBuffer) Reset() { b.headers.reset() b.eventsCount = 0 b.estimatedSize = 0 } +// Size of buffer func (b *RowChangedEventBuffer) Size() int { return b.estimatedSize } +// GetHeaders returns headers of buffer func (b *RowChangedEventBuffer) GetHeaders() *Headers { return b.headers } From 2311f10c1c95b3df7275d61506c6fd0f01c9151d Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 22:56:34 +0800 Subject: [PATCH 22/39] Fix lint Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 965a993811b..fded9526a21 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -218,7 +218,7 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp return nil } -// MessageEncoder +// MessageEncoder is encoder for message type MessageEncoder struct { bits []byte sizeTables [][]uint64 From a29ab5b14446a02a05117322fb275b773bf25950 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 23:05:58 +0800 Subject: [PATCH 23/39] Fix lint --- cdc/sink/codec/craft/decoder.go | 2 +- cdc/sink/codec/craft/encoder.go | 4 ++-- cdc/sink/codec/craft/model.go | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index ef5178c9af9..f0b4e7b58ac 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -189,7 +189,7 @@ func decodeNullableStringChunk(bits []byte, size int, allocator *SliceAllocator) return newBits, data, nil } -//nolint:unused +//nolint:unused,deadcode func decodeBytesChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, [][]byte, error) { return doDecodeBytesChunk(bits, size, decodeUvarintLength, allocator) } diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index fded9526a21..2bcdda56109 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -69,7 +69,7 @@ func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { return bits, i + 1 } -//nolint:unused +//nolint:unused,deadcode func encodeBytes(bits []byte, data []byte) []byte { l := len(data) bits = encodeUvarint(bits, uint64(l)) @@ -109,7 +109,7 @@ func encodeNullableStringChunk(bits []byte, data []*string) []byte { return bits } -//nolint:unused +//nolint:unused,deadcode func encodeBytesChunk(bits []byte, data [][]byte) []byte { for _, b := range data { bits = encodeUvarint(bits, uint64(len(b))) diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 9c421f192c5..994da491e3e 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -286,7 +286,7 @@ func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) } if size, group := newColumnGroup(allocator, columnGroupTypeOld, ev.PreColumns); group != nil { groups[idx] = group - idx++ + idx += 1 estimatedSize += size } return estimatedSize, rowChangedEvent(groups) From f31ddeaced920b64c106f8a67ad98a8ec53def3b Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 23:10:34 +0800 Subject: [PATCH 24/39] Remove unnecessary assignment Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/model.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 994da491e3e..e5842bcd035 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -286,7 +286,6 @@ func newRowChangedMessage(allocator *SliceAllocator, ev *model.RowChangedEvent) } if size, group := newColumnGroup(allocator, columnGroupTypeOld, ev.PreColumns); group != nil { groups[idx] = group - idx += 1 estimatedSize += size } return estimatedSize, rowChangedEvent(groups) From dc24ba36991a2fbafa297fd7473a31f711b88ed5 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sat, 12 Jun 2021 23:23:32 +0800 Subject: [PATCH 25/39] Fix errdoc Signed-off-by: Xiaoguang Sun --- errors.toml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/errors.toml b/errors.toml index ce4f1665cba..19ea3eb9240 100755 --- a/errors.toml +++ b/errors.toml @@ -131,6 +131,11 @@ error = ''' codec decode error ''' +["CDC:ErrCraftCodecInvalidData"] +error = ''' +craft codec invalid data +''' + ["CDC:ErrCreateMarkTableFailed"] error = ''' create mark table failed @@ -691,6 +696,11 @@ error = ''' service safepoint lost. current safepoint is %d, please remove all changefeed(s) whose checkpoints are behind the current safepoint ''' +["CDC:ErrSinkInvalidConfig"] +error = ''' +sink config invalid +''' + ["CDC:ErrSinkURIInvalid"] error = ''' sink uri invalid From c7f1018114ca2a2903058f28503ca5b204cc771f Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sun, 13 Jun 2021 10:57:24 +0800 Subject: [PATCH 26/39] Change decoder to use body size table instead of offset table Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 8 ++++---- cdc/sink/codec/craft/decoder.go | 32 +++++++++++++++++++------------- 2 files changed, 23 insertions(+), 17 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index f4f2972d393..5ba5a1e51b3 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -43,7 +43,7 @@ var ( }, }}, {{ CommitTs: 424316553934667777, - Table: &model.TableName{Schema: "a", Table: "b"}, + Table: &model.TableName{Schema: "a", Table: "c"}, Columns: []*model.Column{ {Name: "varchar1", Type: mysql.TypeVarchar, Value: []byte("varchar")}, {Name: "string1", Type: mysql.TypeString, Value: []byte("string")}, @@ -56,7 +56,7 @@ var ( }, }, { CommitTs: 424316554327097345, - Table: &model.TableName{Schema: "a", Table: "b"}, + Table: &model.TableName{Schema: "a", Table: "d"}, Columns: []*model.Column{ {Name: "varchar2", Type: mysql.TypeVarchar, Value: []byte("varchar")}, {Name: "string2", Type: mysql.TypeString, Value: []byte("string")}, @@ -69,7 +69,7 @@ var ( }, }, { CommitTs: 424316554746789889, - Table: &model.TableName{Schema: "a", Table: "b"}, + Table: &model.TableName{Schema: "a", Table: "e"}, Columns: []*model.Column{ {Name: "varchar3", Type: mysql.TypeVarchar, Value: []byte("varchar")}, {Name: "string3", Type: mysql.TypeString, Value: []byte("string")}, @@ -82,7 +82,7 @@ var ( }, }, { CommitTs: 424316555073945601, - Table: &model.TableName{Schema: "a", Table: "c", TableID: 6, IsPartition: true}, + Table: &model.TableName{Schema: "a", Table: "f", TableID: 6, IsPartition: true}, Columns: []*model.Column{ {Name: "varchar4", Type: mysql.TypeVarchar, Value: []byte("varchar")}, {Name: "string4", Type: mysql.TypeString, Value: []byte("string")}, diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index f0b4e7b58ac..4c349df16d9 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -357,10 +357,10 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ // MessageDecoder decoder type MessageDecoder struct { - bits []byte - sizeTables [][]uint64 - bodySizeTable []uint64 - allocator *SliceAllocator + bits []byte + sizeTables [][]uint64 + bodyOffsetTable []int + allocator *SliceAllocator } // NewMessageDecoder create a new message decode with bits and allocator @@ -376,11 +376,21 @@ func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, if err != nil { return nil, errors.Trace(err) } + bodySizeTable := sizeTables[bodySizeTableIndex] + // build body offset table from size of each body + // offset table has number of bodies plus 1 elements + bodyOffsetTable := make([]int, len(bodySizeTable)+1) + start := 0 + for i, size := range bodySizeTable { + bodyOffsetTable[i] = start + start += int(size) + } + bodyOffsetTable[len(bodySizeTable)] = start return &MessageDecoder{ - bits: bits[:len(bits)-sizeTablesSize], - sizeTables: sizeTables, - bodySizeTable: sizeTables[bodySizeTableIndex], - allocator: allocator, + bits: bits[:len(bits)-sizeTablesSize], + sizeTables: sizeTables, + bodyOffsetTable: bodyOffsetTable, + allocator: allocator, }, nil } @@ -404,11 +414,7 @@ func (d *MessageDecoder) Headers() (*Headers, error) { } func (d *MessageDecoder) bodyBits(index int) []byte { - start := 0 - if index > 0 { - start = int(d.bodySizeTable[index-1]) - } - return d.bits[start:int(d.bodySizeTable[index])] + return d.bits[d.bodyOffsetTable[index]:d.bodyOffsetTable[index+1]] } // DDLEvent decode a DDL event From dfa49290cebb3729f45021427102528cec214afb Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sun, 13 Jun 2021 16:47:09 +0800 Subject: [PATCH 27/39] Fix json.Number.Int64() overflow Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/json.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index ff439d3d8e5..805d6e5cde6 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -122,14 +122,14 @@ func formatColumnVal(c column) column { } case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: 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)) - } + var err error if c.Flag.IsUnsigned() { - c.Value = uint64(intNum) + c.Value, err = strconv.ParseUint(s.String(), 10, 64) } else { - c.Value = intNum + c.Value, err = strconv.ParseInt(s.String(), 10, 64) + } + if err != nil { + log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) } } case mysql.TypeBit: From c3987fa087acc5b88ef96f0f26a8cf23de8d47c8 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Sun, 13 Jun 2021 17:47:23 +0800 Subject: [PATCH 28/39] Convert from float64 to int64 or uint64 --- cdc/sink/codec/json.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 805d6e5cde6..1eac1a015d7 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -131,6 +131,12 @@ func formatColumnVal(c column) column { if err != nil { log.Panic("invalid column value, please report a bug", zap.Any("col", c), zap.Error(err)) } + } else if f, ok := c.Value.(float64); ok { + if c.Flag.IsUnsigned() { + c.Value = uint64(f) + } else { + c.Value = int64(f) + } } case mysql.TypeBit: if s, ok := c.Value.(json.Number); ok { From 695358c9783f12069d3ccc08b68bde577f5f62e6 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 15 Jun 2021 14:17:55 +0800 Subject: [PATCH 29/39] Use c.Logf instead of fmt.Printf Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 5ba5a1e51b3..41043340226 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -16,7 +16,6 @@ package codec import ( "bytes" "compress/zlib" - "fmt" "testing" "github.com/pingcap/check" @@ -164,8 +163,8 @@ func (s *codecTestSuite) encodeRowCase(c *check.C, encoder EventBatchEncoder, ev func (s *codecTestSuite) TestJsonVsCraftVsPB(c *check.C) { defer testleak.AfterTest(c)() - fmt.Println("| case | craft size | json size | protobuf 1 size | protobuf 2 size | craft compressed | json compressed | protobuf 1 compressed | protobuf 2 compressed |") - fmt.Println("| :---- | :--------- | :-------- | :-------------- | :-------------- | :--------------- | :-------------- | :-------------------- | :-------------------- |") + c.Logf("| case | craft size | json size | protobuf 1 size | protobuf 2 size | craft compressed | json compressed | protobuf 1 compressed | protobuf 2 compressed |") + c.Logf("| :---- | :--------- | :-------- | :-------------- | :-------------- | :--------------- | :-------------- | :-------------------- | :-------------------- |") for i, cs := range codecRowCases { if len(cs) == 0 { continue @@ -180,7 +179,7 @@ func (s *codecTestSuite) TestJsonVsCraftVsPB(c *check.C) { jsonOriginal, jsonCompressed := s.checkCompressedSize(jsonMessages) protobuf1Original, protobuf1Compressed := s.checkCompressedSize(protobuf1Messages) protobuf2Original, protobuf2Compressed := s.checkCompressedSize(protobuf2Messages) - fmt.Printf("| case %d | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ |\n", i, + c.Logf("| case %d | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ |\n", i, craftOriginal, jsonOriginal, 100*jsonOriginal/craftOriginal-100, protobuf1Original, 100*protobuf1Original/craftOriginal-100, protobuf2Original, 100*protobuf2Original/craftOriginal-100, From c1437010f1c274e35862514844b81448cfa8ac16 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 15 Jun 2021 17:45:41 +0800 Subject: [PATCH 30/39] Seed random number generator in init function Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/codec_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/cdc/sink/codec/craft/codec_test.go b/cdc/sink/codec/craft/codec_test.go index f0c1cf3528f..f6b19b1f757 100644 --- a/cdc/sink/codec/craft/codec_test.go +++ b/cdc/sink/codec/craft/codec_test.go @@ -16,11 +16,16 @@ package craft import ( "math/rand" "testing" + "time" "github.com/pingcap/check" "github.com/pingcap/ticdc/pkg/util/testleak" ) +func init() { + rand.Seed(time.Now().UnixNano()) +} + var _ = check.Suite(&codecSuite{allocator: NewSliceAllocator(64)}) func Test(t *testing.T) { check.TestingT(t) } From c0ae25382cdfdacafe999912df270a7600efe829 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Tue, 15 Jun 2021 17:54:47 +0800 Subject: [PATCH 31/39] Add comments to varint encoding logic Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/decoder.go | 2 ++ cdc/sink/codec/craft/encoder.go | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 4c349df16d9..258000e8599 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -55,6 +55,8 @@ func decodeUvarint(bits []byte) ([]byte, uint64, error) { } func decodeUvarintReversed(bits []byte) (int, uint64, error) { + // Decode uint64 in varint format that is similar to protobuf but with bytes order reversed + // Reference: https://developers.google.com/protocol-buffers/docs/encoding#varints l := len(bits) - 1 var x uint64 var s uint diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 2bcdda56109..4c18f892a61 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -47,6 +47,8 @@ func encodeVarint(bits []byte, data int64) []byte { } func encodeUvarint(bits []byte, data uint64) []byte { + // Encode uint64 in varint format that is used in protobuf + // Reference: https://developers.google.com/protocol-buffers/docs/encoding#varints for data >= 0x80 { bits = append(bits, byte(data)|0x80) data >>= 7 @@ -55,6 +57,8 @@ func encodeUvarint(bits []byte, data uint64) []byte { } func encodeUvarintReversed(bits []byte, data uint64) ([]byte, int) { + // Encode uint64 in varint format that is similar to protobuf but with bytes order reversed + // Reference: https://developers.google.com/protocol-buffers/docs/encoding#varints buf := make([]byte, binary.MaxVarintLen64) i := 0 for data >= 0x80 { From b17d59ca65ae0f6bce33f42b274c7e56aef6d093 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Fri, 18 Jun 2021 16:09:39 +0800 Subject: [PATCH 32/39] Fix couple of typos and add some comments Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft.go | 18 ++++++++---------- cdc/sink/codec/craft/encoder.go | 6 +++--- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 68e2c96a531..03a4cb75cd2 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -23,11 +23,6 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" ) -const ( - // CraftVersion1 represents the version of craft format - CraftVersion1 uint64 = 1 -) - // CraftEventBatchEncoder encodes the events into the byte of a batch into craft binary format. type CraftEventBatchEncoder struct { rowChangedBuffer *craft.RowChangedEventBuffer @@ -131,6 +126,9 @@ func (e *CraftEventBatchEncoder) SetParams(params map[string]string) error { // NewCraftEventBatchEncoder creates a new CraftEventBatchEncoder. func NewCraftEventBatchEncoder() EventBatchEncoder { + // 64 is a magic number that come up with these assumptions and manual benchmark. + // 1. Most table will not have more than 64 columns + // 2. It only worth allocating slices in batch for slices that's small enough return NewCraftEventBatchEncoderWithAllocator(craft.NewSliceAllocator(64)) } @@ -183,18 +181,18 @@ func (b *CraftEventBatchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, if !hasNext || ty != model.MqMessageTypeRow { return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("not found row changed event message") } - old, new, err := b.decoder.RowChangedEvent(b.index) + oldValue, newValue, err := b.decoder.RowChangedEvent(b.index) if err != nil { return nil, errors.Trace(err) } ev := &model.RowChangedEvent{} - if old != nil { - if ev.PreColumns, err = old.ToModel(); err != nil { + if oldValue != nil { + if ev.PreColumns, err = oldValue.ToModel(); err != nil { return nil, errors.Trace(err) } } - if new != nil { - if ev.Columns, err = new.ToModel(); err != nil { + if newValue != nil { + if ev.Columns, err = newValue.ToModel(); err != nil { return nil, errors.Trace(err) } } diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 4c18f892a61..cc2daa4b00b 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -194,7 +194,7 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp // value type for these mysql types are string return unsafeStringToBytes(value.(string)) case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: - // value type for thest mysql types are uint64 + // value type for these mysql types are uint64 return encodeUvarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(uint64)) case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: @@ -289,7 +289,7 @@ func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *Messag return e } -// NewResolvedEventEncoder creates a new encoder with given allocator and timestsamp +// NewResolvedEventEncoder creates a new encoder with given allocator and timestamp func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncoder { return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), @@ -302,7 +302,7 @@ func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncod }).encodeBodySize() } -// NewDDLEventEncoder creates a new encoder with given allocator and timestsamp +// NewDDLEventEncoder creates a new encoder with given allocator and timestamp func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageEncoder { ty := uint64(ev.Type) query := ev.Query From 686b55c2916761e568170ccfe07bcfcf8f04d938 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 11:33:06 +0800 Subject: [PATCH 33/39] Remove deprecated RowID from header Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 2 -- cdc/sink/codec/craft/model.go | 13 ++----------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index cc2daa4b00b..157f8c21be0 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -294,7 +294,6 @@ func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncod return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), - rowID: allocator.oneInt64Slice(-1), partition: allocator.oneInt64Slice(-1), schema: allocator.oneNullableStringSlice(nil), table: allocator.oneNullableStringSlice(nil), @@ -316,7 +315,6 @@ func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageE return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ev.CommitTs), ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), - rowID: allocator.oneInt64Slice(-1), partition: allocator.oneInt64Slice(-1), schema: allocator.oneNullableStringSlice(schema), table: allocator.oneNullableStringSlice(table), diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index e5842bcd035..2720dc8b7b9 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -39,7 +39,6 @@ const ( type Headers struct { ts []uint64 ty []uint64 - rowID []int64 partition []int64 schema []*string table []*string @@ -55,27 +54,24 @@ func (h *Headers) Count() int { func (h *Headers) encode(bits []byte) []byte { bits = encodeDeltaUvarintChunk(bits, h.ts[:h.count]) bits = encodeDeltaUvarintChunk(bits, h.ty[:h.count]) - bits = encodeDeltaVarintChunk(bits, h.rowID[:h.count]) bits = encodeDeltaVarintChunk(bits, h.partition[:h.count]) bits = encodeNullableStringChunk(bits, h.schema[:h.count]) bits = encodeNullableStringChunk(bits, h.table[:h.count]) return bits } -func (h *Headers) appendHeader(allocator *SliceAllocator, ts, ty uint64, rowID, partition int64, schema, table *string) int { +func (h *Headers) appendHeader(allocator *SliceAllocator, ts, ty uint64, partition int64, schema, table *string) int { idx := h.count if idx+1 > len(h.ty) { size := newBufferSize(idx) h.ts = allocator.resizeUint64Slice(h.ts, size) h.ty = allocator.resizeUint64Slice(h.ty, size) - h.rowID = allocator.resizeInt64Slice(h.rowID, size) h.partition = allocator.resizeInt64Slice(h.partition, size) h.schema = allocator.resizeNullableStringSlice(h.schema, size) h.table = allocator.resizeNullableStringSlice(h.table, size) } h.ts[idx] = ts h.ty[idx] = ty - h.rowID[idx] = rowID h.partition[idx] = partition h.schema[idx] = schema h.table[idx] = table @@ -121,7 +117,7 @@ func (h *Headers) GetTable(index int) string { func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Headers, error) { var ts, ty []uint64 - var rowID, partition []int64 + var partition []int64 var schema, table []*string var err error if bits, ts, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { @@ -130,9 +126,6 @@ func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Hea if bits, ty, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } - if bits, rowID, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { - return nil, errors.Trace(err) - } if bits, partition, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } @@ -145,7 +138,6 @@ func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Hea return &Headers{ ts: ts, ty: ty, - rowID: rowID, partition: partition, schema: schema, table: table, @@ -336,7 +328,6 @@ func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) b.allocator, ev.CommitTs, uint64(model.MqMessageTypeRow), - ev.RowID, partition, schema, table, From 618b773c558851800e49e18500ddf651c08f59b4 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 18:33:33 +0800 Subject: [PATCH 34/39] Optimize codec size 1. Remove deprecated RowID field from header 2. Use term dictionary to compress terms Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/codec_test.go | 122 +++++++++++++++++------- cdc/sink/codec/craft/decoder.go | 35 ++++++- cdc/sink/codec/craft/encoder.go | 23 +++-- cdc/sink/codec/craft/model.go | 162 +++++++++++++++++++++++++++++--- 4 files changed, 281 insertions(+), 61 deletions(-) diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 41043340226..2c0af292cdc 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -30,9 +30,9 @@ var ( codecRowCases = [][]*model.RowChangedEvent{{{ CommitTs: 424316552636792833, Table: &model.TableName{Schema: "a", Table: "b"}, - Columns: []*model.Column{ - {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar")}, - {Name: "string", Type: mysql.TypeString, Value: []byte("string")}, + PreColumns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, @@ -40,57 +40,107 @@ var ( {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, {Name: "null", Type: mysql.TypeNull, Value: nil}, }, + Columns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar1")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string1")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/02"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/02 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/02 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(2.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(2000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, }}, {{ CommitTs: 424316553934667777, Table: &model.TableName{Schema: "a", Table: "c"}, + PreColumns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, Columns: []*model.Column{ - {Name: "varchar1", Type: mysql.TypeVarchar, Value: []byte("varchar")}, - {Name: "string1", Type: mysql.TypeString, Value: []byte("string")}, - {Name: "date1", Type: mysql.TypeDate, Value: "2021/01/01"}, - {Name: "timestamp1", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, - {Name: "datetime1", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, - {Name: "float1", Type: mysql.TypeFloat, Value: float64(1.0)}, - {Name: "long1", Type: mysql.TypeLong, Value: int64(1000)}, - {Name: "null1", Type: mysql.TypeNull, Value: nil}, + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar1")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string1")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/02"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/02 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/02 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(2.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(2000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { CommitTs: 424316554327097345, Table: &model.TableName{Schema: "a", Table: "d"}, + PreColumns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, Columns: []*model.Column{ - {Name: "varchar2", Type: mysql.TypeVarchar, Value: []byte("varchar")}, - {Name: "string2", Type: mysql.TypeString, Value: []byte("string")}, - {Name: "date2", Type: mysql.TypeDate, Value: "2021/01/01"}, - {Name: "timestamp2", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, - {Name: "datetime2", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, - {Name: "float2", Type: mysql.TypeFloat, Value: float64(1.0)}, - {Name: "long2", Type: mysql.TypeLong, Value: int64(1000)}, - {Name: "null2", Type: mysql.TypeNull, Value: nil}, + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar1")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string1")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/02"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/02 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/02 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(2.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(2000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { CommitTs: 424316554746789889, Table: &model.TableName{Schema: "a", Table: "e"}, + PreColumns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, Columns: []*model.Column{ - {Name: "varchar3", Type: mysql.TypeVarchar, Value: []byte("varchar")}, - {Name: "string3", Type: mysql.TypeString, Value: []byte("string")}, - {Name: "date3", Type: mysql.TypeDate, Value: "2021/01/01"}, - {Name: "timestamp3", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, - {Name: "datetime3", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, - {Name: "float3", Type: mysql.TypeFloat, Value: float64(1.0)}, - {Name: "long3", Type: mysql.TypeLong, Value: int64(1000)}, - {Name: "null3", Type: mysql.TypeNull, Value: nil}, + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar1")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string1")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/02"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/02 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/02 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(2.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(2000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }, { CommitTs: 424316555073945601, Table: &model.TableName{Schema: "a", Table: "f", TableID: 6, IsPartition: true}, + PreColumns: []*model.Column{ + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar0")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string0")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/01"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(1.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(1000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, + }, Columns: []*model.Column{ - {Name: "varchar4", Type: mysql.TypeVarchar, Value: []byte("varchar")}, - {Name: "string4", Type: mysql.TypeString, Value: []byte("string")}, - {Name: "date4", Type: mysql.TypeDate, Value: "2021/01/01"}, - {Name: "timestamp4", Type: mysql.TypeTimestamp, Value: "2021/01/01 00:00:00"}, - {Name: "datetime4", Type: mysql.TypeDatetime, Value: "2021/01/01 00:00:00"}, - {Name: "float4", Type: mysql.TypeFloat, Value: float64(1.0)}, - {Name: "long4", Type: mysql.TypeLong, Value: int64(1000)}, - {Name: "null4", Type: mysql.TypeNull, Value: nil}, + {Name: "varchar", Type: mysql.TypeVarchar, Value: []byte("varchar1")}, + {Name: "string", Type: mysql.TypeString, Value: []byte("string1")}, + {Name: "date", Type: mysql.TypeDate, Value: "2021/01/02"}, + {Name: "timestamp", Type: mysql.TypeTimestamp, Value: "2021/01/02 00:00:00"}, + {Name: "datetime", Type: mysql.TypeDatetime, Value: "2021/01/02 00:00:00"}, + {Name: "float", Type: mysql.TypeFloat, Value: float64(2.0)}, + {Name: "long", Type: mysql.TypeLong, Value: int64(2000)}, + {Name: "null", Type: mysql.TypeNull, Value: nil}, }, }}, {}} @@ -179,7 +229,7 @@ func (s *codecTestSuite) TestJsonVsCraftVsPB(c *check.C) { jsonOriginal, jsonCompressed := s.checkCompressedSize(jsonMessages) protobuf1Original, protobuf1Compressed := s.checkCompressedSize(protobuf1Messages) protobuf2Original, protobuf2Compressed := s.checkCompressedSize(protobuf2Messages) - c.Logf("| case %d | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ |\n", i, + c.Logf("| case %d | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ | %d | %d (%d%%)+ | %d (%d%%)+ | %d (%d%%)+ |", i, craftOriginal, jsonOriginal, 100*jsonOriginal/craftOriginal-100, protobuf1Original, 100*protobuf1Original/craftOriginal-100, protobuf2Original, 100*protobuf2Original/craftOriginal-100, diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 258000e8599..4c2b09d37e2 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -361,8 +361,10 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ type MessageDecoder struct { bits []byte sizeTables [][]uint64 + metaSizeTable []uint64 bodyOffsetTable []int allocator *SliceAllocator + dict *termDictionary } // NewMessageDecoder create a new message decode with bits and allocator @@ -378,21 +380,43 @@ func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, if err != nil { return nil, errors.Trace(err) } + + // truncate tailing size tables + bits = bits[:len(bits)-sizeTablesSize] + + // get size table for each body element bodySizeTable := sizeTables[bodySizeTableIndex] // build body offset table from size of each body // offset table has number of bodies plus 1 elements + // TODO check bodyOffsetTable size - 1 bodyOffsetTable := make([]int, len(bodySizeTable)+1) + + // start offset of last body element start := 0 for i, size := range bodySizeTable { bodyOffsetTable[i] = start start += int(size) } + // start equals total size of body elements bodyOffsetTable[len(bodySizeTable)] = start + + // get meta data size table which contains size of headers and term dictionary + metaSizeTable := sizeTables[metaSizeTableIndex] + + // term dictionary offset starts from header size + body size + termDictionaryOffset := int(metaSizeTable[headerSizeIndex]) + start + termDictionaryEnd := termDictionaryOffset + int(metaSizeTable[termDictionarySizeIndex]) + _, dict, err := decodeTermDictionary(bits[termDictionaryOffset:termDictionaryEnd], allocator) + if err != nil { + return nil, errors.Trace(err) + } return &MessageDecoder{ - bits: bits[:len(bits)-sizeTablesSize], + bits: bits[:termDictionaryOffset], sizeTables: sizeTables, + metaSizeTable: metaSizeTable, bodyOffsetTable: bodyOffsetTable, allocator: allocator, + dict: dict, }, nil } @@ -400,13 +424,14 @@ func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, func (d *MessageDecoder) Headers() (*Headers, error) { var pairs, headersSize int var err error - d.bits, pairs, err = decodeUvarintLength(d.bits) + // get number of pairs from size of body size table + pairs = len(d.sizeTables[bodySizeTableIndex]) if err != nil { return nil, errors.Trace(err) } - headersSize = int(d.sizeTables[headerSizeTableIndex][0]) + headersSize = int(d.metaSizeTable[headerSizeIndex]) var headers *Headers - headers, err = decodeHeaders(d.bits[:headersSize], pairs, d.allocator) + headers, err = decodeHeaders(d.bits[:headersSize], pairs, d.allocator, d.dict) if err != nil { return nil, errors.Trace(err) } @@ -436,7 +461,7 @@ func (d *MessageDecoder) RowChangedEvent(index int) (preColumns, columns *column columnGroupIndex := 0 for len(bits) > 0 { columnGroupSize := columnGroupSizeTable[columnGroupIndex] - columnGroup, err := decodeColumnGroup(bits[:columnGroupSize], d.allocator) + columnGroup, err := decodeColumnGroup(bits[:columnGroupSize], d.allocator, d.dict) bits = bits[columnGroupSize:] columnGroupIndex++ if err != nil { diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 157f8c21be0..ee874c25795 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -229,8 +229,10 @@ type MessageEncoder struct { bodyLastOffset int bodySize []uint64 bodySizeIndex int + metaSizeTable []uint64 allocator *SliceAllocator + dict *termDictionary } // NewMessageEncoder creates a new encoder with given allocator @@ -238,6 +240,7 @@ func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { return &MessageEncoder{ bits: encodeUvarint(make([]byte, 0, DefaultBufferCapacity), Version1), allocator: allocator, + dict: newEncodingTermDictionary(), } } @@ -259,17 +262,21 @@ func (e *MessageEncoder) encodeString(s string) *MessageEncoder { } func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { - e.bits = encodeUvarint(e.bits, uint64(headers.count)) oldSize := len(e.bits) e.bodySize = e.allocator.uint64Slice(headers.count) - e.bits = headers.encode(e.bits) + e.bits = headers.encode(e.bits, e.dict) e.bodyLastOffset = len(e.bits) - e.sizeTables = append(e.sizeTables, e.allocator.oneUint64Slice(uint64(len(e.bits)-oldSize)), e.bodySize) + e.metaSizeTable = e.allocator.uint64Slice(maxMetaSizeIndex + 1) + e.metaSizeTable[headerSizeIndex] = uint64(len(e.bits) - oldSize) + e.sizeTables = append(e.sizeTables, e.metaSizeTable, e.bodySize) return e } // Encode message into bits func (e *MessageEncoder) Encode() []byte { + offset := len(e.bits) + e.bits = encodeTermDictionary(e.bits, e.dict) + e.metaSizeTable[termDictionarySizeIndex] = uint64(len(e.bits) - offset) return encodeSizeTables(e.bits, e.sizeTables) } @@ -279,7 +286,7 @@ func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *Messag columnGroupSizeTable := e.allocator.uint64Slice(len(event)) for gi, group := range event { oldSize := len(e.bits) - e.bits = group.encode(e.bits) + e.bits = group.encode(e.bits, e.dict) columnGroupSizeTable[gi] = uint64(len(e.bits) - oldSize) } sizeTables = append(sizeTables, columnGroupSizeTable) @@ -294,9 +301,9 @@ func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncod return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), - partition: allocator.oneInt64Slice(-1), - schema: allocator.oneNullableStringSlice(nil), - table: allocator.oneNullableStringSlice(nil), + partition: oneNullInt64Slice, + schema: oneNullStringSlice, + table: oneNullStringSlice, count: 1, }).encodeBodySize() } @@ -315,7 +322,7 @@ func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageE return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ev.CommitTs), ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), - partition: allocator.oneInt64Slice(-1), + partition: oneNullInt64Slice, schema: allocator.oneNullableStringSlice(schema), table: allocator.oneNullableStringSlice(table), count: 1, diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 2720dc8b7b9..05428f47126 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -16,6 +16,7 @@ package craft import ( "github.com/pingcap/errors" "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" ) const ( @@ -30,11 +31,137 @@ const ( columnGroupTypeNew = 0x1 // Size tables index - headerSizeTableIndex = 0 + metaSizeTableIndex = 0 bodySizeTableIndex = 1 columnGroupSizeTableStartIndex = 2 + + // meta size table index + headerSizeIndex = 0 + termDictionarySizeIndex = 1 + maxMetaSizeIndex = termDictionarySizeIndex + + nullInt64 = -1 +) + +var ( + oneNullInt64Slice = []int64{nullInt64} + oneNullStringSlice = []*string{nil} ) +type termDictionary struct { + term map[string]int + id []string +} + +func newEncodingTermDictionaryWithSize(size int) *termDictionary { + return &termDictionary{ + term: make(map[string]int), + id: make([]string, 0, size), + } +} + +func newEncodingTermDictionary() *termDictionary { + return newEncodingTermDictionaryWithSize(8) // TODO, this number should be evaluated +} + +func (d *termDictionary) encodeNullable(s *string) int64 { + if s == nil { + return nullInt64 + } + return int64(d.encode(*s)) +} + +func (d *termDictionary) encode(s string) uint64 { + if id, ok := d.term[s]; !ok { + id := len(d.id) + d.term[s] = id + d.id = append(d.id, s) + return uint64(id) + } else { + return uint64(id) + } +} + +func (d *termDictionary) encodeNullableChunk(array []*string) []int64 { + result := make([]int64, len(array)) + for idx, s := range array { + result[idx] = d.encodeNullable(s) + } + return result +} + +func (d *termDictionary) encodeChunk(array []string) []uint64 { + result := make([]uint64, len(array)) + for idx, s := range array { + result[idx] = d.encode(s) + } + return result +} + +func (d *termDictionary) decode(id uint64) (string, error) { + i := int(id) + if len(d.id) <= i || i < 0 { + return "", cerror.ErrCraftCodecInvalidData.GenWithStack("invalid term id") + } + return d.id[i], nil +} + +func (d *termDictionary) decodeNullable(id int64) (*string, error) { + if id == nullInt64 { + return nil, nil + } + if id < nullInt64 { + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid term id") + } + if s, err := d.decode(uint64(id)); err != nil { + return nil, err + } else { + return &s, nil + } +} + +func (d *termDictionary) decodeChunk(array []uint64) ([]string, error) { + result := make([]string, len(array)) + for idx, id := range array { + t, err := d.decode(id) + if err != nil { + return nil, err + } + result[idx] = t + } + return result, nil +} + +func (d *termDictionary) decodeNullableChunk(array []int64) ([]*string, error) { + result := make([]*string, len(array)) + for idx, id := range array { + t, err := d.decodeNullable(id) + if err != nil { + return nil, err + } + result[idx] = t + } + return result, nil +} + +func encodeTermDictionary(bits []byte, dict *termDictionary) []byte { + bits = encodeUvarint(bits, uint64(len(dict.id))) + bits = encodeStringChunk(bits, dict.id) + return bits +} + +func decodeTermDictionary(bits []byte, allocator *SliceAllocator) ([]byte, *termDictionary, error) { + newBits, l, err := decodeUvarint(bits) + if err != nil { + return bits, nil, err + } + newBits, id, err := decodeStringChunk(newBits, int(l), allocator) + if err != nil { + return bits, nil, err + } + return newBits, &termDictionary{id: id}, nil +} + // Headers in columnar layout type Headers struct { ts []uint64 @@ -51,12 +178,12 @@ func (h *Headers) Count() int { return h.count } -func (h *Headers) encode(bits []byte) []byte { +func (h *Headers) encode(bits []byte, dict *termDictionary) []byte { bits = encodeDeltaUvarintChunk(bits, h.ts[:h.count]) bits = encodeDeltaUvarintChunk(bits, h.ty[:h.count]) bits = encodeDeltaVarintChunk(bits, h.partition[:h.count]) - bits = encodeNullableStringChunk(bits, h.schema[:h.count]) - bits = encodeNullableStringChunk(bits, h.table[:h.count]) + bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.schema[:h.count])) + bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.table[:h.count])) return bits } @@ -115,9 +242,9 @@ func (h *Headers) GetTable(index int) string { return "" } -func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Headers, error) { +func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator, dict *termDictionary) (*Headers, error) { var ts, ty []uint64 - var partition []int64 + var partition, tmp []int64 var schema, table []*string var err error if bits, ts, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { @@ -129,10 +256,16 @@ func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator) (*Hea if bits, partition, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } - if bits, schema, err = decodeNullableStringChunk(bits, numHeaders, allocator); err != nil { + if bits, tmp, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } - if _, table, err = decodeNullableStringChunk(bits, numHeaders, allocator); err != nil { + if schema, err = dict.decodeNullableChunk(tmp); err != nil { + return nil, errors.Trace(err) + } + if _, tmp, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + if table, err = dict.decodeNullableChunk(tmp); err != nil { return nil, errors.Trace(err) } return &Headers{ @@ -154,10 +287,10 @@ type columnGroup struct { values [][]byte } -func (g *columnGroup) encode(bits []byte) []byte { +func (g *columnGroup) encode(bits []byte, dict *termDictionary) []byte { bits = append(bits, g.ty) bits = encodeUvarint(bits, uint64(len(g.names))) - bits = encodeStringChunk(bits, g.names) + bits = encodeDeltaUvarintChunk(bits, dict.encodeChunk(g.names)) bits = encodeUvarintChunk(bits, g.types) bits = encodeUvarintChunk(bits, g.flags) bits = encodeNullableBytesChunk(bits, g.values) @@ -184,7 +317,7 @@ func (g *columnGroup) ToModel() ([]*model.Column, error) { return columns, nil } -func decodeColumnGroup(bits []byte, allocator *SliceAllocator) (*columnGroup, error) { +func decodeColumnGroup(bits []byte, allocator *SliceAllocator, dict *termDictionary) (*columnGroup, error) { var numColumns int bits, ty, err := decodeUint8(bits) if err != nil { @@ -195,9 +328,14 @@ func decodeColumnGroup(bits []byte, allocator *SliceAllocator) (*columnGroup, er return nil, errors.Trace(err) } var names []string + var tmp []uint64 var values [][]byte var types, flags []uint64 - bits, names, err = decodeStringChunk(bits, numColumns, allocator) + bits, tmp, err = decodeDeltaUvarintChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + names, err = dict.decodeChunk(tmp) if err != nil { return nil, errors.Trace(err) } From 34a222f52023df7cca797856e840e52aa943e91e Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 18:56:55 +0800 Subject: [PATCH 35/39] Fix lints Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/buffer.go | 2 ++ cdc/sink/codec/craft/model.go | 12 ++++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/cdc/sink/codec/craft/buffer.go b/cdc/sink/codec/craft/buffer.go index b6fe098a473..bab76b89453 100644 --- a/cdc/sink/codec/craft/buffer.go +++ b/cdc/sink/codec/craft/buffer.go @@ -92,6 +92,7 @@ func (b *int64SliceAllocator) alloc(size int) []int64 { return result } +//nolint:unused func (b *int64SliceAllocator) one(x int64) []int64 { r := b.alloc(1) r[0] = x @@ -409,6 +410,7 @@ func (b *SliceAllocator) int64Slice(size int) []int64 { return b.int64Allocator.alloc(size) } +//nolint:unused func (b *SliceAllocator) oneInt64Slice(x int64) []int64 { return b.int64Allocator.one(x) } diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 05428f47126..aa3fb24f248 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -72,14 +72,14 @@ func (d *termDictionary) encodeNullable(s *string) int64 { } func (d *termDictionary) encode(s string) uint64 { - if id, ok := d.term[s]; !ok { + id, ok := d.term[s] + if !ok { id := len(d.id) d.term[s] = id d.id = append(d.id, s) return uint64(id) - } else { - return uint64(id) } + return uint64(id) } func (d *termDictionary) encodeNullableChunk(array []*string) []int64 { @@ -113,11 +113,11 @@ func (d *termDictionary) decodeNullable(id int64) (*string, error) { if id < nullInt64 { return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid term id") } - if s, err := d.decode(uint64(id)); err != nil { + s, err := d.decode(uint64(id)) + if err != nil { return nil, err - } else { - return &s, nil } + return &s, nil } func (d *termDictionary) decodeChunk(array []uint64) ([]string, error) { From 7c09d32578d368a012f249550cde8452d15ebb7b Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 21:16:36 +0800 Subject: [PATCH 36/39] Change size table to encode with varint chunk Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/codec_test.go | 2 +- cdc/sink/codec/craft/decoder.go | 27 ++++++++++++++++----------- cdc/sink/codec/craft/encoder.go | 24 ++++++++++++------------ cdc/sink/codec/craft/model.go | 10 ++++++++-- 4 files changed, 37 insertions(+), 26 deletions(-) diff --git a/cdc/sink/codec/craft/codec_test.go b/cdc/sink/codec/craft/codec_test.go index f6b19b1f757..38221a136bc 100644 --- a/cdc/sink/codec/craft/codec_test.go +++ b/cdc/sink/codec/craft/codec_test.go @@ -36,7 +36,7 @@ type codecSuite struct { func (s *codecSuite) TestSizeTable(c *check.C) { defer testleak.AfterTest(c)() - tables := [][]uint64{ + tables := [][]int64{ { 1, 3, 5, 7, 9, }, diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 4c2b09d37e2..1e51ae2ede5 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -290,7 +290,7 @@ func decodeDeltaUvarintChunk(bits []byte, size int, allocator *SliceAllocator) ( } // size tables are always at end of serialized data, there is no unread bytes to return -func decodeSizeTables(bits []byte, allocator *SliceAllocator) (int, [][]uint64, error) { +func decodeSizeTables(bits []byte, allocator *SliceAllocator) (int, [][]int64, error) { nb, size, _ := decodeUvarintReversedLength(bits) sizeOffset := len(bits) - nb tablesOffset := sizeOffset - size @@ -298,14 +298,14 @@ func decodeSizeTables(bits []byte, allocator *SliceAllocator) (int, [][]uint64, tableSize := size + nb var err error - var table []uint64 - result := make([][]uint64, 0, 1) + var table []int64 + result := make([][]int64, 0, 1) for len(tables) > 0 { tables, size, err = decodeUvarintLength(tables) if err != nil { return 0, nil, errors.Trace(err) } - tables, table, err = decodeDeltaUvarintChunk(tables, size, allocator) + tables, table, err = decodeDeltaVarintChunk(tables, size, allocator) if err != nil { return 0, nil, errors.Trace(err) } @@ -360,8 +360,8 @@ func DecodeTiDBType(ty byte, flag model.ColumnFlagType, bits []byte) (interface{ // MessageDecoder decoder type MessageDecoder struct { bits []byte - sizeTables [][]uint64 - metaSizeTable []uint64 + sizeTables [][]int64 + metaSizeTable []int64 bodyOffsetTable []int allocator *SliceAllocator dict *termDictionary @@ -403,12 +403,17 @@ func NewMessageDecoder(bits []byte, allocator *SliceAllocator) (*MessageDecoder, // get meta data size table which contains size of headers and term dictionary metaSizeTable := sizeTables[metaSizeTableIndex] - // term dictionary offset starts from header size + body size + var dict *termDictionary termDictionaryOffset := int(metaSizeTable[headerSizeIndex]) + start - termDictionaryEnd := termDictionaryOffset + int(metaSizeTable[termDictionarySizeIndex]) - _, dict, err := decodeTermDictionary(bits[termDictionaryOffset:termDictionaryEnd], allocator) - if err != nil { - return nil, errors.Trace(err) + if metaSizeTable[termDictionarySizeIndex] > 0 { + // term dictionary offset starts from header size + body size + termDictionaryEnd := termDictionaryOffset + int(metaSizeTable[termDictionarySizeIndex]) + _, dict, err = decodeTermDictionary(bits[termDictionaryOffset:termDictionaryEnd], allocator) + if err != nil { + return nil, errors.Trace(err) + } + } else { + dict = emptyDecodingTermDictionary } return &MessageDecoder{ bits: bits[:termDictionaryOffset], diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index ee874c25795..9a56af27ca4 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -174,11 +174,11 @@ func encodeDeltaUvarintChunk(bits []byte, data []uint64) []byte { return bits } -func encodeSizeTables(bits []byte, tables [][]uint64) []byte { +func encodeSizeTables(bits []byte, tables [][]int64) []byte { size := len(bits) for _, table := range tables { bits = encodeUvarint(bits, uint64(len(table))) - bits = encodeDeltaUvarintChunk(bits, table) + bits = encodeDeltaVarintChunk(bits, table) } bits, _ = encodeUvarintReversed(bits, uint64(len(bits)-size)) return bits @@ -225,11 +225,11 @@ func EncodeTiDBType(allocator *SliceAllocator, ty byte, flag model.ColumnFlagTyp // MessageEncoder is encoder for message type MessageEncoder struct { bits []byte - sizeTables [][]uint64 + sizeTables [][]int64 bodyLastOffset int - bodySize []uint64 + bodySize []int64 bodySizeIndex int - metaSizeTable []uint64 + metaSizeTable []int64 allocator *SliceAllocator dict *termDictionary @@ -245,7 +245,7 @@ func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { } func (e *MessageEncoder) encodeBodySize() *MessageEncoder { - e.bodySize[e.bodySizeIndex] = uint64(len(e.bits) - e.bodyLastOffset) + e.bodySize[e.bodySizeIndex] = int64(len(e.bits) - e.bodyLastOffset) e.bodyLastOffset = len(e.bits) e.bodySizeIndex++ return e @@ -263,11 +263,11 @@ func (e *MessageEncoder) encodeString(s string) *MessageEncoder { func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { oldSize := len(e.bits) - e.bodySize = e.allocator.uint64Slice(headers.count) + e.bodySize = e.allocator.int64Slice(headers.count) e.bits = headers.encode(e.bits, e.dict) e.bodyLastOffset = len(e.bits) - e.metaSizeTable = e.allocator.uint64Slice(maxMetaSizeIndex + 1) - e.metaSizeTable[headerSizeIndex] = uint64(len(e.bits) - oldSize) + e.metaSizeTable = e.allocator.int64Slice(maxMetaSizeIndex + 1) + e.metaSizeTable[headerSizeIndex] = int64(len(e.bits) - oldSize) e.sizeTables = append(e.sizeTables, e.metaSizeTable, e.bodySize) return e } @@ -276,18 +276,18 @@ func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { func (e *MessageEncoder) Encode() []byte { offset := len(e.bits) e.bits = encodeTermDictionary(e.bits, e.dict) - e.metaSizeTable[termDictionarySizeIndex] = uint64(len(e.bits) - offset) + e.metaSizeTable[termDictionarySizeIndex] = int64(len(e.bits) - offset) return encodeSizeTables(e.bits, e.sizeTables) } func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *MessageEncoder { sizeTables := e.sizeTables for _, event := range events { - columnGroupSizeTable := e.allocator.uint64Slice(len(event)) + columnGroupSizeTable := e.allocator.int64Slice(len(event)) for gi, group := range event { oldSize := len(e.bits) e.bits = group.encode(e.bits, e.dict) - columnGroupSizeTable[gi] = uint64(len(e.bits) - oldSize) + columnGroupSizeTable[gi] = int64(len(e.bits) - oldSize) } sizeTables = append(sizeTables, columnGroupSizeTable) e.encodeBodySize() diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index aa3fb24f248..45c229a60d8 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -44,8 +44,11 @@ const ( ) var ( - oneNullInt64Slice = []int64{nullInt64} - oneNullStringSlice = []*string{nil} + oneNullInt64Slice = []int64{nullInt64} + oneNullStringSlice = []*string{nil} + emptyDecodingTermDictionary = &termDictionary{ + id: make([]string, 0), + } ) type termDictionary struct { @@ -145,6 +148,9 @@ func (d *termDictionary) decodeNullableChunk(array []int64) ([]*string, error) { } func encodeTermDictionary(bits []byte, dict *termDictionary) []byte { + if len(dict.id) == 0 { + return bits + } bits = encodeUvarint(bits, uint64(len(dict.id))) bits = encodeStringChunk(bits, dict.id) return bits From 8ff5468ec015315a66a62c9c2b245d9b8e2480ba Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 21:36:58 +0800 Subject: [PATCH 37/39] Always encode terms in delta varint chunk Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/model.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 45c229a60d8..797642f46e8 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -74,15 +74,15 @@ func (d *termDictionary) encodeNullable(s *string) int64 { return int64(d.encode(*s)) } -func (d *termDictionary) encode(s string) uint64 { +func (d *termDictionary) encode(s string) int64 { id, ok := d.term[s] if !ok { id := len(d.id) d.term[s] = id d.id = append(d.id, s) - return uint64(id) + return int64(id) } - return uint64(id) + return int64(id) } func (d *termDictionary) encodeNullableChunk(array []*string) []int64 { @@ -93,15 +93,15 @@ func (d *termDictionary) encodeNullableChunk(array []*string) []int64 { return result } -func (d *termDictionary) encodeChunk(array []string) []uint64 { - result := make([]uint64, len(array)) +func (d *termDictionary) encodeChunk(array []string) []int64 { + result := make([]int64, len(array)) for idx, s := range array { result[idx] = d.encode(s) } return result } -func (d *termDictionary) decode(id uint64) (string, error) { +func (d *termDictionary) decode(id int64) (string, error) { i := int(id) if len(d.id) <= i || i < 0 { return "", cerror.ErrCraftCodecInvalidData.GenWithStack("invalid term id") @@ -116,14 +116,14 @@ func (d *termDictionary) decodeNullable(id int64) (*string, error) { if id < nullInt64 { return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("invalid term id") } - s, err := d.decode(uint64(id)) + s, err := d.decode(id) if err != nil { return nil, err } return &s, nil } -func (d *termDictionary) decodeChunk(array []uint64) ([]string, error) { +func (d *termDictionary) decodeChunk(array []int64) ([]string, error) { result := make([]string, len(array)) for idx, id := range array { t, err := d.decode(id) @@ -296,7 +296,7 @@ type columnGroup struct { func (g *columnGroup) encode(bits []byte, dict *termDictionary) []byte { bits = append(bits, g.ty) bits = encodeUvarint(bits, uint64(len(g.names))) - bits = encodeDeltaUvarintChunk(bits, dict.encodeChunk(g.names)) + bits = encodeDeltaVarintChunk(bits, dict.encodeChunk(g.names)) bits = encodeUvarintChunk(bits, g.types) bits = encodeUvarintChunk(bits, g.flags) bits = encodeNullableBytesChunk(bits, g.values) @@ -334,10 +334,10 @@ func decodeColumnGroup(bits []byte, allocator *SliceAllocator, dict *termDiction return nil, errors.Trace(err) } var names []string - var tmp []uint64 + var tmp []int64 var values [][]byte var types, flags []uint64 - bits, tmp, err = decodeDeltaUvarintChunk(bits, numColumns, allocator) + bits, tmp, err = decodeDeltaVarintChunk(bits, numColumns, allocator) if err != nil { return nil, errors.Trace(err) } From 1314185aa1eda39cee1d913c8e576c32b63b5838 Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 21:50:47 +0800 Subject: [PATCH 38/39] Change header to encode type with delta varint chunk Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 4 ++-- cdc/sink/codec/craft/model.go | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 9a56af27ca4..a36a88f7683 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -300,7 +300,7 @@ func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *Messag func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncoder { return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), - ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), + ty: allocator.oneInt64Slice(int64(model.MqMessageTypeResolved)), partition: oneNullInt64Slice, schema: oneNullStringSlice, table: oneNullStringSlice, @@ -321,7 +321,7 @@ func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageE } return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ev.CommitTs), - ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), + ty: allocator.oneInt64Slice(int64(model.MqMessageTypeDDL)), partition: oneNullInt64Slice, schema: allocator.oneNullableStringSlice(schema), table: allocator.oneNullableStringSlice(table), diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 797642f46e8..5ffe4f651df 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -171,7 +171,7 @@ func decodeTermDictionary(bits []byte, allocator *SliceAllocator) ([]byte, *term // Headers in columnar layout type Headers struct { ts []uint64 - ty []uint64 + ty []int64 partition []int64 schema []*string table []*string @@ -186,19 +186,19 @@ func (h *Headers) Count() int { func (h *Headers) encode(bits []byte, dict *termDictionary) []byte { bits = encodeDeltaUvarintChunk(bits, h.ts[:h.count]) - bits = encodeDeltaUvarintChunk(bits, h.ty[:h.count]) + bits = encodeDeltaVarintChunk(bits, h.ty[:h.count]) bits = encodeDeltaVarintChunk(bits, h.partition[:h.count]) bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.schema[:h.count])) bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.table[:h.count])) return bits } -func (h *Headers) appendHeader(allocator *SliceAllocator, ts, ty uint64, partition int64, schema, table *string) int { +func (h *Headers) appendHeader(allocator *SliceAllocator, ts uint64, ty, partition int64, schema, table *string) int { idx := h.count if idx+1 > len(h.ty) { size := newBufferSize(idx) h.ts = allocator.resizeUint64Slice(h.ts, size) - h.ty = allocator.resizeUint64Slice(h.ty, size) + h.ty = allocator.resizeInt64Slice(h.ty, size) h.partition = allocator.resizeInt64Slice(h.partition, size) h.schema = allocator.resizeNullableStringSlice(h.schema, size) h.table = allocator.resizeNullableStringSlice(h.table, size) @@ -249,14 +249,14 @@ func (h *Headers) GetTable(index int) string { } func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator, dict *termDictionary) (*Headers, error) { - var ts, ty []uint64 - var partition, tmp []int64 + var ts []uint64 + var ty, partition, tmp []int64 var schema, table []*string var err error if bits, ts, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } - if bits, ty, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { + if bits, ty, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } if bits, partition, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { @@ -471,7 +471,7 @@ func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) b.estimatedSize += b.headers.appendHeader( b.allocator, ev.CommitTs, - uint64(model.MqMessageTypeRow), + int64(model.MqMessageTypeRow), partition, schema, table, From b96c3f5a6e4a6e56717ea8b62169a1b16a5d163a Mon Sep 17 00:00:00 2001 From: Xiaoguang Sun Date: Mon, 21 Jun 2021 22:05:29 +0800 Subject: [PATCH 39/39] Use uvarint chunk to encode type of headers Signed-off-by: Xiaoguang Sun --- cdc/sink/codec/craft/encoder.go | 4 ++-- cdc/sink/codec/craft/model.go | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index a36a88f7683..9a56af27ca4 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -300,7 +300,7 @@ func (e *MessageEncoder) encodeRowChangeEvents(events []rowChangedEvent) *Messag func NewResolvedEventEncoder(allocator *SliceAllocator, ts uint64) *MessageEncoder { return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ts), - ty: allocator.oneInt64Slice(int64(model.MqMessageTypeResolved)), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), partition: oneNullInt64Slice, schema: oneNullStringSlice, table: oneNullStringSlice, @@ -321,7 +321,7 @@ func NewDDLEventEncoder(allocator *SliceAllocator, ev *model.DDLEvent) *MessageE } return NewMessageEncoder(allocator).encodeHeaders(&Headers{ ts: allocator.oneUint64Slice(ev.CommitTs), - ty: allocator.oneInt64Slice(int64(model.MqMessageTypeDDL)), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), partition: oneNullInt64Slice, schema: allocator.oneNullableStringSlice(schema), table: allocator.oneNullableStringSlice(table), diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 5ffe4f651df..e4c90449d0b 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -171,7 +171,7 @@ func decodeTermDictionary(bits []byte, allocator *SliceAllocator) ([]byte, *term // Headers in columnar layout type Headers struct { ts []uint64 - ty []int64 + ty []uint64 partition []int64 schema []*string table []*string @@ -186,19 +186,19 @@ func (h *Headers) Count() int { func (h *Headers) encode(bits []byte, dict *termDictionary) []byte { bits = encodeDeltaUvarintChunk(bits, h.ts[:h.count]) - bits = encodeDeltaVarintChunk(bits, h.ty[:h.count]) + bits = encodeUvarintChunk(bits, h.ty[:h.count]) bits = encodeDeltaVarintChunk(bits, h.partition[:h.count]) bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.schema[:h.count])) bits = encodeDeltaVarintChunk(bits, dict.encodeNullableChunk(h.table[:h.count])) return bits } -func (h *Headers) appendHeader(allocator *SliceAllocator, ts uint64, ty, partition int64, schema, table *string) int { +func (h *Headers) appendHeader(allocator *SliceAllocator, ts, ty uint64, partition int64, schema, table *string) int { idx := h.count if idx+1 > len(h.ty) { size := newBufferSize(idx) h.ts = allocator.resizeUint64Slice(h.ts, size) - h.ty = allocator.resizeInt64Slice(h.ty, size) + h.ty = allocator.resizeUint64Slice(h.ty, size) h.partition = allocator.resizeInt64Slice(h.partition, size) h.schema = allocator.resizeNullableStringSlice(h.schema, size) h.table = allocator.resizeNullableStringSlice(h.table, size) @@ -249,14 +249,14 @@ func (h *Headers) GetTable(index int) string { } func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator, dict *termDictionary) (*Headers, error) { - var ts []uint64 - var ty, partition, tmp []int64 + var ts, ty []uint64 + var partition, tmp []int64 var schema, table []*string var err error if bits, ts, err = decodeDeltaUvarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } - if bits, ty, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { + if bits, ty, err = decodeUvarintChunk(bits, numHeaders, allocator); err != nil { return nil, errors.Trace(err) } if bits, partition, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { @@ -471,7 +471,7 @@ func (b *RowChangedEventBuffer) AppendRowChangedEvent(ev *model.RowChangedEvent) b.estimatedSize += b.headers.appendHeader( b.allocator, ev.CommitTs, - int64(model.MqMessageTypeRow), + uint64(model.MqMessageTypeRow), partition, schema, table,