diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go new file mode 100644 index 00000000000..2c0af292cdc --- /dev/null +++ b/cdc/sink/codec/codec_test.go @@ -0,0 +1,558 @@ +// 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" + "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: 424316552636792833, + Table: &model.TableName{Schema: "a", Table: "b"}, + 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: "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: "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: "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: "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: "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}, + }, + }}, {}} + + codecDDLCases = [][]*model.DDLEvent{{{ + CommitTs: 424316555979653121, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table a", + Type: 1, + }}, {{ + CommitTs: 424316583965360129, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table a", + Type: 1, + }, { + CommitTs: 424316586087940097, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table b", + Type: 2, + }, { + CommitTs: 424316588736118785, + TableInfo: &model.SimpleTableInfo{ + Schema: "a", Table: "b", + }, + Query: "create table c", + Type: 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{}) + +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 { + msg, err := codecEncodeRowCase(encoder, events) + c.Assert(err, check.IsNil) + return msg +} + +func (s *codecTestSuite) TestJsonVsCraftVsPB(c *check.C) { + defer testleak.AfterTest(c)() + 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 + } + 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) + 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%%)+ |", 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{{ + 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{} + if err := key.Unmarshal(message.Key); err != nil { + panic(err) + } + value := &benchmark.RowChanged{} + if err := value.Unmarshal(message.Value); err != nil { + panic(err) + } + 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{} + if err := keys.Unmarshal(message.Key); err != nil { + panic(err) + } + values := &benchmark.RowChangedColumnar{} + if err := values.Unmarshal(message.Value); err != nil { + panic(err) + } + + 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 new file mode 100644 index 00000000000..03a4cb75cd2 --- /dev/null +++ b/cdc/sink/codec/craft.go @@ -0,0 +1,260 @@ +// 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 ( + "math" + "strconv" + + "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" +) + +// CraftEventBatchEncoder encodes the events into the byte of a batch into craft binary format. +type CraftEventBatchEncoder struct { + 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, craft.NewResolvedEventEncoder(e.allocator, ts).Encode(), ts), nil +} + +func (e *CraftEventBatchEncoder) flush() { + 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) + 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 +} + +// EncodeDDLEvent implements the EventBatchEncoder interface +func (e *CraftEventBatchEncoder) EncodeDDLEvent(ev *model.DDLEvent) (*MQMessage, error) { + 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 { + // flush buffered data to message buffer + e.flush() + } + ret := e.messageBuf + e.messageBuf = make([]*MQMessage, 0, 2) + 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 { + 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 +} + +// 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)) +} + +// NewCraftEventBatchEncoderWithAllocator creates a new CraftEventBatchEncoder with given allocator. +func NewCraftEventBatchEncoderWithAllocator(allocator *craft.SliceAllocator) EventBatchEncoder { + return &CraftEventBatchEncoder{ + 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 { + 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.headers.Count() { + return model.MqMessageTypeUnknown, false, nil + } + return b.headers.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.headers.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 row changed event message") + } + oldValue, newValue, err := b.decoder.RowChangedEvent(b.index) + if err != nil { + return nil, errors.Trace(err) + } + ev := &model.RowChangedEvent{} + if oldValue != nil { + if ev.PreColumns, err = oldValue.ToModel(); err != nil { + return nil, errors.Trace(err) + } + } + if newValue != nil { + if ev.Columns, err = newValue.ToModel(); err != nil { + return nil, errors.Trace(err) + } + } + ev.CommitTs = b.headers.GetTs(b.index) + ev.Table = &model.TableName{ + Schema: b.headers.GetSchema(b.index), + Table: b.headers.GetTable(b.index), + } + partition := b.headers.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 ddl event message") + } + ddlType, query, err := b.decoder.DDLEvent(b.index) + if err != nil { + return nil, errors.Trace(err) + } + event := &model.DDLEvent{ + CommitTs: b.headers.GetTs(b.index), + Query: query, + Type: ddlType, + TableInfo: &model.SimpleTableInfo{ + Schema: b.headers.GetSchema(b.index), + Table: b.headers.GetTable(b.index), + }, + } + b.index++ + return event, nil +} + +// NewCraftEventBatchDecoder creates a new CraftEventBatchDecoder. +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 { + return nil, errors.Trace(err) + } + headers, err := decoder.Headers() + if err != nil { + return nil, errors.Trace(err) + } + + return &CraftEventBatchDecoder{ + 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..bab76b89453 --- /dev/null +++ b/cdc/sink/codec/craft/buffer.go @@ -0,0 +1,514 @@ +// 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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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 +} + +//nolint:unused +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)} +} + +// SliceAllocator 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 +} + +// NewSliceAllocator creates a new slice allocator with given batch allocation size. +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) +} + +//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) +} + +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) +} + +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) +} + +//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) +} + +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) +} + +//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) +} + +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) +} + +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) +} + +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..38221a136bc --- /dev/null +++ b/cdc/sink/codec/craft/codec_test.go @@ -0,0 +1,106 @@ +// 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" + "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) } + +type codecSuite struct { + allocator *SliceAllocator +} + +func (s *codecSuite) TestSizeTable(c *check.C) { + defer testleak.AfterTest(c)() + tables := [][]int64{ + { + 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 new file mode 100644 index 00000000000..1e51ae2ede5 --- /dev/null +++ b/cdc/sink/codec/craft/decoder.go @@ -0,0 +1,483 @@ +// 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 ( + "encoding/binary" + "math" + "unsafe" + + "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" +) + +/// 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 { + 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) { + // 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 + 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 is greater than max int32") + } + 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 is greater than max int32") + } + 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 is greater than max int32") + } + 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.LittleEndian.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, unsafeBytesToString(bytes), nil + } + return bits, "", errors.Trace(err) +} + +// Chunk decoders +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 + } + + data := allocator.stringSlice(size) + for i := 0; i < size; i++ { + data[i] = unsafeBytesToString(newBits[:larray[i]]) + newBits = newBits[larray[i]:] + } + return newBits, data, nil +} + +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 + } + + 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, data, nil +} + +//nolint:unused,deadcode +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), allocator *SliceAllocator) ([]byte, [][]byte, error) { + larray := allocator.intSlice(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 := allocator.bytesSlice(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, allocator *SliceAllocator) ([]byte, [][]byte, error) { + return doDecodeBytesChunk(bits, size, decodeVarintLength, allocator) +} + +func decodeVarintChunk(bits []byte, size int, allocator *SliceAllocator) ([]byte, []int64, error) { + array := allocator.int64Slice(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, allocator *SliceAllocator) ([]byte, []uint64, error) { + array := allocator.uint64Slice(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, allocator *SliceAllocator) ([]byte, []int64, error) { + array := allocator.int64Slice(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, allocator *SliceAllocator) ([]byte, []uint64, error) { + array := allocator.uint64Slice(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, allocator *SliceAllocator) (int, [][]int64, error) { + nb, size, _ := decodeUvarintReversedLength(bits) + sizeOffset := len(bits) - nb + tablesOffset := sizeOffset - size + tables := bits[tablesOffset:sizeOffset] + + tableSize := size + nb + var err error + 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 = decodeDeltaVarintChunk(tables, size, allocator) + if err != nil { + return 0, nil, errors.Trace(err) + } + result = append(result, table) + } + + return tableSize, result, nil +} + +// DecodeTiDBType decodes TiDB types. +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 unsafeBytesToString(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: + // value type for these mysql types are int64 or uint64 depends on flags + if flag.IsUnsigned() { + _, u64, err := decodeUvarint(bits) + return u64, err + } + _, i64, err := decodeVarint(bits) + return i64, err + case mysql.TypeYear: + _, i64, err := decodeVarint(bits) + return i64, err + case mysql.TypeUnspecified: + fallthrough + case mysql.TypeNull: + fallthrough + case mysql.TypeGeometry: + return nil, nil + } + return nil, nil +} + +// MessageDecoder decoder +type MessageDecoder struct { + bits []byte + sizeTables [][]int64 + metaSizeTable []int64 + bodyOffsetTable []int + allocator *SliceAllocator + dict *termDictionary +} + +// 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 { + return nil, errors.Trace(err) + } + if version < Version1 { + return nil, cerror.ErrCraftCodecInvalidData.GenWithStack("unexpected craft version") + } + sizeTablesSize, sizeTables, err := decodeSizeTables(bits, allocator) + 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] + + var dict *termDictionary + termDictionaryOffset := int(metaSizeTable[headerSizeIndex]) + start + 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], + sizeTables: sizeTables, + metaSizeTable: metaSizeTable, + bodyOffsetTable: bodyOffsetTable, + allocator: allocator, + dict: dict, + }, nil +} + +// Headers decode headers of message +func (d *MessageDecoder) Headers() (*Headers, error) { + var pairs, headersSize int + var err error + // 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.metaSizeTable[headerSizeIndex]) + var headers *Headers + headers, err = decodeHeaders(d.bits[:headersSize], pairs, d.allocator, d.dict) + if err != nil { + return nil, errors.Trace(err) + } + // skip headers + d.bits = d.bits[headersSize:] + return headers, nil +} + +func (d *MessageDecoder) bodyBits(index int) []byte { + return d.bits[d.bodyOffsetTable[index]:d.bodyOffsetTable[index+1]] +} + +// 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 { + return pmodel.ActionNone, "", errors.Trace(err) + } + _, query, err := decodeString(bits) + 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] + columnGroupIndex := 0 + for len(bits) > 0 { + columnGroupSize := columnGroupSizeTable[columnGroupIndex] + columnGroup, err := decodeColumnGroup(bits[:columnGroupSize], d.allocator, d.dict) + bits = bits[columnGroupSize:] + columnGroupIndex++ + if err != nil { + return nil, nil, errors.Trace(err) + } + switch columnGroup.ty { + case columnGroupTypeOld: + preColumns = columnGroup + case columnGroupTypeNew: + columns = columnGroup + } + } + return preColumns, columns, nil +} diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go new file mode 100644 index 00000000000..9a56af27ca4 --- /dev/null +++ b/cdc/sink/codec/craft/encoder.go @@ -0,0 +1,330 @@ +// 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 ( + "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 { + 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 { + udata := uint64(data) << 1 + if data < 0 { + udata = ^udata + } + return encodeUvarint(bits, udata) +} + +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 + } + return append(bits, byte(data)) +} + +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 { + 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 +} + +//nolint:unused,deadcode +func encodeBytes(bits []byte, data []byte) []byte { + l := len(data) + bits = encodeUvarint(bits, uint64(l)) + return append(bits, data...) +} + +func encodeString(bits []byte, data string) []byte { + l := len(data) + bits = encodeUvarint(bits, uint64(l)) + return append(bits, 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, 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, *s...) + } + } + return bits +} + +//nolint:unused,deadcode +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 [][]int64) []byte { + size := len(bits) + for _, table := range tables { + bits = encodeUvarint(bits, uint64(len(table))) + bits = encodeDeltaVarintChunk(bits, table) + } + bits, _ = encodeUvarintReversed(bits, uint64(len(bits)-size)) + return bits +} + +// EncodeTiDBType encodes TiDB types +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 unsafeStringToBytes(value.(string)) + case mysql.TypeEnum, mysql.TypeSet, mysql.TypeBit: + // 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: + // 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(allocator.byteSlice(8)[:0], value.(float64)) + 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)) + } + return encodeVarint(allocator.byteSlice(binary.MaxVarintLen64)[:0], value.(int64)) + case mysql.TypeUnspecified: + fallthrough + case mysql.TypeNull: + fallthrough + case mysql.TypeGeometry: + return nil + } + return nil +} + +// MessageEncoder is encoder for message +type MessageEncoder struct { + bits []byte + sizeTables [][]int64 + bodyLastOffset int + bodySize []int64 + bodySizeIndex int + metaSizeTable []int64 + + allocator *SliceAllocator + dict *termDictionary +} + +// NewMessageEncoder creates a new encoder with given allocator +func NewMessageEncoder(allocator *SliceAllocator) *MessageEncoder { + return &MessageEncoder{ + bits: encodeUvarint(make([]byte, 0, DefaultBufferCapacity), Version1), + allocator: allocator, + dict: newEncodingTermDictionary(), + } +} + +func (e *MessageEncoder) encodeBodySize() *MessageEncoder { + e.bodySize[e.bodySizeIndex] = int64(len(e.bits) - e.bodyLastOffset) + e.bodyLastOffset = len(e.bits) + e.bodySizeIndex++ + return e +} + +func (e *MessageEncoder) encodeUvarint(u64 uint64) *MessageEncoder { + e.bits = encodeUvarint(e.bits, u64) + return e +} + +func (e *MessageEncoder) encodeString(s string) *MessageEncoder { + e.bits = encodeString(e.bits, s) + return e +} + +func (e *MessageEncoder) encodeHeaders(headers *Headers) *MessageEncoder { + oldSize := len(e.bits) + e.bodySize = e.allocator.int64Slice(headers.count) + e.bits = headers.encode(e.bits, e.dict) + e.bodyLastOffset = len(e.bits) + 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 +} + +// Encode message into bits +func (e *MessageEncoder) Encode() []byte { + offset := len(e.bits) + e.bits = encodeTermDictionary(e.bits, e.dict) + 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.int64Slice(len(event)) + for gi, group := range event { + oldSize := len(e.bits) + e.bits = group.encode(e.bits, e.dict) + columnGroupSizeTable[gi] = int64(len(e.bits) - oldSize) + } + sizeTables = append(sizeTables, columnGroupSizeTable) + e.encodeBodySize() + } + e.sizeTables = sizeTables + return e +} + +// 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), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeResolved)), + partition: oneNullInt64Slice, + schema: oneNullStringSlice, + table: oneNullStringSlice, + count: 1, + }).encodeBodySize() +} + +// 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 + var schema, table *string + if len(ev.TableInfo.Schema) > 0 { + schema = &ev.TableInfo.Schema + } + if len(ev.TableInfo.Table) > 0 { + table = &ev.TableInfo.Table + } + return NewMessageEncoder(allocator).encodeHeaders(&Headers{ + ts: allocator.oneUint64Slice(ev.CommitTs), + ty: allocator.oneUint64Slice(uint64(model.MqMessageTypeDDL)), + partition: oneNullInt64Slice, + schema: allocator.oneNullableStringSlice(schema), + table: allocator.oneNullableStringSlice(table), + count: 1, + }).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..e4c90449d0b --- /dev/null +++ b/cdc/sink/codec/craft/model.go @@ -0,0 +1,504 @@ +// 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" + cerror "github.com/pingcap/ticdc/pkg/errors" +) + +const ( + // Version1 represents the version of craft format + Version1 uint64 = 1 + + // DefaultBufferCapacity is default buffer size + DefaultBufferCapacity = 1024 + + // Column group types + columnGroupTypeOld = 0x2 + columnGroupTypeNew = 0x1 + + // Size tables index + 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} + emptyDecodingTermDictionary = &termDictionary{ + id: make([]string, 0), + } +) + +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) int64 { + id, ok := d.term[s] + if !ok { + id := len(d.id) + d.term[s] = id + d.id = append(d.id, s) + return int64(id) + } + return int64(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) []int64 { + result := make([]int64, len(array)) + for idx, s := range array { + result[idx] = d.encode(s) + } + return result +} + +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") + } + 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") + } + s, err := d.decode(id) + if err != nil { + return nil, err + } + return &s, nil +} + +func (d *termDictionary) decodeChunk(array []int64) ([]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 { + if len(dict.id) == 0 { + return bits + } + 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 + ty []uint64 + partition []int64 + schema []*string + table []*string + + count int +} + +// Count returns number of headers +func (h *Headers) Count() int { + return h.count +} + +func (h *Headers) encode(bits []byte, dict *termDictionary) []byte { + bits = encodeDeltaUvarintChunk(bits, h.ts[: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, 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.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.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 +} + +// 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] + } + 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] + } + return "" +} + +func decodeHeaders(bits []byte, numHeaders int, allocator *SliceAllocator, dict *termDictionary) (*Headers, error) { + 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 = decodeUvarintChunk(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, tmp, err = decodeDeltaVarintChunk(bits, numHeaders, allocator); err != nil { + return nil, errors.Trace(err) + } + 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{ + ts: ts, + ty: ty, + 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, dict *termDictionary) []byte { + bits = append(bits, g.ty) + bits = encodeUvarint(bits, uint64(len(g.names))) + bits = encodeDeltaVarintChunk(bits, dict.encodeChunk(g.names)) + bits = encodeUvarintChunk(bits, g.types) + bits = encodeUvarintChunk(bits, g.flags) + bits = encodeNullableBytesChunk(bits, g.values) + 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 { + 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, dict *termDictionary) (*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 tmp []int64 + var values [][]byte + var types, flags []uint64 + bits, tmp, err = decodeDeltaVarintChunk(bits, numColumns, allocator) + if err != nil { + return nil, errors.Trace(err) + } + names, err = dict.decodeChunk(tmp) + 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) + if l == 0 { + return 0, nil + } + 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 + idx := 0 + if size, group := newColumnGroup(allocator, columnGroupTypeNew, ev.Columns); group != nil { + groups[idx] = group + idx++ + estimatedSize += size + } + if size, group := newColumnGroup(allocator, columnGroupTypeOld, ev.PreColumns); group != nil { + groups[idx] = group + estimatedSize += size + } + return estimatedSize, rowChangedEvent(groups) +} + +// RowChangedEventBuffer is a buffer to save row changed events in batch +type RowChangedEventBuffer struct { + headers *Headers + + events []rowChangedEvent + eventsCount int + estimatedSize int + + allocator *SliceAllocator +} + +// NewRowChangedEventBuffer creates new row changed event buffer with given allocator +func NewRowChangedEventBuffer(allocator *SliceAllocator) *RowChangedEventBuffer { + return &RowChangedEventBuffer{ + headers: &Headers{}, + allocator: allocator, + } +} + +// 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 { + 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), + 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 +} + +// 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 +} diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go new file mode 100644 index 00000000000..8c6ec0fc936 --- /dev/null +++ b/cdc/sink/codec/craft_test.go @@ -0,0 +1,250 @@ +// 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" + "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, DefaultMaxBatchSize) + 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, DefaultMaxBatchSize) + 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) +} diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 1e1fafe24ae..883f521ac1e 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 ab9bdf5c51d..1eac1a015d7 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -112,6 +112,32 @@ 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 { + var err error + if c.Flag.IsUnsigned() { + c.Value, err = strconv.ParseUint(s.String(), 10, 64) + } else { + 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)) + } + } 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 { intNum, err := s.Int64() @@ -535,27 +561,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 74ca5bbde5e..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" @@ -33,59 +34,33 @@ 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, }) +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 @@ -98,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++ } diff --git a/errors.toml b/errors.toml index 222c9f4c2d2..68b8cfb5c94 100755 --- a/errors.toml +++ b/errors.toml @@ -146,6 +146,11 @@ error = ''' codec decode error ''' +["CDC:ErrCraftCodecInvalidData"] +error = ''' +craft codec invalid data +''' + ["CDC:ErrCreateMarkTableFailed"] error = ''' create mark table failed @@ -716,6 +721,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 diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index 5a4858eba8d..9a12e29a110 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -107,6 +107,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")) 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