From 3ad654160e7d659546042e7c7748263ed0f521d8 Mon Sep 17 00:00:00 2001 From: oscar-zyh Date: Tue, 16 Jun 2020 21:56:28 +0800 Subject: [PATCH] support canal protocol for mq sink (#649) --- cdc/sink/codec/canal.go | 362 +++ cdc/sink/codec/canal_test.go | 254 ++ cdc/sink/codec/interface.go | 43 +- cdc/sink/codec/java.go | 152 ++ cdc/sink/mq.go | 16 +- cmd/cmd_test.go | 8 + go.mod | 2 + pkg/config/config.go | 4 +- pkg/config/sink.go | 1 + proto/CanalProtocol.proto | 170 ++ proto/EntryProtocol.proto | 246 ++ proto/canal/CanalProtocol.pb.go | 4352 +++++++++++++++++++++++++++++++ proto/canal/EntryProtocol.pb.go | 3927 ++++++++++++++++++++++++++++ proto/generate-proto.sh | 8 + scripts/check-copyright.sh | 2 +- 15 files changed, 9543 insertions(+), 4 deletions(-) create mode 100644 cdc/sink/codec/canal.go create mode 100644 cdc/sink/codec/canal_test.go create mode 100644 cdc/sink/codec/java.go create mode 100644 proto/CanalProtocol.proto create mode 100644 proto/EntryProtocol.proto create mode 100644 proto/canal/CanalProtocol.pb.go create mode 100644 proto/canal/EntryProtocol.pb.go create mode 100755 proto/generate-proto.sh diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go new file mode 100644 index 00000000000..8ee7ae5c86e --- /dev/null +++ b/cdc/sink/codec/canal.go @@ -0,0 +1,362 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "fmt" + "strconv" + + "github.com/golang/protobuf/proto" + "github.com/pingcap/errors" + mm "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" + canal "github.com/pingcap/ticdc/proto/canal" + "golang.org/x/text/encoding" + "golang.org/x/text/encoding/charmap" +) + +// compatible with canal-1.1.4 +// https://github.com/alibaba/canal/tree/canal-1.1.4 +const ( + CanalPacketVersion int32 = 1 + CanalProtocolVersion int32 = 1 + CanalServerEncode string = "UTF-8" +) + +// convert ts in tidb to timestamp(in ms) in canal +func convertToCanalTs(commitTs uint64) int64 { + return int64(commitTs >> 18) +} + +// get the canal EventType according to the RowChangedEvent +func convertRowEventType(e *model.RowChangedEvent) canal.EventType { + if e.Delete { + return canal.EventType_DELETE + } + return canal.EventType_UPDATE +} + +// get the canal EventType according to the DDLEvent +func convertDdlEventType(e *model.DDLEvent) canal.EventType { + // see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/ddl/DruidDdlParser.java + switch e.Type { + case mm.ActionCreateSchema, mm.ActionDropSchema, mm.ActionShardRowID, mm.ActionCreateView, + mm.ActionDropView, mm.ActionRecoverTable, mm.ActionModifySchemaCharsetAndCollate, + mm.ActionLockTable, mm.ActionUnlockTable, mm.ActionRepairTable, mm.ActionSetTiFlashReplica, + mm.ActionUpdateTiFlashReplicaStatus, mm.ActionCreateSequence, mm.ActionAlterSequence, + mm.ActionDropSequence, mm.ActionModifyTableAutoIdCache, mm.ActionRebaseAutoRandomBase: + return canal.EventType_QUERY + case mm.ActionCreateTable: + return canal.EventType_CREATE + case mm.ActionRenameTable: + return canal.EventType_RENAME + case mm.ActionAddIndex, mm.ActionAddForeignKey, mm.ActionAddPrimaryKey: + return canal.EventType_CINDEX + case mm.ActionDropIndex, mm.ActionDropForeignKey, mm.ActionDropPrimaryKey: + return canal.EventType_DINDEX + case mm.ActionAddColumn, mm.ActionDropColumn, mm.ActionModifyColumn, mm.ActionRebaseAutoID, + mm.ActionSetDefaultValue, mm.ActionModifyTableComment, mm.ActionRenameIndex, mm.ActionAddTablePartition, + mm.ActionDropTablePartition, mm.ActionModifyTableCharsetAndCollate, mm.ActionTruncateTablePartition, + mm.ActionAddColumns, mm.ActionDropColumns: + return canal.EventType_ALTER + case mm.ActionDropTable: + return canal.EventType_ERASE + case mm.ActionTruncateTable: + return canal.EventType_TRUNCATE + default: + return canal.EventType_QUERY + } +} + +func isCanalDdl(t canal.EventType) bool { + // EventType_QUERY is not a ddl type in canal, but in cdc it is. + // see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/ddl/DruidDdlParser.java + // & https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L278 + switch t { + case canal.EventType_CREATE, + canal.EventType_RENAME, + canal.EventType_CINDEX, + canal.EventType_DINDEX, + canal.EventType_ALTER, + canal.EventType_ERASE, + canal.EventType_TRUNCATE: + return true + } + return false +} + +type canalEntryBuilder struct { + bytesDecoder *encoding.Decoder // default charset is ISO-8859-1 +} + +// build the header of a canal entry +func (b *canalEntryBuilder) buildHeader(commitTs uint64, schema string, table string, eventType canal.EventType, rowCount int) *canal.Header { + t := convertToCanalTs(commitTs) + h := &canal.Header{ + VersionPresent: &canal.Header_Version{Version: CanalProtocolVersion}, + ServerenCode: CanalServerEncode, + ExecuteTime: t, + SourceTypePresent: &canal.Header_SourceType{SourceType: canal.Type_MYSQL}, + SchemaName: schema, + TableName: table, + EventTypePresent: &canal.Header_EventType{EventType: eventType}, + } + if rowCount > 0 { + p := &canal.Pair{ + Key: "rowsCount", + Value: strconv.Itoa(rowCount), + } + h.Props = append(h.Props, p) + } + return h +} + +// build the Column in the canal RowData +func (b *canalEntryBuilder) buildColumn(c *model.Column, colName string, updated bool) (*canal.Column, error) { + sqlType := MysqlToJavaType(c.Type) + // Some special cases handled in canal + // see https://github.com/alibaba/canal/blob/d53bfd7ee76f8fe6eb581049d64b07d4fcdd692d/parse/src/main/java/com/alibaba/otter/canal/parse/inbound/mysql/dbsync/LogEventConvert.java#L733 + switch c.Type { + // Since we cannot get the signed/unsigned flag of the column in the RowChangedEvent currently, + // we promote the sqlTypes regardless of the flag. + case mysql.TypeTiny: + sqlType = JavaSQLTypeSMALLINT + case mysql.TypeShort: + sqlType = JavaSQLTypeINTEGER + case mysql.TypeInt24: + sqlType = JavaSQLTypeINTEGER + case mysql.TypeLong: + sqlType = JavaSQLTypeBIGINT + case mysql.TypeLonglong: + sqlType = JavaSQLTypeDECIMAL + } + switch sqlType { + case JavaSQLTypeBINARY, JavaSQLTypeVARBINARY, JavaSQLTypeLONGVARBINARY: + sqlType = JavaSQLTypeBLOB + } + + isKey := c.WhereHandle != nil && *c.WhereHandle + isNull := c.Value == nil + value := "" + if !isNull { + switch v := c.Value.(type) { + case int64: + value = strconv.FormatInt(v, 10) + case uint64: + value = strconv.FormatUint(v, 10) + case float32: + value = strconv.FormatFloat(float64(v), 'f', -1, 32) + case float64: + value = strconv.FormatFloat(v, 'f', -1, 64) + case string: + value = v + case []byte: + decoded, err := b.bytesDecoder.Bytes(v) + if err != nil { + return nil, errors.Trace(err) + } + value = string(decoded) + sqlType = JavaSQLTypeBLOB // change sql type to Blob when the type is []byte according to canal + default: + value = fmt.Sprintf("%v", v) + } + } + + canalColumn := &canal.Column{ + SqlType: int32(sqlType), + Name: colName, + IsKey: isKey, + Updated: updated, + IsNullPresent: &canal.Column_IsNull{IsNull: isNull}, + Value: value, + } + return canalColumn, nil +} + +// build the RowData of a canal entry +func (b *canalEntryBuilder) buildRowData(e *model.RowChangedEvent) (*canal.RowData, error) { + var columns []*canal.Column + for name, column := range e.Columns { + c, err := b.buildColumn(column, name, !e.Delete) + if err != nil { + return nil, errors.Trace(err) + } + columns = append(columns, c) + } + + rowData := &canal.RowData{} + if e.Delete { + rowData.BeforeColumns = columns + } else { + rowData.AfterColumns = columns + } + return rowData, nil +} + +// FromRowEvent builds canal entry from cdc RowChangedEvent +func (b *canalEntryBuilder) FromRowEvent(e *model.RowChangedEvent) (*canal.Entry, error) { + eventType := convertRowEventType(e) + header := b.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1) + isDdl := isCanalDdl(eventType) // false + rowData, err := b.buildRowData(e) + if err != nil { + return nil, errors.Trace(err) + } + rc := &canal.RowChange{ + EventTypePresent: &canal.RowChange_EventType{EventType: eventType}, + IsDdlPresent: &canal.RowChange_IsDdl{IsDdl: isDdl}, + RowDatas: []*canal.RowData{rowData}, + } + rcBytes, err := proto.Marshal(rc) + if err != nil { + return nil, errors.Trace(err) + } + + // build entry + entry := &canal.Entry{ + Header: header, + EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA}, + StoreValue: rcBytes, + } + return entry, nil +} + +// FromDdlEvent builds canal entry from cdc DDLEvent +func (b *canalEntryBuilder) FromDdlEvent(e *model.DDLEvent) (*canal.Entry, error) { + eventType := convertDdlEventType(e) + header := b.buildHeader(e.CommitTs, e.Schema, e.Table, eventType, -1) + isDdl := isCanalDdl(eventType) + rc := &canal.RowChange{ + EventTypePresent: &canal.RowChange_EventType{EventType: eventType}, + IsDdlPresent: &canal.RowChange_IsDdl{IsDdl: isDdl}, + Sql: e.Query, + RowDatas: nil, + DdlSchemaName: e.Schema, + } + rcBytes, err := proto.Marshal(rc) + if err != nil { + return nil, errors.Trace(err) + } + + // build entry + entry := &canal.Entry{ + Header: header, + EntryTypePresent: &canal.Entry_EntryType{EntryType: canal.EntryType_ROWDATA}, + StoreValue: rcBytes, + } + return entry, nil +} + +// NewCanalEntryBuilder creates a new canalEntryBuilder +func NewCanalEntryBuilder() *canalEntryBuilder { + d := charmap.ISO8859_1.NewDecoder() + return &canalEntryBuilder{ + bytesDecoder: d, + } +} + +// CanalEventBatchEncoder encodes the events into the byte of a batch into. +type CanalEventBatchEncoder struct { + messages *canal.Messages + packet *canal.Packet + entryBuilder *canalEntryBuilder +} + +// AppendResolvedEvent implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) AppendResolvedEvent(ts uint64) error { + // For canal now, there is no such a corresponding type to ResolvedEvent so far. + // Therefore the event is ignored. + return nil +} + +// AppendRowChangedEvent implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) error { + entry, err := d.entryBuilder.FromRowEvent(e) + if err != nil { + return errors.Trace(err) + } + b, err := proto.Marshal(entry) + if err != nil { + return errors.Trace(err) + } + d.messages.Messages = append(d.messages.Messages, b) + return nil +} + +// AppendDDLEvent implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) AppendDDLEvent(e *model.DDLEvent) error { + entry, err := d.entryBuilder.FromDdlEvent(e) + if err != nil { + return errors.Trace(err) + } + b, err := proto.Marshal(entry) + if err != nil { + return errors.Trace(err) + } + d.messages.Messages = append(d.messages.Messages, b) + return nil +} + +// Build implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) Build() (key []byte, value []byte) { + err := d.refreshPacketBody() + if err != nil { + panic(err) + } + value, err = proto.Marshal(d.packet) + if err != nil { + panic(err) + } + return nil, value +} + +// Size implements the EventBatchEncoder interface +func (d *CanalEventBatchEncoder) Size() int { + // TODO: avoid marshaling the messages every time for calculating the size of the packet + err := d.refreshPacketBody() + if err != nil { + panic(err) + } + return proto.Size(d.packet) +} + +// refreshPacketBody() marshals the messages to the packet body +func (d *CanalEventBatchEncoder) refreshPacketBody() error { + oldSize := len(d.packet.Body) + newSize := proto.Size(d.messages) + if newSize > oldSize { + // resize packet body slice + d.packet.Body = append(d.packet.Body, make([]byte, newSize-oldSize)...) + } + _, err := d.messages.MarshalToSizedBuffer(d.packet.Body[:newSize]) + return err +} + +// NewCanalEventBatchEncoder creates a new CanalEventBatchEncoder. +func NewCanalEventBatchEncoder() EventBatchEncoder { + p := &canal.Packet{ + VersionPresent: &canal.Packet_Version{ + Version: CanalPacketVersion, + }, + Type: canal.PacketType_MESSAGES, + } + + encoder := &CanalEventBatchEncoder{ + messages: &canal.Messages{}, + packet: p, + entryBuilder: NewCanalEntryBuilder(), + } + return encoder +} diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go new file mode 100644 index 00000000000..1386d127293 --- /dev/null +++ b/cdc/sink/codec/canal_test.go @@ -0,0 +1,254 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import ( + "github.com/golang/protobuf/proto" + "github.com/pingcap/check" + mm "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/ticdc/cdc/model" + canal "github.com/pingcap/ticdc/proto/canal" + "golang.org/x/text/encoding/charmap" +) + +type canalBatchSuite struct { + rowCases [][]*model.RowChangedEvent + ddlCases [][]*model.DDLEvent +} + +var _ = check.Suite(&canalBatchSuite{ + rowCases: [][]*model.RowChangedEvent{{{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + }}, {{ + CommitTs: 1, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: map[string]*model.Column{"col1": {Type: 1, Value: "aa"}}, + }, { + CommitTs: 2, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + }, { + CommitTs: 3, + Table: &model.TableName{Schema: "a", Table: "b"}, + Columns: map[string]*model.Column{"col1": {Type: 1, Value: "bb"}}, + }, { + CommitTs: 4, + Table: &model.TableName{Schema: "a", Table: "c", Partition: 6}, + Columns: map[string]*model.Column{"col1": {Type: 1, Value: "cc"}}, + }}, {}}, + ddlCases: [][]*model.DDLEvent{{{ + CommitTs: 1, + Schema: "a", + Table: "b", + Query: "create table a", + Type: 1, + }}, {{ + CommitTs: 1, + Schema: "a", + Table: "b", + Query: "create table a", + Type: 1, + }, { + CommitTs: 2, + Schema: "a", + Table: "b", + Query: "create table b", + Type: 2, + }, { + CommitTs: 3, + Schema: "a", + Table: "b", + Query: "create table c", + Type: 3, + }}, {}}, +}) + +func (s *canalBatchSuite) TestCanalEventBatchEncoder(c *check.C) { + for _, cs := range s.rowCases { + encoder := NewCanalEventBatchEncoder() + for _, row := range cs { + err := encoder.AppendRowChangedEvent(row) + c.Assert(err, check.IsNil) + } + key, value := encoder.Build() + c.Assert(key, check.IsNil) + c.Assert(len(value), check.Equals, encoder.Size()) + + packet := &canal.Packet{} + err := proto.Unmarshal(value, packet) + c.Assert(err, check.IsNil) + c.Assert(packet.GetType(), check.Equals, canal.PacketType_MESSAGES) + messages := &canal.Messages{} + err = proto.Unmarshal(packet.GetBody(), messages) + c.Assert(err, check.IsNil) + c.Assert(len(messages.GetMessages()), check.Equals, len(cs)) + } + + for _, cs := range s.ddlCases { + encoder := NewCanalEventBatchEncoder() + for _, ddl := range cs { + err := encoder.AppendDDLEvent(ddl) + c.Assert(err, check.IsNil) + } + key, value := encoder.Build() + c.Assert(key, check.IsNil) + c.Assert(len(value), check.Equals, encoder.Size()) + + packet := &canal.Packet{} + err := proto.Unmarshal(value, packet) + c.Assert(err, check.IsNil) + c.Assert(packet.GetType(), check.Equals, canal.PacketType_MESSAGES) + messages := &canal.Messages{} + err = proto.Unmarshal(packet.GetBody(), messages) + c.Assert(err, check.IsNil) + c.Assert(len(messages.GetMessages()), check.Equals, len(cs)) + } +} + +type canalEntrySuite struct{} + +var _ = check.Suite(canalEntrySuite{}) + +func (s *canalEntrySuite) TestConvertEntry(c *check.C) { + trueVar := true + testCaseUpdate := &model.RowChangedEvent{ + CommitTs: 417318403368288260, + Table: &model.TableName{ + Schema: "cdc", + Table: "person", + }, + Delete: false, + Columns: map[string]*model.Column{ + "id": {Type: mysql.TypeLong, WhereHandle: &trueVar, Value: 1}, + "name": {Type: mysql.TypeVarchar, Value: "Bob"}, + "tiny": {Type: mysql.TypeTiny, Value: 255}, + "comment": {Type: mysql.TypeBlob, Value: []byte("测试")}, + }, + } + testCaseDelete := &model.RowChangedEvent{ + CommitTs: 417318403368288260, + Table: &model.TableName{ + Schema: "cdc", + Table: "person", + }, + Delete: true, + Columns: map[string]*model.Column{ + "id": {Type: mysql.TypeLong, WhereHandle: &trueVar, Value: 1}, + }, + } + testCaseDdl := &model.DDLEvent{ + CommitTs: 417318403368288260, + Schema: "cdc", + Table: "person", + Query: "create table person(id int, name varchar(32), tiny tinyint unsigned, comment text, primary key(id))", + Type: mm.ActionCreateTable, + } + builder := NewCanalEntryBuilder() + + // test update + entry, err := builder.FromRowEvent(testCaseUpdate) + c.Assert(err, check.IsNil) + c.Assert(entry.GetEntryType(), check.Equals, canal.EntryType_ROWDATA) + header := entry.GetHeader() + c.Assert(header.GetExecuteTime(), check.Equals, 1591943372224) + c.Assert(header.GetSourceType(), check.Equals, canal.Type_MYSQL) + c.Assert(header.GetSchemaName(), check.Equals, testCaseUpdate.Table.Schema) + c.Assert(header.GetTableName(), check.Equals, testCaseUpdate.Table.Table) + c.Assert(header.GetEventType(), check.Equals, canal.EventType_UPDATE) + store := entry.GetStoreValue() + c.Assert(store, check.NotNil) + rc := &canal.RowChange{} + err = proto.Unmarshal(store, rc) + c.Assert(err, check.IsNil) + c.Assert(rc.GetIsDdl(), check.IsFalse) + rowDatas := rc.GetRowDatas() + c.Assert(len(rowDatas), check.Equals, 1) + columns := rowDatas[0].AfterColumns + c.Assert(len(columns), check.Equals, len(testCaseUpdate.Columns)) + for _, col := range columns { + c.Assert(col.GetUpdated(), check.IsTrue) + switch col.GetName() { + case "id": + c.Assert(col.GetSqlType(), check.Equals, JavaSQLTypeBIGINT) + c.Assert(col.GetIsKey(), check.IsTrue) + c.Assert(col.GetIsNull(), check.IsFalse) + c.Assert(col.GetValue(), check.Equals, "1") + case "name": + c.Assert(col.GetSqlType(), check.Equals, JavaSQLTypeVARCHAR) + c.Assert(col.GetIsKey(), check.IsFalse) + c.Assert(col.GetIsNull(), check.IsFalse) + c.Assert(col.GetValue(), check.Equals, "Bob") + case "tiny": + c.Assert(col.GetSqlType(), check.Equals, JavaSQLTypeSMALLINT) + c.Assert(col.GetIsKey(), check.IsFalse) + c.Assert(col.GetIsNull(), check.IsFalse) + c.Assert(col.GetValue(), check.Equals, "255") + case "comment": + c.Assert(col.GetSqlType(), check.Equals, JavaSQLTypeBLOB) + c.Assert(col.GetIsKey(), check.IsFalse) + c.Assert(col.GetIsNull(), check.IsFalse) + s, err := charmap.ISO8859_1.NewEncoder().String(col.GetValue()) + c.Assert(err, check.IsNil) + c.Assert(s, check.Equals, "测试") + } + } + + // test delete + entry, err = builder.FromRowEvent(testCaseDelete) + c.Assert(err, check.IsNil) + c.Assert(entry.GetEntryType(), check.Equals, canal.EntryType_ROWDATA) + header = entry.GetHeader() + c.Assert(header.GetSchemaName(), check.Equals, testCaseDelete.Table.Schema) + c.Assert(header.GetTableName(), check.Equals, testCaseDelete.Table.Table) + c.Assert(header.GetEventType(), check.Equals, canal.EventType_DELETE) + store = entry.GetStoreValue() + c.Assert(store, check.NotNil) + rc = &canal.RowChange{} + err = proto.Unmarshal(store, rc) + c.Assert(err, check.IsNil) + c.Assert(rc.GetIsDdl(), check.IsFalse) + rowDatas = rc.GetRowDatas() + c.Assert(len(rowDatas), check.Equals, 1) + columns = rowDatas[0].BeforeColumns + c.Assert(len(columns), check.Equals, len(testCaseDelete.Columns)) + for _, col := range columns { + c.Assert(col.GetUpdated(), check.IsFalse) + switch col.GetName() { + case "id": + c.Assert(col.GetSqlType(), check.Equals, JavaSQLTypeBIGINT) + c.Assert(col.GetIsKey(), check.IsTrue) + c.Assert(col.GetIsNull(), check.IsFalse) + c.Assert(col.GetValue(), check.Equals, "1") + } + } + + // test ddl + entry, err = builder.FromDdlEvent(testCaseDdl) + c.Assert(err, check.IsNil) + c.Assert(entry.GetEntryType(), check.Equals, canal.EntryType_ROWDATA) + header = entry.GetHeader() + c.Assert(header.GetSchemaName(), check.Equals, testCaseDdl.Schema) + c.Assert(header.GetTableName(), check.Equals, testCaseDdl.Table) + c.Assert(header.GetEventType(), check.Equals, canal.EventType_CREATE) + store = entry.GetStoreValue() + c.Assert(store, check.NotNil) + rc = &canal.RowChange{} + err = proto.Unmarshal(store, rc) + c.Assert(err, check.IsNil) + c.Assert(rc.GetIsDdl(), check.IsTrue) + c.Assert(rc.GetDdlSchemaName(), check.Equals, testCaseDdl.Schema) +} diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 166edb0c2b6..caac093ee7e 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -13,7 +13,13 @@ package codec -import "github.com/pingcap/ticdc/cdc/model" +import ( + "strings" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/cdc/model" + "go.uber.org/zap" +) // EventBatchEncoder is an abstraction for events encoder type EventBatchEncoder interface { @@ -44,3 +50,38 @@ type EventBatchDecoder interface { // NextDDLEvent returns the next DDL event if exists NextDDLEvent() (*model.DDLEvent, error) } + +// Protocol is the protocol of the mq message +type Protocol int + +// Enum types of the Protocol +const ( + ProtocolDefault Protocol = iota + ProtocolCanal +) + +// FromString converts the protocol from string to Protocol enum type +func (p *Protocol) FromString(protocol string) { + switch strings.ToLower(protocol) { + case "default": + *p = ProtocolDefault + case "canal": + *p = ProtocolCanal + default: + *p = ProtocolDefault + log.Warn("can't support codec protocol, using default protocol", zap.String("protocol", protocol)) + } +} + +// NewEventBatchEncoder returns a function of creating an EventBatchEncoder +func NewEventBatchEncoder(p Protocol) func() EventBatchEncoder { + switch p { + case ProtocolDefault: + return NewJSONEventBatchEncoder + case ProtocolCanal: + return NewCanalEventBatchEncoder + default: + log.Warn("unknown codec protocol value of EventBatchEncoder", zap.Int("protocol_value", int(p))) + return NewJSONEventBatchEncoder + } +} diff --git a/cdc/sink/codec/java.go b/cdc/sink/codec/java.go new file mode 100644 index 00000000000..e7adc13adbb --- /dev/null +++ b/cdc/sink/codec/java.go @@ -0,0 +1,152 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License") +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.orglicensesLICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package codec + +import "github.com/pingcap/parser/mysql" + +// JavaSQLType is the sql type defined in class java.sql.Types in java sdk +type JavaSQLType int32 + +// jdk 1.8 +const ( + JavaSQLTypeBIT JavaSQLType = -7 + JavaSQLTypeTINYINT JavaSQLType = -6 + JavaSQLTypeSMALLINT JavaSQLType = 5 + JavaSQLTypeINTEGER JavaSQLType = 4 + JavaSQLTypeBIGINT JavaSQLType = -5 + JavaSQLTypeREAL JavaSQLType = 7 + JavaSQLTypeDOUBLE JavaSQLType = 8 + JavaSQLTypeDECIMAL JavaSQLType = 3 + JavaSQLTypeCHAR JavaSQLType = 1 + JavaSQLTypeVARCHAR JavaSQLType = 12 + JavaSQLTypeLONGVARCHAR JavaSQLType = -1 + JavaSQLTypeDATE JavaSQLType = 91 + JavaSQLTypeTIME JavaSQLType = 92 + JavaSQLTypeTIMESTAMP JavaSQLType = 93 + JavaSQLTypeBINARY JavaSQLType = -2 + JavaSQLTypeVARBINARY JavaSQLType = -3 + JavaSQLTypeLONGVARBINARY JavaSQLType = -4 + JavaSQLTypeNULL JavaSQLType = 0 + JavaSQLTypeBLOB JavaSQLType = 2004 + + // unused + //JavaSQLTypeFLOAT JavaSQLType = 6 + //JavaSQLTypeNUMERIC JavaSQLType = 2 + //JavaSQLTypeOTHER JavaSQLType = 1111 + //JavaSQLTypeJAVA_OBJECT JavaSQLType = 2000 + //JavaSQLTypeDISTINCT JavaSQLType = 2001 + //JavaSQLTypeSTRUCT JavaSQLType = 2002 + //JavaSQLTypeARRAY JavaSQLType = 2003 + //JavaSQLTypeCLOB JavaSQLType = 2005 + //JavaSQLTypeREF JavaSQLType = 2006 + //JavaSQLTypeDATALINK JavaSQLType = 70 + //JavaSQLTypeBOOLEAN JavaSQLType = 16 + //JavaSQLTypeROWID JavaSQLType = -8 + //JavaSQLTypeNCHAR JavaSQLType = -15 + //JavaSQLTypeNVARCHAR JavaSQLType = -9 + //JavaSQLTypeLONGNVARCHAR JavaSQLType = -16 + //JavaSQLTypeNCLOB JavaSQLType = 2011 + //JavaSQLTypeSQLXML JavaSQLType = 2009 + //JavaSQLTypeREF_CURSOR JavaSQLType = 2012 + //JavaSQLTypeTIME_WITH_TIMEZONE JavaSQLType = 2013 + //JavaSQLTypeTIMESTAMP_WITH_TIMEZONE JavaSQLType = 2014 +) + +// MysqlToJavaType converts the mysql protocol types to java sql types +func MysqlToJavaType(mysqlType byte) JavaSQLType { + // see https://github.com/mysql/mysql-connector-j/blob/5.1.49/src/com/mysql/jdbc/MysqlDefs.java + switch mysqlType { + case mysql.TypeNewDecimal, mysql.TypeDecimal: + return JavaSQLTypeDECIMAL + + case mysql.TypeTiny: + return JavaSQLTypeTINYINT + + case mysql.TypeShort: + return JavaSQLTypeSMALLINT + + case mysql.TypeLong: + return JavaSQLTypeINTEGER + + case mysql.TypeFloat: + return JavaSQLTypeREAL + + case mysql.TypeDouble: + return JavaSQLTypeDOUBLE + + case mysql.TypeNull: + return JavaSQLTypeNULL + + case mysql.TypeTimestamp: + return JavaSQLTypeTIMESTAMP + + case mysql.TypeLonglong: + return JavaSQLTypeBIGINT + + case mysql.TypeInt24: + return JavaSQLTypeINTEGER + + case mysql.TypeDate: + return JavaSQLTypeDATE + + case mysql.TypeDuration: + return JavaSQLTypeTIME + + case mysql.TypeDatetime: + return JavaSQLTypeTIMESTAMP + + case mysql.TypeYear: + return JavaSQLTypeDATE + + case mysql.TypeNewDate: + return JavaSQLTypeDATE + + case mysql.TypeEnum: + return JavaSQLTypeCHAR + + case mysql.TypeSet: + return JavaSQLTypeCHAR + + case mysql.TypeTinyBlob: + return JavaSQLTypeVARBINARY + + case mysql.TypeMediumBlob: + return JavaSQLTypeLONGVARBINARY + + case mysql.TypeLongBlob: + return JavaSQLTypeLONGVARBINARY + + case mysql.TypeBlob: + return JavaSQLTypeLONGVARBINARY + + case mysql.TypeVarString, mysql.TypeVarchar: + return JavaSQLTypeVARCHAR + + case mysql.TypeJSON: + // json: see jdbc 8.0, https://github.com/mysql/mysql-connector-j/blob/8.0.20/src/main/core-api/java/com/mysql/cj/MysqlType.java + return JavaSQLTypeLONGVARCHAR + + case mysql.TypeString: + return JavaSQLTypeCHAR + + case mysql.TypeGeometry: + return JavaSQLTypeBINARY + + case mysql.TypeBit: + return JavaSQLTypeBIT + + default: + return JavaSQLTypeVARCHAR + } +} diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 7384649031b..32ad6d5a660 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -40,6 +40,7 @@ type mqSink struct { dispatcher dispatcher.Dispatcher newEncoder func() codec.EventBatchEncoder filter *filter.Filter + protocol codec.Protocol partitionNum int32 partitionInput []chan struct { @@ -71,11 +72,15 @@ func newMqSink(ctx context.Context, mqProducer mqProducer.Producer, filter *filt return nil, errors.Trace(err) } notifier := new(notify.Notifier) + var protocol codec.Protocol + protocol.FromString(config.Sink.Protocol) + k := &mqSink{ mqProducer: mqProducer, dispatcher: d, - newEncoder: codec.NewJSONEventBatchEncoder, + newEncoder: codec.NewEventBatchEncoder(protocol), filter: filter, + protocol: protocol, partitionNum: partitionNum, partitionInput: partitionInput, @@ -157,6 +162,10 @@ flushLoop: } func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { + switch k.protocol { + case codec.ProtocolCanal: // ignore resolved events in canal protocol + return nil + } encoder := k.newEncoder() err := encoder.AppendResolvedEvent(ts) if err != nil { @@ -317,6 +326,11 @@ func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Fi config.Compression = s } + s = sinkURI.Query().Get("protocol") + if s != "" { + replicaConfig.Sink.Protocol = s + } + topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { return r == '/' }) diff --git a/cmd/cmd_test.go b/cmd/cmd_test.go index 1cecd0a3756..c1de836f041 100644 --- a/cmd/cmd_test.go +++ b/cmd/cmd_test.go @@ -49,6 +49,7 @@ dispatchers = [ {matcher = ['test1.*', 'test2.*'], dispatcher = "ts"}, {matcher = ['test3.*', 'test4.*'], dispatcher = "rowid"}, ] +protocol = "default" [cyclic-replication] enable = true @@ -82,6 +83,7 @@ polling-time = 5 {Dispatcher: "ts", Matcher: []string{"test1.*", "test2.*"}}, {Dispatcher: "rowid", Matcher: []string{"test3.*", "test4.*"}}, }, + Protocol: "default", }) c.Assert(cfg.Cyclic, check.DeepEquals, &config.CyclicConfig{ Enable: true, @@ -130,6 +132,11 @@ dispatchers = [ {matcher = ['test1.*', 'test2.*'], dispatcher = "ts"}, {matcher = ['test3.*', 'test4.*'], dispatcher = "rowid"}, ] +# 对于 MQ 类的 Sink,可以指定消息的协议格式 +# 协议目前支持 default, canal 两种,default 为 ticdc-open-protocol +# For MQ Sinks, you can configure the protocol of the messages sending to MQ +# Currently the protocol support default and canal +protocol = "default" [cyclic-replication] # 是否开启环形复制 @@ -165,6 +172,7 @@ sync-ddl = true {Dispatcher: "ts", Matcher: []string{"test1.*", "test2.*"}}, {Dispatcher: "rowid", Matcher: []string{"test3.*", "test4.*"}}, }, + Protocol: "default", }) c.Assert(cfg.Cyclic, check.DeepEquals, &config.CyclicConfig{ Enable: false, diff --git a/go.mod b/go.mod index 868d6db8c54..85ac024cdf0 100644 --- a/go.mod +++ b/go.mod @@ -12,6 +12,7 @@ require ( github.com/davecgh/go-spew v1.1.1 github.com/edwingeng/deque v0.0.0-20191220032131-8596380dee17 github.com/go-sql-driver/mysql v1.5.0 + github.com/golang/protobuf v1.3.4 github.com/google/btree v1.0.0 github.com/google/uuid v1.1.1 github.com/gorilla/websocket v1.4.1 // indirect @@ -35,6 +36,7 @@ require ( go.uber.org/zap v1.15.0 golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073 // indirect golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e + golang.org/x/text v0.3.2 golang.org/x/time v0.0.0-20191024005414-555d28b269f0 google.golang.org/genproto v0.0.0-20200113173426-e1de0a7b01eb // indirect google.golang.org/grpc v1.26.0 diff --git a/pkg/config/config.go b/pkg/config/config.go index 79bf477bd6a..b9c3c37a55d 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -32,7 +32,9 @@ var defaultReplicaConfig = &ReplicaConfig{ Mounter: &MounterConfig{ WorkerNum: 16, }, - Sink: &SinkConfig{}, + Sink: &SinkConfig{ + Protocol: "default", + }, Cyclic: &CyclicConfig{ Enable: false, }, diff --git a/pkg/config/sink.go b/pkg/config/sink.go index 1421a052d0b..90fff12451d 100644 --- a/pkg/config/sink.go +++ b/pkg/config/sink.go @@ -16,6 +16,7 @@ package config // SinkConfig represents sink config for a changefeed type SinkConfig struct { DispatchRules []*DispatchRule `toml:"dispatchers" json:"dispatchers"` + Protocol string `toml:"protocol" json:"protocol"` } // DispatchRule represents partition rule for a table diff --git a/proto/CanalProtocol.proto b/proto/CanalProtocol.proto new file mode 100644 index 00000000000..58a0ebbff3f --- /dev/null +++ b/proto/CanalProtocol.proto @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2010-2101 Alibaba Group Holding Limited. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; +package com.alibaba.otter.canal.protocol; + +option java_package = "com.alibaba.otter.canal.protocol"; +option java_outer_classname = "CanalPacket"; +option optimize_for = SPEED; + +enum Compression { + COMPRESSIONCOMPATIBLEPROTO2 = 0; + NONE = 1; + ZLIB = 2; + GZIP = 3; + LZF = 4; +} + +enum PacketType { + //compatible + PACKAGETYPECOMPATIBLEPROTO2 = 0; + HANDSHAKE = 1; + CLIENTAUTHENTICATION = 2; + ACK = 3; + SUBSCRIPTION = 4; + UNSUBSCRIPTION = 5; + GET = 6; + MESSAGES = 7; + CLIENTACK = 8; + // management part + SHUTDOWN = 9; + // integration + DUMP = 10; + HEARTBEAT = 11; + CLIENTROLLBACK = 12; +} + +message Packet { + //[default = 17]; + oneof magic_number_present { + int32 magic_number = 1; + } + //[default = 1]; + oneof version_present { + int32 version = 2; + }; + PacketType type = 3; + //[default = NONE]; + oneof compression_present { + Compression compression = 4; + } + + bytes body = 5; +} + +message HeartBeat { + int64 send_timestamp = 1; + int64 start_timestamp = 2; +} + +message Handshake { + // [default = "utf8"]; + oneof communication_encoding_present { + string communication_encoding = 1; + } + bytes seeds = 2; + Compression supported_compressions = 3; +} + +// client authentication +message ClientAuth { + string username = 1; + bytes password = 2; // hashed password with seeds from Handshake message + // [default = 0] + oneof net_read_timeout_present { + int32 net_read_timeout = 3; // in seconds + } + // [default = 0]; + oneof net_write_timeout_present { + int32 net_write_timeout = 4; // in seconds + } + string destination = 5; + string client_id = 6; + string filter = 7; + int64 start_timestamp = 8; +} + +message Ack { + //[default = 0] + oneof error_code_present { + int32 error_code = 1; + } + string error_message = 2; // if something like compression is not supported, erorr_message will tell about it. +} + +message ClientAck { + string destination = 1; + string client_id = 2; + int64 batch_id = 3; +} + +// subscription +message Sub { + string destination = 1; + string client_id = 2; + string filter = 7; +} + +// Unsubscription +message Unsub { + string destination = 1; + string client_id = 2; + string filter = 7; +} + +// PullRequest +message Get { + string destination = 1; + string client_id = 2; + int32 fetch_size = 3; + //[default = -1] + oneof timeout_present { + int64 timeout = 4; // 默认-1时代表不控制 + } + //[default = 2] + oneof unit_present { + int32 unit = 5;// 数字类型,0:纳秒,1:毫秒,2:微秒,3:秒,4:分钟,5:小时,6:天 + } + //[default = false] + oneof auto_ack_present { + bool auto_ack = 6; // 是否自动ack + } + +} + +// +message Messages { + int64 batch_id = 1; + repeated bytes messages = 2; +} + +// TBD when new packets are required +message Dump{ + string journal = 1; + int64 position = 2; + // [default = 0] + oneof timestamp_present { + int64 timestamp = 3; + } + +} + +message ClientRollback{ + string destination = 1; + string client_id = 2; + int64 batch_id = 3; +} \ No newline at end of file diff --git a/proto/EntryProtocol.proto b/proto/EntryProtocol.proto new file mode 100644 index 00000000000..4f3c75f699a --- /dev/null +++ b/proto/EntryProtocol.proto @@ -0,0 +1,246 @@ +/* + * Copyright (C) 2010-2101 Alibaba Group Holding Limited. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = "proto3"; +package com.alibaba.otter.canal.protocol; + +option java_package = "com.alibaba.otter.canal.protocol"; +option java_outer_classname = "CanalEntry"; +option optimize_for = SPEED; + +/**************************************************************** + * message model + *如果要在Enum中新增类型,确保以前的类型的下标值不变. + ****************************************************************/ +message Entry { + /**协议头部信息**/ + Header header = 1; + ///**打散后的事件类型**/ [default = ROWDATA] + oneof entryType_present{ + EntryType entryType = 2; + } + + /**传输的二进制数组**/ + bytes storeValue = 3; +} + +/**message Header**/ +message Header { + /**协议的版本号**/ //[default = 1] + oneof version_present { + int32 version = 1; + } + + + /**binlog/redolog 文件名**/ + string logfileName = 2; + + /**binlog/redolog 文件的偏移位置**/ + int64 logfileOffset = 3; + + /**服务端serverId**/ + int64 serverId = 4; + + /** 变更数据的编码 **/ + string serverenCode = 5; + + /**变更数据的执行时间 **/ + int64 executeTime = 6; + + /** 变更数据的来源**/ //[default = MYSQL] + oneof sourceType_present { + Type sourceType = 7; + } + + + /** 变更数据的schemaname**/ + string schemaName = 8; + + /**变更数据的tablename**/ + string tableName = 9; + + /**每个event的长度**/ + int64 eventLength = 10; + + /**数据变更类型**/ // [default = UPDATE] + oneof eventType_present { + EventType eventType = 11; + } + + + /**预留扩展**/ + repeated Pair props = 12; + + /**当前事务的gitd**/ + string gtid = 13; +} + +/**每个字段的数据结构**/ +message Column { + /**字段下标**/ + int32 index = 1; + + /**字段java中类型**/ + int32 sqlType = 2; + + /**字段名称(忽略大小写),在mysql中是没有的**/ + string name = 3; + + /**是否是主键**/ + bool isKey = 4; + + /**如果EventType=UPDATE,用于标识这个字段值是否有修改**/ + bool updated = 5; + + /** 标识是否为空 **/ //[default = false] + oneof isNull_present { + bool isNull = 6; + } + + + /**预留扩展**/ + repeated Pair props = 7; + + /** 字段值,timestamp,Datetime是一个时间格式的文本 **/ + string value = 8; + + /** 对应数据对象原始长度 **/ + int32 length = 9; + + /**字段mysql类型**/ + string mysqlType = 10; +} + +message RowData { + + /** 字段信息,增量数据(修改前,删除前) **/ + repeated Column beforeColumns = 1; + + /** 字段信息,增量数据(修改后,新增后) **/ + repeated Column afterColumns = 2; + + /**预留扩展**/ + repeated Pair props = 3; +} + +/**message row 每行变更数据的数据结构**/ +message RowChange { + + /**tableId,由数据库产生**/ + int64 tableId = 1; + + + /**数据变更类型**/ //[default = UPDATE] + oneof eventType_present { + EventType eventType = 2; + } + + + /** 标识是否是ddl语句 **/ // [default = false] + oneof isDdl_present { + bool isDdl = 10; + } + + + /** ddl/query的sql语句 **/ + string sql = 11; + + /** 一次数据库变更可能存在多行 **/ + repeated RowData rowDatas = 12; + + /**预留扩展**/ + repeated Pair props = 13; + + /** ddl/query的schemaName,会存在跨库ddl,需要保留执行ddl的当前schemaName **/ + string ddlSchemaName = 14; +} + +/**开始事务的一些信息**/ +message TransactionBegin{ + + /**已废弃,请使用header里的executeTime**/ + int64 executeTime = 1; + + /**已废弃,Begin里不提供事务id**/ + string transactionId = 2; + + /**预留扩展**/ + repeated Pair props = 3; + + /**执行的thread Id**/ + int64 threadId = 4; +} + +/**结束事务的一些信息**/ +message TransactionEnd{ + + /**已废弃,请使用header里的executeTime**/ + int64 executeTime = 1; + + /**事务号**/ + string transactionId = 2; + + /**预留扩展**/ + repeated Pair props = 3; +} + +/**预留扩展**/ +message Pair{ + string key = 1; + string value = 2; +} + +/**打散后的事件类型,主要用于标识事务的开始,变更数据,结束**/ +enum EntryType{ + ENTRYTYPECOMPATIBLEPROTO2 = 0; + TRANSACTIONBEGIN = 1; + ROWDATA = 2; + TRANSACTIONEND = 3; + /** 心跳类型,内部使用,外部暂不可见,可忽略 **/ + HEARTBEAT = 4; + GTIDLOG = 5; +} + +/** 事件类型 **/ +enum EventType { + EVENTTYPECOMPATIBLEPROTO2 = 0; + INSERT = 1; + UPDATE = 2; + DELETE = 3; + CREATE = 4; + ALTER = 5; + ERASE = 6; + QUERY = 7; + TRUNCATE = 8; + RENAME = 9; + /**CREATE INDEX**/ + CINDEX = 10; + DINDEX = 11; + GTID = 12; + /** XA **/ + XACOMMIT = 13; + XAROLLBACK = 14; + /** MASTER HEARTBEAT **/ + MHEARTBEAT = 15; +} + +/**数据库类型**/ +enum Type { + TYPECOMPATIBLEPROTO2 = 0; + ORACLE = 1; + MYSQL = 2; + PGSQL = 3; +} diff --git a/proto/canal/CanalProtocol.pb.go b/proto/canal/CanalProtocol.pb.go new file mode 100644 index 00000000000..6b9c22c3d06 --- /dev/null +++ b/proto/canal/CanalProtocol.pb.go @@ -0,0 +1,4352 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: CanalProtocol.proto + +package com_alibaba_otter_canal_protocol + +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 Compression int32 + +const ( + Compression_COMPRESSIONCOMPATIBLEPROTO2 Compression = 0 + Compression_NONE Compression = 1 + Compression_ZLIB Compression = 2 + Compression_GZIP Compression = 3 + Compression_LZF Compression = 4 +) + +var Compression_name = map[int32]string{ + 0: "COMPRESSIONCOMPATIBLEPROTO2", + 1: "NONE", + 2: "ZLIB", + 3: "GZIP", + 4: "LZF", +} + +var Compression_value = map[string]int32{ + "COMPRESSIONCOMPATIBLEPROTO2": 0, + "NONE": 1, + "ZLIB": 2, + "GZIP": 3, + "LZF": 4, +} + +func (x Compression) String() string { + return proto.EnumName(Compression_name, int32(x)) +} + +func (Compression) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{0} +} + +type PacketType int32 + +const ( + //compatible + PacketType_PACKAGETYPECOMPATIBLEPROTO2 PacketType = 0 + PacketType_HANDSHAKE PacketType = 1 + PacketType_CLIENTAUTHENTICATION PacketType = 2 + PacketType_ACK PacketType = 3 + PacketType_SUBSCRIPTION PacketType = 4 + PacketType_UNSUBSCRIPTION PacketType = 5 + PacketType_GET PacketType = 6 + PacketType_MESSAGES PacketType = 7 + PacketType_CLIENTACK PacketType = 8 + // management part + PacketType_SHUTDOWN PacketType = 9 + // integration + PacketType_DUMP PacketType = 10 + PacketType_HEARTBEAT PacketType = 11 + PacketType_CLIENTROLLBACK PacketType = 12 +) + +var PacketType_name = map[int32]string{ + 0: "PACKAGETYPECOMPATIBLEPROTO2", + 1: "HANDSHAKE", + 2: "CLIENTAUTHENTICATION", + 3: "ACK", + 4: "SUBSCRIPTION", + 5: "UNSUBSCRIPTION", + 6: "GET", + 7: "MESSAGES", + 8: "CLIENTACK", + 9: "SHUTDOWN", + 10: "DUMP", + 11: "HEARTBEAT", + 12: "CLIENTROLLBACK", +} + +var PacketType_value = map[string]int32{ + "PACKAGETYPECOMPATIBLEPROTO2": 0, + "HANDSHAKE": 1, + "CLIENTAUTHENTICATION": 2, + "ACK": 3, + "SUBSCRIPTION": 4, + "UNSUBSCRIPTION": 5, + "GET": 6, + "MESSAGES": 7, + "CLIENTACK": 8, + "SHUTDOWN": 9, + "DUMP": 10, + "HEARTBEAT": 11, + "CLIENTROLLBACK": 12, +} + +func (x PacketType) String() string { + return proto.EnumName(PacketType_name, int32(x)) +} + +func (PacketType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{1} +} + +type Packet struct { + //[default = 17]; + // + // Types that are valid to be assigned to MagicNumberPresent: + // *Packet_MagicNumber + MagicNumberPresent isPacket_MagicNumberPresent `protobuf_oneof:"magic_number_present"` + //[default = 1]; + // + // Types that are valid to be assigned to VersionPresent: + // *Packet_Version + VersionPresent isPacket_VersionPresent `protobuf_oneof:"version_present"` + Type PacketType `protobuf:"varint,3,opt,name=type,proto3,enum=com.alibaba.otter.canal.protocol.PacketType" json:"type,omitempty"` + //[default = NONE]; + // + // Types that are valid to be assigned to CompressionPresent: + // *Packet_Compression + CompressionPresent isPacket_CompressionPresent `protobuf_oneof:"compression_present"` + Body []byte `protobuf:"bytes,5,opt,name=body,proto3" json:"body,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Packet) Reset() { *m = Packet{} } +func (m *Packet) String() string { return proto.CompactTextString(m) } +func (*Packet) ProtoMessage() {} +func (*Packet) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{0} +} +func (m *Packet) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Packet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Packet.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 *Packet) XXX_Merge(src proto.Message) { + xxx_messageInfo_Packet.Merge(m, src) +} +func (m *Packet) XXX_Size() int { + return m.Size() +} +func (m *Packet) XXX_DiscardUnknown() { + xxx_messageInfo_Packet.DiscardUnknown(m) +} + +var xxx_messageInfo_Packet proto.InternalMessageInfo + +type isPacket_MagicNumberPresent interface { + isPacket_MagicNumberPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isPacket_VersionPresent interface { + isPacket_VersionPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isPacket_CompressionPresent interface { + isPacket_CompressionPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Packet_MagicNumber struct { + MagicNumber int32 `protobuf:"varint,1,opt,name=magic_number,json=magicNumber,proto3,oneof" json:"magic_number,omitempty"` +} +type Packet_Version struct { + Version int32 `protobuf:"varint,2,opt,name=version,proto3,oneof" json:"version,omitempty"` +} +type Packet_Compression struct { + Compression Compression `protobuf:"varint,4,opt,name=compression,proto3,enum=com.alibaba.otter.canal.protocol.Compression,oneof" json:"compression,omitempty"` +} + +func (*Packet_MagicNumber) isPacket_MagicNumberPresent() {} +func (*Packet_Version) isPacket_VersionPresent() {} +func (*Packet_Compression) isPacket_CompressionPresent() {} + +func (m *Packet) GetMagicNumberPresent() isPacket_MagicNumberPresent { + if m != nil { + return m.MagicNumberPresent + } + return nil +} +func (m *Packet) GetVersionPresent() isPacket_VersionPresent { + if m != nil { + return m.VersionPresent + } + return nil +} +func (m *Packet) GetCompressionPresent() isPacket_CompressionPresent { + if m != nil { + return m.CompressionPresent + } + return nil +} + +func (m *Packet) GetMagicNumber() int32 { + if x, ok := m.GetMagicNumberPresent().(*Packet_MagicNumber); ok { + return x.MagicNumber + } + return 0 +} + +func (m *Packet) GetVersion() int32 { + if x, ok := m.GetVersionPresent().(*Packet_Version); ok { + return x.Version + } + return 0 +} + +func (m *Packet) GetType() PacketType { + if m != nil { + return m.Type + } + return PacketType_PACKAGETYPECOMPATIBLEPROTO2 +} + +func (m *Packet) GetCompression() Compression { + if x, ok := m.GetCompressionPresent().(*Packet_Compression); ok { + return x.Compression + } + return Compression_COMPRESSIONCOMPATIBLEPROTO2 +} + +func (m *Packet) GetBody() []byte { + if m != nil { + return m.Body + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Packet) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Packet_MagicNumber)(nil), + (*Packet_Version)(nil), + (*Packet_Compression)(nil), + } +} + +type HeartBeat struct { + SendTimestamp int64 `protobuf:"varint,1,opt,name=send_timestamp,json=sendTimestamp,proto3" json:"send_timestamp,omitempty"` + StartTimestamp int64 `protobuf:"varint,2,opt,name=start_timestamp,json=startTimestamp,proto3" json:"start_timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HeartBeat) Reset() { *m = HeartBeat{} } +func (m *HeartBeat) String() string { return proto.CompactTextString(m) } +func (*HeartBeat) ProtoMessage() {} +func (*HeartBeat) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{1} +} +func (m *HeartBeat) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HeartBeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_HeartBeat.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 *HeartBeat) XXX_Merge(src proto.Message) { + xxx_messageInfo_HeartBeat.Merge(m, src) +} +func (m *HeartBeat) XXX_Size() int { + return m.Size() +} +func (m *HeartBeat) XXX_DiscardUnknown() { + xxx_messageInfo_HeartBeat.DiscardUnknown(m) +} + +var xxx_messageInfo_HeartBeat proto.InternalMessageInfo + +func (m *HeartBeat) GetSendTimestamp() int64 { + if m != nil { + return m.SendTimestamp + } + return 0 +} + +func (m *HeartBeat) GetStartTimestamp() int64 { + if m != nil { + return m.StartTimestamp + } + return 0 +} + +type Handshake struct { + // [default = "utf8"]; + // + // Types that are valid to be assigned to CommunicationEncodingPresent: + // *Handshake_CommunicationEncoding + CommunicationEncodingPresent isHandshake_CommunicationEncodingPresent `protobuf_oneof:"communication_encoding_present"` + Seeds []byte `protobuf:"bytes,2,opt,name=seeds,proto3" json:"seeds,omitempty"` + SupportedCompressions Compression `protobuf:"varint,3,opt,name=supported_compressions,json=supportedCompressions,proto3,enum=com.alibaba.otter.canal.protocol.Compression" json:"supported_compressions,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Handshake) Reset() { *m = Handshake{} } +func (m *Handshake) String() string { return proto.CompactTextString(m) } +func (*Handshake) ProtoMessage() {} +func (*Handshake) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{2} +} +func (m *Handshake) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Handshake) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Handshake.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 *Handshake) XXX_Merge(src proto.Message) { + xxx_messageInfo_Handshake.Merge(m, src) +} +func (m *Handshake) XXX_Size() int { + return m.Size() +} +func (m *Handshake) XXX_DiscardUnknown() { + xxx_messageInfo_Handshake.DiscardUnknown(m) +} + +var xxx_messageInfo_Handshake proto.InternalMessageInfo + +type isHandshake_CommunicationEncodingPresent interface { + isHandshake_CommunicationEncodingPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Handshake_CommunicationEncoding struct { + CommunicationEncoding string `protobuf:"bytes,1,opt,name=communication_encoding,json=communicationEncoding,proto3,oneof" json:"communication_encoding,omitempty"` +} + +func (*Handshake_CommunicationEncoding) isHandshake_CommunicationEncodingPresent() {} + +func (m *Handshake) GetCommunicationEncodingPresent() isHandshake_CommunicationEncodingPresent { + if m != nil { + return m.CommunicationEncodingPresent + } + return nil +} + +func (m *Handshake) GetCommunicationEncoding() string { + if x, ok := m.GetCommunicationEncodingPresent().(*Handshake_CommunicationEncoding); ok { + return x.CommunicationEncoding + } + return "" +} + +func (m *Handshake) GetSeeds() []byte { + if m != nil { + return m.Seeds + } + return nil +} + +func (m *Handshake) GetSupportedCompressions() Compression { + if m != nil { + return m.SupportedCompressions + } + return Compression_COMPRESSIONCOMPATIBLEPROTO2 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Handshake) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Handshake_CommunicationEncoding)(nil), + } +} + +// client authentication +type ClientAuth struct { + Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + Password []byte `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"` + // [default = 0] + // + // Types that are valid to be assigned to NetReadTimeoutPresent: + // *ClientAuth_NetReadTimeout + NetReadTimeoutPresent isClientAuth_NetReadTimeoutPresent `protobuf_oneof:"net_read_timeout_present"` + // [default = 0]; + // + // Types that are valid to be assigned to NetWriteTimeoutPresent: + // *ClientAuth_NetWriteTimeout + NetWriteTimeoutPresent isClientAuth_NetWriteTimeoutPresent `protobuf_oneof:"net_write_timeout_present"` + Destination string `protobuf:"bytes,5,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,6,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + Filter string `protobuf:"bytes,7,opt,name=filter,proto3" json:"filter,omitempty"` + StartTimestamp int64 `protobuf:"varint,8,opt,name=start_timestamp,json=startTimestamp,proto3" json:"start_timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ClientAuth) Reset() { *m = ClientAuth{} } +func (m *ClientAuth) String() string { return proto.CompactTextString(m) } +func (*ClientAuth) ProtoMessage() {} +func (*ClientAuth) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{3} +} +func (m *ClientAuth) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientAuth) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ClientAuth.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 *ClientAuth) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientAuth.Merge(m, src) +} +func (m *ClientAuth) XXX_Size() int { + return m.Size() +} +func (m *ClientAuth) XXX_DiscardUnknown() { + xxx_messageInfo_ClientAuth.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientAuth proto.InternalMessageInfo + +type isClientAuth_NetReadTimeoutPresent interface { + isClientAuth_NetReadTimeoutPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isClientAuth_NetWriteTimeoutPresent interface { + isClientAuth_NetWriteTimeoutPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type ClientAuth_NetReadTimeout struct { + NetReadTimeout int32 `protobuf:"varint,3,opt,name=net_read_timeout,json=netReadTimeout,proto3,oneof" json:"net_read_timeout,omitempty"` +} +type ClientAuth_NetWriteTimeout struct { + NetWriteTimeout int32 `protobuf:"varint,4,opt,name=net_write_timeout,json=netWriteTimeout,proto3,oneof" json:"net_write_timeout,omitempty"` +} + +func (*ClientAuth_NetReadTimeout) isClientAuth_NetReadTimeoutPresent() {} +func (*ClientAuth_NetWriteTimeout) isClientAuth_NetWriteTimeoutPresent() {} + +func (m *ClientAuth) GetNetReadTimeoutPresent() isClientAuth_NetReadTimeoutPresent { + if m != nil { + return m.NetReadTimeoutPresent + } + return nil +} +func (m *ClientAuth) GetNetWriteTimeoutPresent() isClientAuth_NetWriteTimeoutPresent { + if m != nil { + return m.NetWriteTimeoutPresent + } + return nil +} + +func (m *ClientAuth) GetUsername() string { + if m != nil { + return m.Username + } + return "" +} + +func (m *ClientAuth) GetPassword() []byte { + if m != nil { + return m.Password + } + return nil +} + +func (m *ClientAuth) GetNetReadTimeout() int32 { + if x, ok := m.GetNetReadTimeoutPresent().(*ClientAuth_NetReadTimeout); ok { + return x.NetReadTimeout + } + return 0 +} + +func (m *ClientAuth) GetNetWriteTimeout() int32 { + if x, ok := m.GetNetWriteTimeoutPresent().(*ClientAuth_NetWriteTimeout); ok { + return x.NetWriteTimeout + } + return 0 +} + +func (m *ClientAuth) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *ClientAuth) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *ClientAuth) GetFilter() string { + if m != nil { + return m.Filter + } + return "" +} + +func (m *ClientAuth) GetStartTimestamp() int64 { + if m != nil { + return m.StartTimestamp + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*ClientAuth) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*ClientAuth_NetReadTimeout)(nil), + (*ClientAuth_NetWriteTimeout)(nil), + } +} + +type Ack struct { + //[default = 0] + // + // Types that are valid to be assigned to ErrorCodePresent: + // *Ack_ErrorCode + ErrorCodePresent isAck_ErrorCodePresent `protobuf_oneof:"error_code_present"` + ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Ack) Reset() { *m = Ack{} } +func (m *Ack) String() string { return proto.CompactTextString(m) } +func (*Ack) ProtoMessage() {} +func (*Ack) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{4} +} +func (m *Ack) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Ack) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Ack.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 *Ack) XXX_Merge(src proto.Message) { + xxx_messageInfo_Ack.Merge(m, src) +} +func (m *Ack) XXX_Size() int { + return m.Size() +} +func (m *Ack) XXX_DiscardUnknown() { + xxx_messageInfo_Ack.DiscardUnknown(m) +} + +var xxx_messageInfo_Ack proto.InternalMessageInfo + +type isAck_ErrorCodePresent interface { + isAck_ErrorCodePresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Ack_ErrorCode struct { + ErrorCode int32 `protobuf:"varint,1,opt,name=error_code,json=errorCode,proto3,oneof" json:"error_code,omitempty"` +} + +func (*Ack_ErrorCode) isAck_ErrorCodePresent() {} + +func (m *Ack) GetErrorCodePresent() isAck_ErrorCodePresent { + if m != nil { + return m.ErrorCodePresent + } + return nil +} + +func (m *Ack) GetErrorCode() int32 { + if x, ok := m.GetErrorCodePresent().(*Ack_ErrorCode); ok { + return x.ErrorCode + } + return 0 +} + +func (m *Ack) GetErrorMessage() string { + if m != nil { + return m.ErrorMessage + } + return "" +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Ack) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Ack_ErrorCode)(nil), + } +} + +type ClientAck struct { + Destination string `protobuf:"bytes,1,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + BatchId int64 `protobuf:"varint,3,opt,name=batch_id,json=batchId,proto3" json:"batch_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ClientAck) Reset() { *m = ClientAck{} } +func (m *ClientAck) String() string { return proto.CompactTextString(m) } +func (*ClientAck) ProtoMessage() {} +func (*ClientAck) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{5} +} +func (m *ClientAck) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientAck) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ClientAck.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 *ClientAck) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientAck.Merge(m, src) +} +func (m *ClientAck) XXX_Size() int { + return m.Size() +} +func (m *ClientAck) XXX_DiscardUnknown() { + xxx_messageInfo_ClientAck.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientAck proto.InternalMessageInfo + +func (m *ClientAck) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *ClientAck) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *ClientAck) GetBatchId() int64 { + if m != nil { + return m.BatchId + } + return 0 +} + +// subscription +type Sub struct { + Destination string `protobuf:"bytes,1,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + Filter string `protobuf:"bytes,7,opt,name=filter,proto3" json:"filter,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Sub) Reset() { *m = Sub{} } +func (m *Sub) String() string { return proto.CompactTextString(m) } +func (*Sub) ProtoMessage() {} +func (*Sub) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{6} +} +func (m *Sub) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Sub) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Sub.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 *Sub) XXX_Merge(src proto.Message) { + xxx_messageInfo_Sub.Merge(m, src) +} +func (m *Sub) XXX_Size() int { + return m.Size() +} +func (m *Sub) XXX_DiscardUnknown() { + xxx_messageInfo_Sub.DiscardUnknown(m) +} + +var xxx_messageInfo_Sub proto.InternalMessageInfo + +func (m *Sub) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *Sub) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *Sub) GetFilter() string { + if m != nil { + return m.Filter + } + return "" +} + +// Unsubscription +type Unsub struct { + Destination string `protobuf:"bytes,1,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + Filter string `protobuf:"bytes,7,opt,name=filter,proto3" json:"filter,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Unsub) Reset() { *m = Unsub{} } +func (m *Unsub) String() string { return proto.CompactTextString(m) } +func (*Unsub) ProtoMessage() {} +func (*Unsub) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{7} +} +func (m *Unsub) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Unsub) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Unsub.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 *Unsub) XXX_Merge(src proto.Message) { + xxx_messageInfo_Unsub.Merge(m, src) +} +func (m *Unsub) XXX_Size() int { + return m.Size() +} +func (m *Unsub) XXX_DiscardUnknown() { + xxx_messageInfo_Unsub.DiscardUnknown(m) +} + +var xxx_messageInfo_Unsub proto.InternalMessageInfo + +func (m *Unsub) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *Unsub) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *Unsub) GetFilter() string { + if m != nil { + return m.Filter + } + return "" +} + +// PullRequest +type Get struct { + Destination string `protobuf:"bytes,1,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + FetchSize int32 `protobuf:"varint,3,opt,name=fetch_size,json=fetchSize,proto3" json:"fetch_size,omitempty"` + //[default = -1] + // + // Types that are valid to be assigned to TimeoutPresent: + // *Get_Timeout + TimeoutPresent isGet_TimeoutPresent `protobuf_oneof:"timeout_present"` + //[default = 2] + // + // Types that are valid to be assigned to UnitPresent: + // *Get_Unit + UnitPresent isGet_UnitPresent `protobuf_oneof:"unit_present"` + //[default = false] + // + // Types that are valid to be assigned to AutoAckPresent: + // *Get_AutoAck + AutoAckPresent isGet_AutoAckPresent `protobuf_oneof:"auto_ack_present"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Get) Reset() { *m = Get{} } +func (m *Get) String() string { return proto.CompactTextString(m) } +func (*Get) ProtoMessage() {} +func (*Get) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{8} +} +func (m *Get) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Get) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Get.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 *Get) XXX_Merge(src proto.Message) { + xxx_messageInfo_Get.Merge(m, src) +} +func (m *Get) XXX_Size() int { + return m.Size() +} +func (m *Get) XXX_DiscardUnknown() { + xxx_messageInfo_Get.DiscardUnknown(m) +} + +var xxx_messageInfo_Get proto.InternalMessageInfo + +type isGet_TimeoutPresent interface { + isGet_TimeoutPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isGet_UnitPresent interface { + isGet_UnitPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isGet_AutoAckPresent interface { + isGet_AutoAckPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Get_Timeout struct { + Timeout int64 `protobuf:"varint,4,opt,name=timeout,proto3,oneof" json:"timeout,omitempty"` +} +type Get_Unit struct { + Unit int32 `protobuf:"varint,5,opt,name=unit,proto3,oneof" json:"unit,omitempty"` +} +type Get_AutoAck struct { + AutoAck bool `protobuf:"varint,6,opt,name=auto_ack,json=autoAck,proto3,oneof" json:"auto_ack,omitempty"` +} + +func (*Get_Timeout) isGet_TimeoutPresent() {} +func (*Get_Unit) isGet_UnitPresent() {} +func (*Get_AutoAck) isGet_AutoAckPresent() {} + +func (m *Get) GetTimeoutPresent() isGet_TimeoutPresent { + if m != nil { + return m.TimeoutPresent + } + return nil +} +func (m *Get) GetUnitPresent() isGet_UnitPresent { + if m != nil { + return m.UnitPresent + } + return nil +} +func (m *Get) GetAutoAckPresent() isGet_AutoAckPresent { + if m != nil { + return m.AutoAckPresent + } + return nil +} + +func (m *Get) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *Get) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *Get) GetFetchSize() int32 { + if m != nil { + return m.FetchSize + } + return 0 +} + +func (m *Get) GetTimeout() int64 { + if x, ok := m.GetTimeoutPresent().(*Get_Timeout); ok { + return x.Timeout + } + return 0 +} + +func (m *Get) GetUnit() int32 { + if x, ok := m.GetUnitPresent().(*Get_Unit); ok { + return x.Unit + } + return 0 +} + +func (m *Get) GetAutoAck() bool { + if x, ok := m.GetAutoAckPresent().(*Get_AutoAck); ok { + return x.AutoAck + } + return false +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Get) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Get_Timeout)(nil), + (*Get_Unit)(nil), + (*Get_AutoAck)(nil), + } +} + +// +type Messages struct { + BatchId int64 `protobuf:"varint,1,opt,name=batch_id,json=batchId,proto3" json:"batch_id,omitempty"` + Messages [][]byte `protobuf:"bytes,2,rep,name=messages,proto3" json:"messages,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Messages) Reset() { *m = Messages{} } +func (m *Messages) String() string { return proto.CompactTextString(m) } +func (*Messages) ProtoMessage() {} +func (*Messages) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{9} +} +func (m *Messages) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Messages) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Messages.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 *Messages) XXX_Merge(src proto.Message) { + xxx_messageInfo_Messages.Merge(m, src) +} +func (m *Messages) XXX_Size() int { + return m.Size() +} +func (m *Messages) XXX_DiscardUnknown() { + xxx_messageInfo_Messages.DiscardUnknown(m) +} + +var xxx_messageInfo_Messages proto.InternalMessageInfo + +func (m *Messages) GetBatchId() int64 { + if m != nil { + return m.BatchId + } + return 0 +} + +func (m *Messages) GetMessages() [][]byte { + if m != nil { + return m.Messages + } + return nil +} + +// TBD when new packets are required +type Dump struct { + Journal string `protobuf:"bytes,1,opt,name=journal,proto3" json:"journal,omitempty"` + Position int64 `protobuf:"varint,2,opt,name=position,proto3" json:"position,omitempty"` + // [default = 0] + // + // Types that are valid to be assigned to TimestampPresent: + // *Dump_Timestamp + TimestampPresent isDump_TimestampPresent `protobuf_oneof:"timestamp_present"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Dump) Reset() { *m = Dump{} } +func (m *Dump) String() string { return proto.CompactTextString(m) } +func (*Dump) ProtoMessage() {} +func (*Dump) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{10} +} +func (m *Dump) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Dump) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Dump.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 *Dump) XXX_Merge(src proto.Message) { + xxx_messageInfo_Dump.Merge(m, src) +} +func (m *Dump) XXX_Size() int { + return m.Size() +} +func (m *Dump) XXX_DiscardUnknown() { + xxx_messageInfo_Dump.DiscardUnknown(m) +} + +var xxx_messageInfo_Dump proto.InternalMessageInfo + +type isDump_TimestampPresent interface { + isDump_TimestampPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Dump_Timestamp struct { + Timestamp int64 `protobuf:"varint,3,opt,name=timestamp,proto3,oneof" json:"timestamp,omitempty"` +} + +func (*Dump_Timestamp) isDump_TimestampPresent() {} + +func (m *Dump) GetTimestampPresent() isDump_TimestampPresent { + if m != nil { + return m.TimestampPresent + } + return nil +} + +func (m *Dump) GetJournal() string { + if m != nil { + return m.Journal + } + return "" +} + +func (m *Dump) GetPosition() int64 { + if m != nil { + return m.Position + } + return 0 +} + +func (m *Dump) GetTimestamp() int64 { + if x, ok := m.GetTimestampPresent().(*Dump_Timestamp); ok { + return x.Timestamp + } + return 0 +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Dump) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Dump_Timestamp)(nil), + } +} + +type ClientRollback struct { + Destination string `protobuf:"bytes,1,opt,name=destination,proto3" json:"destination,omitempty"` + ClientId string `protobuf:"bytes,2,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + BatchId int64 `protobuf:"varint,3,opt,name=batch_id,json=batchId,proto3" json:"batch_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ClientRollback) Reset() { *m = ClientRollback{} } +func (m *ClientRollback) String() string { return proto.CompactTextString(m) } +func (*ClientRollback) ProtoMessage() {} +func (*ClientRollback) Descriptor() ([]byte, []int) { + return fileDescriptor_638f57be23f1b015, []int{11} +} +func (m *ClientRollback) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientRollback) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ClientRollback.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 *ClientRollback) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientRollback.Merge(m, src) +} +func (m *ClientRollback) XXX_Size() int { + return m.Size() +} +func (m *ClientRollback) XXX_DiscardUnknown() { + xxx_messageInfo_ClientRollback.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientRollback proto.InternalMessageInfo + +func (m *ClientRollback) GetDestination() string { + if m != nil { + return m.Destination + } + return "" +} + +func (m *ClientRollback) GetClientId() string { + if m != nil { + return m.ClientId + } + return "" +} + +func (m *ClientRollback) GetBatchId() int64 { + if m != nil { + return m.BatchId + } + return 0 +} + +func init() { + proto.RegisterEnum("com.alibaba.otter.canal.protocol.Compression", Compression_name, Compression_value) + proto.RegisterEnum("com.alibaba.otter.canal.protocol.PacketType", PacketType_name, PacketType_value) + proto.RegisterType((*Packet)(nil), "com.alibaba.otter.canal.protocol.Packet") + proto.RegisterType((*HeartBeat)(nil), "com.alibaba.otter.canal.protocol.HeartBeat") + proto.RegisterType((*Handshake)(nil), "com.alibaba.otter.canal.protocol.Handshake") + proto.RegisterType((*ClientAuth)(nil), "com.alibaba.otter.canal.protocol.ClientAuth") + proto.RegisterType((*Ack)(nil), "com.alibaba.otter.canal.protocol.Ack") + proto.RegisterType((*ClientAck)(nil), "com.alibaba.otter.canal.protocol.ClientAck") + proto.RegisterType((*Sub)(nil), "com.alibaba.otter.canal.protocol.Sub") + proto.RegisterType((*Unsub)(nil), "com.alibaba.otter.canal.protocol.Unsub") + proto.RegisterType((*Get)(nil), "com.alibaba.otter.canal.protocol.Get") + proto.RegisterType((*Messages)(nil), "com.alibaba.otter.canal.protocol.Messages") + proto.RegisterType((*Dump)(nil), "com.alibaba.otter.canal.protocol.Dump") + proto.RegisterType((*ClientRollback)(nil), "com.alibaba.otter.canal.protocol.ClientRollback") +} + +func init() { proto.RegisterFile("CanalProtocol.proto", fileDescriptor_638f57be23f1b015) } + +var fileDescriptor_638f57be23f1b015 = []byte{ + // 1038 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x5f, 0x6f, 0xe3, 0x44, + 0x10, 0xaf, 0xe3, 0xa4, 0x89, 0x27, 0x69, 0xea, 0x6e, 0xff, 0x28, 0xd7, 0x8a, 0x5c, 0x94, 0x13, + 0xa2, 0xaa, 0x8e, 0x3c, 0x1c, 0x0f, 0xbc, 0x62, 0xa7, 0xa6, 0x8e, 0x9a, 0x26, 0x61, 0xe3, 0xe8, + 0x44, 0x41, 0x44, 0x1b, 0x7b, 0xaf, 0xf5, 0x25, 0xf6, 0x06, 0x7b, 0xcd, 0xa9, 0xf7, 0x49, 0xf8, + 0x20, 0x7c, 0x08, 0x1e, 0x79, 0xe3, 0x15, 0x15, 0x09, 0x89, 0x6f, 0x81, 0x76, 0xed, 0xb8, 0x69, + 0x29, 0xaa, 0xd0, 0xe9, 0xde, 0x76, 0x7e, 0x33, 0x3b, 0xb3, 0xf3, 0x9b, 0xdf, 0xae, 0x0d, 0xbb, + 0x5d, 0x12, 0x92, 0xc5, 0x28, 0x62, 0x9c, 0xb9, 0x6c, 0xd1, 0x59, 0x8a, 0x05, 0x6a, 0xb9, 0x2c, + 0xe8, 0x90, 0x85, 0x3f, 0x23, 0x33, 0xd2, 0x61, 0x9c, 0xd3, 0xa8, 0xe3, 0x8a, 0xb0, 0xd4, 0xed, + 0xb2, 0x45, 0xfb, 0x97, 0x02, 0x6c, 0x8e, 0x88, 0x3b, 0xa7, 0x1c, 0xbd, 0x80, 0x5a, 0x40, 0xae, + 0x7c, 0x77, 0x1a, 0x26, 0xc1, 0x8c, 0x46, 0x0d, 0xa5, 0xa5, 0x1c, 0x97, 0xec, 0x0d, 0x5c, 0x95, + 0xe8, 0x40, 0x82, 0xe8, 0x10, 0xca, 0x3f, 0xd1, 0x28, 0xf6, 0x59, 0xd8, 0x28, 0x48, 0xbf, 0x82, + 0x57, 0x00, 0xfa, 0x0a, 0x8a, 0xfc, 0x66, 0x49, 0x1b, 0x6a, 0x4b, 0x39, 0xae, 0xbf, 0x7a, 0xd9, + 0x79, 0xaa, 0x78, 0x27, 0x2d, 0xec, 0xdc, 0x2c, 0x29, 0x96, 0x3b, 0xd1, 0x37, 0x50, 0x75, 0x59, + 0xb0, 0x8c, 0x68, 0x2c, 0x2b, 0x14, 0x65, 0xa2, 0xcf, 0x9f, 0x4e, 0xd4, 0xbd, 0xdb, 0x64, 0x17, + 0xf0, 0x7a, 0x0e, 0x84, 0xa0, 0x38, 0x63, 0xde, 0x4d, 0xa3, 0xd4, 0x52, 0x8e, 0x6b, 0x58, 0xae, + 0xcd, 0x03, 0xd8, 0x5b, 0xef, 0x74, 0x2a, 0x82, 0x69, 0xc8, 0xcd, 0x1d, 0xd8, 0xce, 0x7a, 0xc9, + 0xa1, 0x7d, 0xd8, 0x5d, 0xcb, 0xb6, 0x82, 0xdb, 0xdf, 0x81, 0x66, 0x53, 0x12, 0x71, 0x93, 0x12, + 0x8e, 0x3e, 0x85, 0x7a, 0x4c, 0x43, 0x6f, 0xca, 0xfd, 0x80, 0xc6, 0x9c, 0x04, 0x4b, 0x49, 0x9d, + 0x8a, 0xb7, 0x04, 0xea, 0xac, 0x40, 0xf4, 0x19, 0x6c, 0xc7, 0x9c, 0x44, 0x7c, 0x2d, 0xae, 0x20, + 0xe3, 0xea, 0x12, 0xce, 0x03, 0xdb, 0xb7, 0x0a, 0x68, 0x36, 0x09, 0xbd, 0xf8, 0x9a, 0xcc, 0x29, + 0xfa, 0x12, 0x0e, 0x5c, 0x16, 0x04, 0x49, 0xe8, 0xbb, 0x84, 0x8b, 0x33, 0xd0, 0xd0, 0x65, 0x9e, + 0x1f, 0x5e, 0xc9, 0x2a, 0x9a, 0xbd, 0x81, 0xf7, 0xef, 0xf9, 0xad, 0xcc, 0x8d, 0xf6, 0xa0, 0x14, + 0x53, 0xea, 0xc5, 0xb2, 0x4a, 0x0d, 0xa7, 0x06, 0xf2, 0xe0, 0x20, 0x4e, 0x96, 0x4b, 0x16, 0x71, + 0xea, 0x4d, 0xd7, 0x5a, 0x8b, 0xb3, 0xb1, 0xfd, 0x3f, 0xb6, 0xf1, 0x7e, 0x9e, 0x6c, 0x0d, 0x8d, + 0xcd, 0x16, 0x34, 0x1f, 0x3f, 0x74, 0xce, 0xe0, 0xef, 0x05, 0x80, 0xee, 0xc2, 0xa7, 0x21, 0x37, + 0x12, 0x7e, 0x8d, 0x0e, 0xa1, 0x92, 0xc4, 0x34, 0x0a, 0x49, 0x40, 0xd3, 0xbe, 0x70, 0x6e, 0x0b, + 0xdf, 0x92, 0xc4, 0xf1, 0x3b, 0x16, 0x79, 0x59, 0x2f, 0xb9, 0x8d, 0x4e, 0x40, 0x0f, 0x29, 0x9f, + 0x46, 0x94, 0xa4, 0xfc, 0xb3, 0x84, 0xcb, 0x46, 0x84, 0x70, 0xeb, 0x21, 0xe5, 0x98, 0x12, 0x39, + 0x02, 0x96, 0x70, 0xf4, 0x12, 0x76, 0x44, 0xec, 0xbb, 0xc8, 0xe7, 0x34, 0x0f, 0x2e, 0x66, 0x2a, + 0xde, 0x0e, 0x29, 0x7f, 0x2d, 0x3c, 0xab, 0xe8, 0x16, 0x54, 0x3d, 0x1a, 0x73, 0x3f, 0x94, 0x0d, + 0x48, 0xfd, 0x68, 0x78, 0x1d, 0x42, 0x47, 0xa0, 0xb9, 0xb2, 0x83, 0xa9, 0xef, 0x35, 0x36, 0xd3, + 0x43, 0xa7, 0x40, 0xcf, 0x43, 0x07, 0xb0, 0xf9, 0xc6, 0x5f, 0x70, 0x1a, 0x35, 0xca, 0xd2, 0x93, + 0x59, 0x8f, 0xa9, 0xa0, 0xf2, 0x98, 0x0a, 0xcc, 0x43, 0x68, 0x3c, 0xec, 0x2c, 0x57, 0xe5, 0x11, + 0x3c, 0xfb, 0x57, 0x27, 0x39, 0xb3, 0x04, 0x54, 0xc3, 0x9d, 0xa3, 0xe7, 0x00, 0x34, 0x8a, 0x58, + 0x34, 0x75, 0x99, 0x47, 0xf3, 0xcb, 0xac, 0x49, 0xac, 0xcb, 0x3c, 0x8a, 0x5e, 0xc0, 0x56, 0x1a, + 0x10, 0xd0, 0x38, 0x26, 0x57, 0x54, 0x72, 0xab, 0xe1, 0x9a, 0x04, 0x2f, 0x52, 0xcc, 0xdc, 0x03, + 0x74, 0x97, 0x25, 0x2f, 0x41, 0x41, 0xcb, 0x66, 0xe7, 0xce, 0x1f, 0x12, 0xa5, 0x3c, 0x41, 0x54, + 0xe1, 0x01, 0x51, 0xcf, 0xa0, 0x32, 0x23, 0xdc, 0xbd, 0x16, 0x3e, 0x55, 0x32, 0x51, 0x96, 0x76, + 0xcf, 0x6b, 0x7f, 0x0f, 0xea, 0x38, 0x99, 0x7d, 0x68, 0x81, 0xff, 0x98, 0x44, 0xfb, 0x07, 0x28, + 0x4d, 0xc2, 0xf8, 0xe3, 0xe5, 0xff, 0x5b, 0x01, 0xf5, 0x8c, 0xf2, 0x0f, 0x4d, 0xff, 0x09, 0xc0, + 0x1b, 0x2a, 0xf8, 0x89, 0xfd, 0xf7, 0xe9, 0xdb, 0x5a, 0xc2, 0x9a, 0x44, 0xc6, 0xfe, 0x7b, 0x71, + 0x39, 0xca, 0xeb, 0x52, 0x56, 0xed, 0x0d, 0xbc, 0x02, 0xd0, 0x1e, 0x14, 0x93, 0xd0, 0xe7, 0x52, + 0xbb, 0x42, 0xe3, 0xd2, 0x42, 0x47, 0x50, 0x21, 0x09, 0x67, 0x53, 0xe2, 0xce, 0xa5, 0x6a, 0x2b, + 0x76, 0x01, 0x97, 0x05, 0x62, 0xb8, 0x73, 0xf1, 0x04, 0x3e, 0x14, 0x5b, 0x1d, 0x6a, 0x62, 0x5f, + 0x6e, 0x23, 0xd0, 0x57, 0xfb, 0x73, 0x41, 0x18, 0x50, 0xc9, 0x14, 0x13, 0xdf, 0x1b, 0xa8, 0x72, + 0x6f, 0xa0, 0xe2, 0x26, 0x67, 0x62, 0x13, 0xaf, 0x92, 0x2a, 0x6e, 0xf2, 0xca, 0x6e, 0xff, 0x08, + 0xc5, 0xd3, 0x24, 0x58, 0xa2, 0x06, 0x94, 0xdf, 0xb2, 0x24, 0x0a, 0xc9, 0x22, 0xa3, 0x6a, 0x65, + 0xca, 0x77, 0x80, 0xc5, 0x3e, 0x5f, 0x7d, 0x7c, 0x54, 0x9c, 0xdb, 0xa8, 0x09, 0xda, 0xdd, 0x85, + 0x52, 0x33, 0x22, 0xee, 0x20, 0x73, 0x17, 0x76, 0x72, 0x23, 0x3f, 0xf5, 0x5b, 0xa8, 0xa7, 0x32, + 0xc6, 0x6c, 0xb1, 0x98, 0x91, 0x8f, 0xa9, 0xe5, 0x93, 0x09, 0x54, 0xd7, 0x5e, 0x48, 0xf4, 0x1c, + 0x8e, 0xba, 0xc3, 0x8b, 0x11, 0xb6, 0xc6, 0xe3, 0xde, 0x70, 0x20, 0x96, 0x86, 0xd3, 0x33, 0xfb, + 0xd6, 0x08, 0x0f, 0x9d, 0xe1, 0x2b, 0x7d, 0x03, 0x55, 0xa0, 0x38, 0x18, 0x0e, 0x2c, 0x5d, 0x11, + 0xab, 0xcb, 0x7e, 0xcf, 0xd4, 0x0b, 0x62, 0x75, 0x76, 0xd9, 0x1b, 0xe9, 0x2a, 0x2a, 0x83, 0xda, + 0xbf, 0xfc, 0x5a, 0x2f, 0x9e, 0xfc, 0xa5, 0x00, 0xdc, 0x7d, 0x46, 0x45, 0xda, 0x91, 0xd1, 0x3d, + 0x37, 0xce, 0x2c, 0xe7, 0xdb, 0x91, 0xf5, 0x48, 0xda, 0x2d, 0xd0, 0x6c, 0x63, 0x70, 0x3a, 0xb6, + 0x8d, 0x73, 0x91, 0xbb, 0x01, 0x7b, 0xdd, 0x7e, 0xcf, 0x1a, 0x38, 0xc6, 0xc4, 0xb1, 0xad, 0x81, + 0xd3, 0xeb, 0x1a, 0x4e, 0x6f, 0x38, 0xd0, 0x0b, 0xa2, 0x82, 0xd1, 0x3d, 0xd7, 0x55, 0xa4, 0x43, + 0x6d, 0x3c, 0x31, 0xc7, 0x5d, 0xdc, 0x1b, 0x49, 0x57, 0x11, 0x21, 0xa8, 0x4f, 0x06, 0xf7, 0xb0, + 0x92, 0x08, 0x3f, 0xb3, 0x1c, 0x7d, 0x13, 0xd5, 0xa0, 0x72, 0x61, 0x8d, 0xc7, 0xc6, 0x99, 0x35, + 0xd6, 0xcb, 0xa2, 0x5c, 0x96, 0xbf, 0x7b, 0xae, 0x57, 0x84, 0x73, 0x6c, 0x4f, 0x9c, 0xd3, 0xe1, + 0xeb, 0x81, 0xae, 0x89, 0x76, 0x4e, 0x27, 0x17, 0x23, 0x1d, 0xe4, 0xa9, 0x2c, 0x03, 0x3b, 0xa6, + 0x65, 0x38, 0x7a, 0x55, 0x14, 0x48, 0x77, 0xe1, 0x61, 0xbf, 0x6f, 0x8a, 0xad, 0x35, 0xd3, 0xf8, + 0xf5, 0xb6, 0xa9, 0xfc, 0x76, 0xdb, 0x54, 0xfe, 0xb8, 0x6d, 0x2a, 0x3f, 0xff, 0xd9, 0xdc, 0x80, + 0x27, 0x7f, 0x6e, 0xcc, 0x6a, 0xfa, 0x4f, 0x24, 0xe9, 0xb1, 0x95, 0xd9, 0xa6, 0x74, 0x7c, 0xf1, + 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x97, 0x69, 0xe7, 0x75, 0x2b, 0x09, 0x00, 0x00, +} + +func (m *Packet) 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 *Packet) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Packet) 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.Body) > 0 { + i -= len(m.Body) + copy(dAtA[i:], m.Body) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Body))) + i-- + dAtA[i] = 0x2a + } + if m.CompressionPresent != nil { + { + size := m.CompressionPresent.Size() + i -= size + if _, err := m.CompressionPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.Type != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x18 + } + if m.VersionPresent != nil { + { + size := m.VersionPresent.Size() + i -= size + if _, err := m.VersionPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.MagicNumberPresent != nil { + { + size := m.MagicNumberPresent.Size() + i -= size + if _, err := m.MagicNumberPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Packet_MagicNumber) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Packet_MagicNumber) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.MagicNumber)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *Packet_Version) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Packet_Version) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x10 + return len(dAtA) - i, nil +} +func (m *Packet_Compression) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Packet_Compression) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Compression)) + i-- + dAtA[i] = 0x20 + return len(dAtA) - i, nil +} +func (m *HeartBeat) 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 *HeartBeat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *HeartBeat) 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.StartTimestamp != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.StartTimestamp)) + i-- + dAtA[i] = 0x10 + } + if m.SendTimestamp != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.SendTimestamp)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Handshake) 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 *Handshake) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Handshake) 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.SupportedCompressions != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.SupportedCompressions)) + i-- + dAtA[i] = 0x18 + } + if len(m.Seeds) > 0 { + i -= len(m.Seeds) + copy(dAtA[i:], m.Seeds) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Seeds))) + i-- + dAtA[i] = 0x12 + } + if m.CommunicationEncodingPresent != nil { + { + size := m.CommunicationEncodingPresent.Size() + i -= size + if _, err := m.CommunicationEncodingPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Handshake_CommunicationEncoding) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Handshake_CommunicationEncoding) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.CommunicationEncoding) + copy(dAtA[i:], m.CommunicationEncoding) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.CommunicationEncoding))) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} +func (m *ClientAuth) 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 *ClientAuth) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClientAuth) 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.StartTimestamp != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.StartTimestamp)) + i-- + dAtA[i] = 0x40 + } + if len(m.Filter) > 0 { + i -= len(m.Filter) + copy(dAtA[i:], m.Filter) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Filter))) + i-- + dAtA[i] = 0x3a + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x32 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0x2a + } + if m.NetWriteTimeoutPresent != nil { + { + size := m.NetWriteTimeoutPresent.Size() + i -= size + if _, err := m.NetWriteTimeoutPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.NetReadTimeoutPresent != nil { + { + size := m.NetReadTimeoutPresent.Size() + i -= size + if _, err := m.NetReadTimeoutPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if len(m.Password) > 0 { + i -= len(m.Password) + copy(dAtA[i:], m.Password) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Password))) + i-- + dAtA[i] = 0x12 + } + if len(m.Username) > 0 { + i -= len(m.Username) + copy(dAtA[i:], m.Username) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Username))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ClientAuth_NetReadTimeout) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClientAuth_NetReadTimeout) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.NetReadTimeout)) + i-- + dAtA[i] = 0x18 + return len(dAtA) - i, nil +} +func (m *ClientAuth_NetWriteTimeout) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClientAuth_NetWriteTimeout) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.NetWriteTimeout)) + i-- + dAtA[i] = 0x20 + return len(dAtA) - i, nil +} +func (m *Ack) 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 *Ack) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Ack) 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.ErrorMessage) > 0 { + i -= len(m.ErrorMessage) + copy(dAtA[i:], m.ErrorMessage) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ErrorMessage))) + i-- + dAtA[i] = 0x12 + } + if m.ErrorCodePresent != nil { + { + size := m.ErrorCodePresent.Size() + i -= size + if _, err := m.ErrorCodePresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Ack_ErrorCode) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Ack_ErrorCode) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.ErrorCode)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *ClientAck) 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 *ClientAck) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClientAck) 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.BatchId != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.BatchId)) + i-- + dAtA[i] = 0x18 + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Sub) 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 *Sub) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Sub) 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.Filter) > 0 { + i -= len(m.Filter) + copy(dAtA[i:], m.Filter) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Filter))) + i-- + dAtA[i] = 0x3a + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Unsub) 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 *Unsub) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Unsub) 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.Filter) > 0 { + i -= len(m.Filter) + copy(dAtA[i:], m.Filter) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Filter))) + i-- + dAtA[i] = 0x3a + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Get) 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 *Get) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Get) 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.AutoAckPresent != nil { + { + size := m.AutoAckPresent.Size() + i -= size + if _, err := m.AutoAckPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.UnitPresent != nil { + { + size := m.UnitPresent.Size() + i -= size + if _, err := m.UnitPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.TimeoutPresent != nil { + { + size := m.TimeoutPresent.Size() + i -= size + if _, err := m.TimeoutPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.FetchSize != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.FetchSize)) + i-- + dAtA[i] = 0x18 + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Get_Timeout) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Get_Timeout) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Timeout)) + i-- + dAtA[i] = 0x20 + return len(dAtA) - i, nil +} +func (m *Get_Unit) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Get_Unit) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Unit)) + i-- + dAtA[i] = 0x28 + return len(dAtA) - i, nil +} +func (m *Get_AutoAck) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Get_AutoAck) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i-- + if m.AutoAck { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + return len(dAtA) - i, nil +} +func (m *Messages) 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 *Messages) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Messages) 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.Messages) > 0 { + for iNdEx := len(m.Messages) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Messages[iNdEx]) + copy(dAtA[i:], m.Messages[iNdEx]) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Messages[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if m.BatchId != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.BatchId)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Dump) 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 *Dump) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Dump) 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.TimestampPresent != nil { + { + size := m.TimestampPresent.Size() + i -= size + if _, err := m.TimestampPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.Position != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Position)) + i-- + dAtA[i] = 0x10 + } + if len(m.Journal) > 0 { + i -= len(m.Journal) + copy(dAtA[i:], m.Journal) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Journal))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Dump_Timestamp) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Dump_Timestamp) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x18 + return len(dAtA) - i, nil +} +func (m *ClientRollback) 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 *ClientRollback) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ClientRollback) 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.BatchId != 0 { + i = encodeVarintCanalProtocol(dAtA, i, uint64(m.BatchId)) + i-- + dAtA[i] = 0x18 + } + if len(m.ClientId) > 0 { + i -= len(m.ClientId) + copy(dAtA[i:], m.ClientId) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.ClientId))) + i-- + dAtA[i] = 0x12 + } + if len(m.Destination) > 0 { + i -= len(m.Destination) + copy(dAtA[i:], m.Destination) + i = encodeVarintCanalProtocol(dAtA, i, uint64(len(m.Destination))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintCanalProtocol(dAtA []byte, offset int, v uint64) int { + offset -= sovCanalProtocol(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Packet) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.MagicNumberPresent != nil { + n += m.MagicNumberPresent.Size() + } + if m.VersionPresent != nil { + n += m.VersionPresent.Size() + } + if m.Type != 0 { + n += 1 + sovCanalProtocol(uint64(m.Type)) + } + if m.CompressionPresent != nil { + n += m.CompressionPresent.Size() + } + l = len(m.Body) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Packet_MagicNumber) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.MagicNumber)) + return n +} +func (m *Packet_Version) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.Version)) + return n +} +func (m *Packet_Compression) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.Compression)) + return n +} +func (m *HeartBeat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SendTimestamp != 0 { + n += 1 + sovCanalProtocol(uint64(m.SendTimestamp)) + } + if m.StartTimestamp != 0 { + n += 1 + sovCanalProtocol(uint64(m.StartTimestamp)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Handshake) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.CommunicationEncodingPresent != nil { + n += m.CommunicationEncodingPresent.Size() + } + l = len(m.Seeds) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.SupportedCompressions != 0 { + n += 1 + sovCanalProtocol(uint64(m.SupportedCompressions)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Handshake_CommunicationEncoding) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.CommunicationEncoding) + n += 1 + l + sovCanalProtocol(uint64(l)) + return n +} +func (m *ClientAuth) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Username) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.Password) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.NetReadTimeoutPresent != nil { + n += m.NetReadTimeoutPresent.Size() + } + if m.NetWriteTimeoutPresent != nil { + n += m.NetWriteTimeoutPresent.Size() + } + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.Filter) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.StartTimestamp != 0 { + n += 1 + sovCanalProtocol(uint64(m.StartTimestamp)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ClientAuth_NetReadTimeout) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.NetReadTimeout)) + return n +} +func (m *ClientAuth_NetWriteTimeout) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.NetWriteTimeout)) + return n +} +func (m *Ack) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ErrorCodePresent != nil { + n += m.ErrorCodePresent.Size() + } + l = len(m.ErrorMessage) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Ack_ErrorCode) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.ErrorCode)) + return n +} +func (m *ClientAck) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.BatchId != 0 { + n += 1 + sovCanalProtocol(uint64(m.BatchId)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Sub) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.Filter) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Unsub) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.Filter) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Get) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.FetchSize != 0 { + n += 1 + sovCanalProtocol(uint64(m.FetchSize)) + } + if m.TimeoutPresent != nil { + n += m.TimeoutPresent.Size() + } + if m.UnitPresent != nil { + n += m.UnitPresent.Size() + } + if m.AutoAckPresent != nil { + n += m.AutoAckPresent.Size() + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Get_Timeout) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.Timeout)) + return n +} +func (m *Get_Unit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.Unit)) + return n +} +func (m *Get_AutoAck) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 2 + return n +} +func (m *Messages) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BatchId != 0 { + n += 1 + sovCanalProtocol(uint64(m.BatchId)) + } + if len(m.Messages) > 0 { + for _, b := range m.Messages { + l = len(b) + n += 1 + l + sovCanalProtocol(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Dump) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Journal) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.Position != 0 { + n += 1 + sovCanalProtocol(uint64(m.Position)) + } + if m.TimestampPresent != nil { + n += m.TimestampPresent.Size() + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Dump_Timestamp) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovCanalProtocol(uint64(m.Timestamp)) + return n +} +func (m *ClientRollback) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Destination) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + l = len(m.ClientId) + if l > 0 { + n += 1 + l + sovCanalProtocol(uint64(l)) + } + if m.BatchId != 0 { + n += 1 + sovCanalProtocol(uint64(m.BatchId)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovCanalProtocol(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozCanalProtocol(x uint64) (n int) { + return sovCanalProtocol(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Packet) 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 ErrIntOverflowCanalProtocol + } + 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: Packet: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Packet: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MagicNumber", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.MagicNumberPresent = &Packet_MagicNumber{v} + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.VersionPresent = &Packet_Version{v} + case 3: + 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 ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= PacketType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) + } + var v Compression + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Compression(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.CompressionPresent = &Packet_Compression{v} + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Body", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Body = append(m.Body[:0], dAtA[iNdEx:postIndex]...) + if m.Body == nil { + m.Body = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *HeartBeat) 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 ErrIntOverflowCanalProtocol + } + 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: HeartBeat: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HeartBeat: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SendTimestamp", wireType) + } + m.SendTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SendTimestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTimestamp", wireType) + } + m.StartTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartTimestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Handshake) 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 ErrIntOverflowCanalProtocol + } + 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: Handshake: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Handshake: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommunicationEncoding", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CommunicationEncodingPresent = &Handshake_CommunicationEncoding{string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Seeds", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Seeds = append(m.Seeds[:0], dAtA[iNdEx:postIndex]...) + if m.Seeds == nil { + m.Seeds = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SupportedCompressions", wireType) + } + m.SupportedCompressions = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SupportedCompressions |= Compression(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *ClientAuth) 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 ErrIntOverflowCanalProtocol + } + 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: ClientAuth: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientAuth: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Username", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Username = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Password", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Password = append(m.Password[:0], dAtA[iNdEx:postIndex]...) + if m.Password == nil { + m.Password = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NetReadTimeout", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetReadTimeoutPresent = &ClientAuth_NetReadTimeout{v} + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NetWriteTimeout", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.NetWriteTimeoutPresent = &ClientAuth_NetWriteTimeout{v} + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Filter", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Filter = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StartTimestamp", wireType) + } + m.StartTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StartTimestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Ack) 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 ErrIntOverflowCanalProtocol + } + 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: Ack: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Ack: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ErrorCode", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ErrorCodePresent = &Ack_ErrorCode{v} + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ErrorMessage", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ErrorMessage = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *ClientAck) 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 ErrIntOverflowCanalProtocol + } + 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: ClientAck: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientAck: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchId", wireType) + } + m.BatchId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BatchId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Sub) 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 ErrIntOverflowCanalProtocol + } + 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: Sub: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Sub: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Filter", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Filter = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Unsub) 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 ErrIntOverflowCanalProtocol + } + 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: Unsub: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Unsub: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Filter", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Filter = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Get) 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 ErrIntOverflowCanalProtocol + } + 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: Get: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Get: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FetchSize", wireType) + } + m.FetchSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.FetchSize |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeout", wireType) + } + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TimeoutPresent = &Get_Timeout{v} + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Unit", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.UnitPresent = &Get_Unit{v} + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AutoAck", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.AutoAckPresent = &Get_AutoAck{b} + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Messages) 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 ErrIntOverflowCanalProtocol + } + 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: Messages: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Messages: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchId", wireType) + } + m.BatchId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BatchId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Messages", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Messages = append(m.Messages, make([]byte, postIndex-iNdEx)) + copy(m.Messages[len(m.Messages)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *Dump) 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 ErrIntOverflowCanalProtocol + } + 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: Dump: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Dump: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Journal", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Journal = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Position", wireType) + } + m.Position = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Position |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var v int64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TimestampPresent = &Dump_Timestamp{v} + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 *ClientRollback) 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 ErrIntOverflowCanalProtocol + } + 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: ClientRollback: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientRollback: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Destination", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Destination = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClientId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + 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 ErrInvalidLengthCanalProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthCanalProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClientId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BatchId", wireType) + } + m.BatchId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowCanalProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BatchId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipCanalProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthCanalProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthCanalProtocol + } + 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 skipCanalProtocol(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, ErrIntOverflowCanalProtocol + } + 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, ErrIntOverflowCanalProtocol + } + 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, ErrIntOverflowCanalProtocol + } + 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, ErrInvalidLengthCanalProtocol + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupCanalProtocol + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthCanalProtocol + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthCanalProtocol = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowCanalProtocol = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupCanalProtocol = fmt.Errorf("proto: unexpected end of group") +) diff --git a/proto/canal/EntryProtocol.pb.go b/proto/canal/EntryProtocol.pb.go new file mode 100644 index 00000000000..8b13dafc8e3 --- /dev/null +++ b/proto/canal/EntryProtocol.pb.go @@ -0,0 +1,3927 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: EntryProtocol.proto + +package com_alibaba_otter_canal_protocol + +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 EntryType int32 + +const ( + EntryType_ENTRYTYPECOMPATIBLEPROTO2 EntryType = 0 + EntryType_TRANSACTIONBEGIN EntryType = 1 + EntryType_ROWDATA EntryType = 2 + EntryType_TRANSACTIONEND EntryType = 3 + //* 心跳类型,内部使用,外部暂不可见,可忽略 * + EntryType_HEARTBEAT EntryType = 4 + EntryType_GTIDLOG EntryType = 5 +) + +var EntryType_name = map[int32]string{ + 0: "ENTRYTYPECOMPATIBLEPROTO2", + 1: "TRANSACTIONBEGIN", + 2: "ROWDATA", + 3: "TRANSACTIONEND", + 4: "HEARTBEAT", + 5: "GTIDLOG", +} + +var EntryType_value = map[string]int32{ + "ENTRYTYPECOMPATIBLEPROTO2": 0, + "TRANSACTIONBEGIN": 1, + "ROWDATA": 2, + "TRANSACTIONEND": 3, + "HEARTBEAT": 4, + "GTIDLOG": 5, +} + +func (x EntryType) String() string { + return proto.EnumName(EntryType_name, int32(x)) +} + +func (EntryType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{0} +} + +//* 事件类型 * +type EventType int32 + +const ( + EventType_EVENTTYPECOMPATIBLEPROTO2 EventType = 0 + EventType_INSERT EventType = 1 + EventType_UPDATE EventType = 2 + EventType_DELETE EventType = 3 + EventType_CREATE EventType = 4 + EventType_ALTER EventType = 5 + EventType_ERASE EventType = 6 + EventType_QUERY EventType = 7 + EventType_TRUNCATE EventType = 8 + EventType_RENAME EventType = 9 + //*CREATE INDEX* + EventType_CINDEX EventType = 10 + EventType_DINDEX EventType = 11 + EventType_GTID EventType = 12 + //* XA * + EventType_XACOMMIT EventType = 13 + EventType_XAROLLBACK EventType = 14 + //* MASTER HEARTBEAT * + EventType_MHEARTBEAT EventType = 15 +) + +var EventType_name = map[int32]string{ + 0: "EVENTTYPECOMPATIBLEPROTO2", + 1: "INSERT", + 2: "UPDATE", + 3: "DELETE", + 4: "CREATE", + 5: "ALTER", + 6: "ERASE", + 7: "QUERY", + 8: "TRUNCATE", + 9: "RENAME", + 10: "CINDEX", + 11: "DINDEX", + 12: "GTID", + 13: "XACOMMIT", + 14: "XAROLLBACK", + 15: "MHEARTBEAT", +} + +var EventType_value = map[string]int32{ + "EVENTTYPECOMPATIBLEPROTO2": 0, + "INSERT": 1, + "UPDATE": 2, + "DELETE": 3, + "CREATE": 4, + "ALTER": 5, + "ERASE": 6, + "QUERY": 7, + "TRUNCATE": 8, + "RENAME": 9, + "CINDEX": 10, + "DINDEX": 11, + "GTID": 12, + "XACOMMIT": 13, + "XAROLLBACK": 14, + "MHEARTBEAT": 15, +} + +func (x EventType) String() string { + return proto.EnumName(EventType_name, int32(x)) +} + +func (EventType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{1} +} + +//*数据库类型* +type Type int32 + +const ( + Type_TYPECOMPATIBLEPROTO2 Type = 0 + Type_ORACLE Type = 1 + Type_MYSQL Type = 2 + Type_PGSQL Type = 3 +) + +var Type_name = map[int32]string{ + 0: "TYPECOMPATIBLEPROTO2", + 1: "ORACLE", + 2: "MYSQL", + 3: "PGSQL", +} + +var Type_value = map[string]int32{ + "TYPECOMPATIBLEPROTO2": 0, + "ORACLE": 1, + "MYSQL": 2, + "PGSQL": 3, +} + +func (x Type) String() string { + return proto.EnumName(Type_name, int32(x)) +} + +func (Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{2} +} + +//*************************************************************** +// message model +//如果要在Enum中新增类型,确保以前的类型的下标值不变. +//************************************************************** +type Entry struct { + //*协议头部信息* + Header *Header `protobuf:"bytes,1,opt,name=header,proto3" json:"header,omitempty"` + ///**打散后的事件类型**/ [default = ROWDATA] + // + // Types that are valid to be assigned to EntryTypePresent: + // *Entry_EntryType + EntryTypePresent isEntry_EntryTypePresent `protobuf_oneof:"entryType_present"` + //*传输的二进制数组* + StoreValue []byte `protobuf:"bytes,3,opt,name=storeValue,proto3" json:"storeValue,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Entry) Reset() { *m = Entry{} } +func (m *Entry) String() string { return proto.CompactTextString(m) } +func (*Entry) ProtoMessage() {} +func (*Entry) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{0} +} +func (m *Entry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Entry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Entry.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 *Entry) XXX_Merge(src proto.Message) { + xxx_messageInfo_Entry.Merge(m, src) +} +func (m *Entry) XXX_Size() int { + return m.Size() +} +func (m *Entry) XXX_DiscardUnknown() { + xxx_messageInfo_Entry.DiscardUnknown(m) +} + +var xxx_messageInfo_Entry proto.InternalMessageInfo + +type isEntry_EntryTypePresent interface { + isEntry_EntryTypePresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Entry_EntryType struct { + EntryType EntryType `protobuf:"varint,2,opt,name=entryType,proto3,enum=com.alibaba.otter.canal.protocol.EntryType,oneof" json:"entryType,omitempty"` +} + +func (*Entry_EntryType) isEntry_EntryTypePresent() {} + +func (m *Entry) GetEntryTypePresent() isEntry_EntryTypePresent { + if m != nil { + return m.EntryTypePresent + } + return nil +} + +func (m *Entry) GetHeader() *Header { + if m != nil { + return m.Header + } + return nil +} + +func (m *Entry) GetEntryType() EntryType { + if x, ok := m.GetEntryTypePresent().(*Entry_EntryType); ok { + return x.EntryType + } + return EntryType_ENTRYTYPECOMPATIBLEPROTO2 +} + +func (m *Entry) GetStoreValue() []byte { + if m != nil { + return m.StoreValue + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Entry) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Entry_EntryType)(nil), + } +} + +//*message Header* +type Header struct { + //[default = 1] + // + // Types that are valid to be assigned to VersionPresent: + // *Header_Version + VersionPresent isHeader_VersionPresent `protobuf_oneof:"version_present"` + //*binlog/redolog 文件名* + LogfileName string `protobuf:"bytes,2,opt,name=logfileName,proto3" json:"logfileName,omitempty"` + //*binlog/redolog 文件的偏移位置* + LogfileOffset int64 `protobuf:"varint,3,opt,name=logfileOffset,proto3" json:"logfileOffset,omitempty"` + //*服务端serverId* + ServerId int64 `protobuf:"varint,4,opt,name=serverId,proto3" json:"serverId,omitempty"` + //* 变更数据的编码 * + ServerenCode string `protobuf:"bytes,5,opt,name=serverenCode,proto3" json:"serverenCode,omitempty"` + //*变更数据的执行时间 * + ExecuteTime int64 `protobuf:"varint,6,opt,name=executeTime,proto3" json:"executeTime,omitempty"` + //[default = MYSQL] + // + // Types that are valid to be assigned to SourceTypePresent: + // *Header_SourceType + SourceTypePresent isHeader_SourceTypePresent `protobuf_oneof:"sourceType_present"` + //* 变更数据的schemaname* + SchemaName string `protobuf:"bytes,8,opt,name=schemaName,proto3" json:"schemaName,omitempty"` + //*变更数据的tablename* + TableName string `protobuf:"bytes,9,opt,name=tableName,proto3" json:"tableName,omitempty"` + //*每个event的长度* + EventLength int64 `protobuf:"varint,10,opt,name=eventLength,proto3" json:"eventLength,omitempty"` + // [default = UPDATE] + // + // Types that are valid to be assigned to EventTypePresent: + // *Header_EventType + EventTypePresent isHeader_EventTypePresent `protobuf_oneof:"eventType_present"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,12,rep,name=props,proto3" json:"props,omitempty"` + //*当前事务的gitd* + Gtid string `protobuf:"bytes,13,opt,name=gtid,proto3" json:"gtid,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Header) Reset() { *m = Header{} } +func (m *Header) String() string { return proto.CompactTextString(m) } +func (*Header) ProtoMessage() {} +func (*Header) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{1} +} +func (m *Header) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Header) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Header.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 *Header) XXX_Merge(src proto.Message) { + xxx_messageInfo_Header.Merge(m, src) +} +func (m *Header) XXX_Size() int { + return m.Size() +} +func (m *Header) XXX_DiscardUnknown() { + xxx_messageInfo_Header.DiscardUnknown(m) +} + +var xxx_messageInfo_Header proto.InternalMessageInfo + +type isHeader_VersionPresent interface { + isHeader_VersionPresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isHeader_SourceTypePresent interface { + isHeader_SourceTypePresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isHeader_EventTypePresent interface { + isHeader_EventTypePresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Header_Version struct { + Version int32 `protobuf:"varint,1,opt,name=version,proto3,oneof" json:"version,omitempty"` +} +type Header_SourceType struct { + SourceType Type `protobuf:"varint,7,opt,name=sourceType,proto3,enum=com.alibaba.otter.canal.protocol.Type,oneof" json:"sourceType,omitempty"` +} +type Header_EventType struct { + EventType EventType `protobuf:"varint,11,opt,name=eventType,proto3,enum=com.alibaba.otter.canal.protocol.EventType,oneof" json:"eventType,omitempty"` +} + +func (*Header_Version) isHeader_VersionPresent() {} +func (*Header_SourceType) isHeader_SourceTypePresent() {} +func (*Header_EventType) isHeader_EventTypePresent() {} + +func (m *Header) GetVersionPresent() isHeader_VersionPresent { + if m != nil { + return m.VersionPresent + } + return nil +} +func (m *Header) GetSourceTypePresent() isHeader_SourceTypePresent { + if m != nil { + return m.SourceTypePresent + } + return nil +} +func (m *Header) GetEventTypePresent() isHeader_EventTypePresent { + if m != nil { + return m.EventTypePresent + } + return nil +} + +func (m *Header) GetVersion() int32 { + if x, ok := m.GetVersionPresent().(*Header_Version); ok { + return x.Version + } + return 0 +} + +func (m *Header) GetLogfileName() string { + if m != nil { + return m.LogfileName + } + return "" +} + +func (m *Header) GetLogfileOffset() int64 { + if m != nil { + return m.LogfileOffset + } + return 0 +} + +func (m *Header) GetServerId() int64 { + if m != nil { + return m.ServerId + } + return 0 +} + +func (m *Header) GetServerenCode() string { + if m != nil { + return m.ServerenCode + } + return "" +} + +func (m *Header) GetExecuteTime() int64 { + if m != nil { + return m.ExecuteTime + } + return 0 +} + +func (m *Header) GetSourceType() Type { + if x, ok := m.GetSourceTypePresent().(*Header_SourceType); ok { + return x.SourceType + } + return Type_TYPECOMPATIBLEPROTO2 +} + +func (m *Header) GetSchemaName() string { + if m != nil { + return m.SchemaName + } + return "" +} + +func (m *Header) GetTableName() string { + if m != nil { + return m.TableName + } + return "" +} + +func (m *Header) GetEventLength() int64 { + if m != nil { + return m.EventLength + } + return 0 +} + +func (m *Header) GetEventType() EventType { + if x, ok := m.GetEventTypePresent().(*Header_EventType); ok { + return x.EventType + } + return EventType_EVENTTYPECOMPATIBLEPROTO2 +} + +func (m *Header) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +func (m *Header) GetGtid() string { + if m != nil { + return m.Gtid + } + return "" +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Header) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Header_Version)(nil), + (*Header_SourceType)(nil), + (*Header_EventType)(nil), + } +} + +//*每个字段的数据结构* +type Column struct { + //*字段下标* + Index int32 `protobuf:"varint,1,opt,name=index,proto3" json:"index,omitempty"` + //*字段java中类型* + SqlType int32 `protobuf:"varint,2,opt,name=sqlType,proto3" json:"sqlType,omitempty"` + //*字段名称(忽略大小写),在mysql中是没有的* + Name string `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` + //*是否是主键* + IsKey bool `protobuf:"varint,4,opt,name=isKey,proto3" json:"isKey,omitempty"` + //*如果EventType=UPDATE,用于标识这个字段值是否有修改* + Updated bool `protobuf:"varint,5,opt,name=updated,proto3" json:"updated,omitempty"` + //[default = false] + // + // Types that are valid to be assigned to IsNullPresent: + // *Column_IsNull + IsNullPresent isColumn_IsNullPresent `protobuf_oneof:"isNull_present"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,7,rep,name=props,proto3" json:"props,omitempty"` + //* 字段值,timestamp,Datetime是一个时间格式的文本 * + Value string `protobuf:"bytes,8,opt,name=value,proto3" json:"value,omitempty"` + //* 对应数据对象原始长度 * + Length int32 `protobuf:"varint,9,opt,name=length,proto3" json:"length,omitempty"` + //*字段mysql类型* + MysqlType string `protobuf:"bytes,10,opt,name=mysqlType,proto3" json:"mysqlType,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_237ce6ff565bd62b, []int{2} +} +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 + +type isColumn_IsNullPresent interface { + isColumn_IsNullPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type Column_IsNull struct { + IsNull bool `protobuf:"varint,6,opt,name=isNull,proto3,oneof" json:"isNull,omitempty"` +} + +func (*Column_IsNull) isColumn_IsNullPresent() {} + +func (m *Column) GetIsNullPresent() isColumn_IsNullPresent { + if m != nil { + return m.IsNullPresent + } + return nil +} + +func (m *Column) GetIndex() int32 { + if m != nil { + return m.Index + } + return 0 +} + +func (m *Column) GetSqlType() int32 { + if m != nil { + return m.SqlType + } + return 0 +} + +func (m *Column) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *Column) GetIsKey() bool { + if m != nil { + return m.IsKey + } + return false +} + +func (m *Column) GetUpdated() bool { + if m != nil { + return m.Updated + } + return false +} + +func (m *Column) GetIsNull() bool { + if x, ok := m.GetIsNullPresent().(*Column_IsNull); ok { + return x.IsNull + } + return false +} + +func (m *Column) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +func (m *Column) GetValue() string { + if m != nil { + return m.Value + } + return "" +} + +func (m *Column) GetLength() int32 { + if m != nil { + return m.Length + } + return 0 +} + +func (m *Column) GetMysqlType() string { + if m != nil { + return m.MysqlType + } + return "" +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Column) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Column_IsNull)(nil), + } +} + +type RowData struct { + //* 字段信息,增量数据(修改前,删除前) * + BeforeColumns []*Column `protobuf:"bytes,1,rep,name=beforeColumns,proto3" json:"beforeColumns,omitempty"` + //* 字段信息,增量数据(修改后,新增后) * + AfterColumns []*Column `protobuf:"bytes,2,rep,name=afterColumns,proto3" json:"afterColumns,omitempty"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,3,rep,name=props,proto3" json:"props,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RowData) Reset() { *m = RowData{} } +func (m *RowData) String() string { return proto.CompactTextString(m) } +func (*RowData) ProtoMessage() {} +func (*RowData) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{3} +} +func (m *RowData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RowData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_RowData.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 *RowData) XXX_Merge(src proto.Message) { + xxx_messageInfo_RowData.Merge(m, src) +} +func (m *RowData) XXX_Size() int { + return m.Size() +} +func (m *RowData) XXX_DiscardUnknown() { + xxx_messageInfo_RowData.DiscardUnknown(m) +} + +var xxx_messageInfo_RowData proto.InternalMessageInfo + +func (m *RowData) GetBeforeColumns() []*Column { + if m != nil { + return m.BeforeColumns + } + return nil +} + +func (m *RowData) GetAfterColumns() []*Column { + if m != nil { + return m.AfterColumns + } + return nil +} + +func (m *RowData) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +//*message row 每行变更数据的数据结构* +type RowChange struct { + //*tableId,由数据库产生* + TableId int64 `protobuf:"varint,1,opt,name=tableId,proto3" json:"tableId,omitempty"` + //[default = UPDATE] + // + // Types that are valid to be assigned to EventTypePresent: + // *RowChange_EventType + EventTypePresent isRowChange_EventTypePresent `protobuf_oneof:"eventType_present"` + // [default = false] + // + // Types that are valid to be assigned to IsDdlPresent: + // *RowChange_IsDdl + IsDdlPresent isRowChange_IsDdlPresent `protobuf_oneof:"isDdl_present"` + //* ddl/query的sql语句 * + Sql string `protobuf:"bytes,11,opt,name=sql,proto3" json:"sql,omitempty"` + //* 一次数据库变更可能存在多行 * + RowDatas []*RowData `protobuf:"bytes,12,rep,name=rowDatas,proto3" json:"rowDatas,omitempty"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,13,rep,name=props,proto3" json:"props,omitempty"` + //* ddl/query的schemaName,会存在跨库ddl,需要保留执行ddl的当前schemaName * + DdlSchemaName string `protobuf:"bytes,14,opt,name=ddlSchemaName,proto3" json:"ddlSchemaName,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RowChange) Reset() { *m = RowChange{} } +func (m *RowChange) String() string { return proto.CompactTextString(m) } +func (*RowChange) ProtoMessage() {} +func (*RowChange) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{4} +} +func (m *RowChange) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RowChange) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_RowChange.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 *RowChange) XXX_Merge(src proto.Message) { + xxx_messageInfo_RowChange.Merge(m, src) +} +func (m *RowChange) XXX_Size() int { + return m.Size() +} +func (m *RowChange) XXX_DiscardUnknown() { + xxx_messageInfo_RowChange.DiscardUnknown(m) +} + +var xxx_messageInfo_RowChange proto.InternalMessageInfo + +type isRowChange_EventTypePresent interface { + isRowChange_EventTypePresent() + MarshalTo([]byte) (int, error) + Size() int +} +type isRowChange_IsDdlPresent interface { + isRowChange_IsDdlPresent() + MarshalTo([]byte) (int, error) + Size() int +} + +type RowChange_EventType struct { + EventType EventType `protobuf:"varint,2,opt,name=eventType,proto3,enum=com.alibaba.otter.canal.protocol.EventType,oneof" json:"eventType,omitempty"` +} +type RowChange_IsDdl struct { + IsDdl bool `protobuf:"varint,10,opt,name=isDdl,proto3,oneof" json:"isDdl,omitempty"` +} + +func (*RowChange_EventType) isRowChange_EventTypePresent() {} +func (*RowChange_IsDdl) isRowChange_IsDdlPresent() {} + +func (m *RowChange) GetEventTypePresent() isRowChange_EventTypePresent { + if m != nil { + return m.EventTypePresent + } + return nil +} +func (m *RowChange) GetIsDdlPresent() isRowChange_IsDdlPresent { + if m != nil { + return m.IsDdlPresent + } + return nil +} + +func (m *RowChange) GetTableId() int64 { + if m != nil { + return m.TableId + } + return 0 +} + +func (m *RowChange) GetEventType() EventType { + if x, ok := m.GetEventTypePresent().(*RowChange_EventType); ok { + return x.EventType + } + return EventType_EVENTTYPECOMPATIBLEPROTO2 +} + +func (m *RowChange) GetIsDdl() bool { + if x, ok := m.GetIsDdlPresent().(*RowChange_IsDdl); ok { + return x.IsDdl + } + return false +} + +func (m *RowChange) GetSql() string { + if m != nil { + return m.Sql + } + return "" +} + +func (m *RowChange) GetRowDatas() []*RowData { + if m != nil { + return m.RowDatas + } + return nil +} + +func (m *RowChange) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +func (m *RowChange) GetDdlSchemaName() string { + if m != nil { + return m.DdlSchemaName + } + return "" +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*RowChange) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*RowChange_EventType)(nil), + (*RowChange_IsDdl)(nil), + } +} + +//*开始事务的一些信息* +type TransactionBegin struct { + //*已废弃,请使用header里的executeTime* + ExecuteTime int64 `protobuf:"varint,1,opt,name=executeTime,proto3" json:"executeTime,omitempty"` + //*已废弃,Begin里不提供事务id* + TransactionId string `protobuf:"bytes,2,opt,name=transactionId,proto3" json:"transactionId,omitempty"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,3,rep,name=props,proto3" json:"props,omitempty"` + //*执行的thread Id* + ThreadId int64 `protobuf:"varint,4,opt,name=threadId,proto3" json:"threadId,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TransactionBegin) Reset() { *m = TransactionBegin{} } +func (m *TransactionBegin) String() string { return proto.CompactTextString(m) } +func (*TransactionBegin) ProtoMessage() {} +func (*TransactionBegin) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{5} +} +func (m *TransactionBegin) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionBegin) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TransactionBegin.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 *TransactionBegin) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionBegin.Merge(m, src) +} +func (m *TransactionBegin) XXX_Size() int { + return m.Size() +} +func (m *TransactionBegin) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionBegin.DiscardUnknown(m) +} + +var xxx_messageInfo_TransactionBegin proto.InternalMessageInfo + +func (m *TransactionBegin) GetExecuteTime() int64 { + if m != nil { + return m.ExecuteTime + } + return 0 +} + +func (m *TransactionBegin) GetTransactionId() string { + if m != nil { + return m.TransactionId + } + return "" +} + +func (m *TransactionBegin) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +func (m *TransactionBegin) GetThreadId() int64 { + if m != nil { + return m.ThreadId + } + return 0 +} + +//*结束事务的一些信息* +type TransactionEnd struct { + //*已废弃,请使用header里的executeTime* + ExecuteTime int64 `protobuf:"varint,1,opt,name=executeTime,proto3" json:"executeTime,omitempty"` + //*事务号* + TransactionId string `protobuf:"bytes,2,opt,name=transactionId,proto3" json:"transactionId,omitempty"` + //*预留扩展* + Props []*Pair `protobuf:"bytes,3,rep,name=props,proto3" json:"props,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TransactionEnd) Reset() { *m = TransactionEnd{} } +func (m *TransactionEnd) String() string { return proto.CompactTextString(m) } +func (*TransactionEnd) ProtoMessage() {} +func (*TransactionEnd) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{6} +} +func (m *TransactionEnd) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionEnd) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TransactionEnd.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 *TransactionEnd) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionEnd.Merge(m, src) +} +func (m *TransactionEnd) XXX_Size() int { + return m.Size() +} +func (m *TransactionEnd) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionEnd.DiscardUnknown(m) +} + +var xxx_messageInfo_TransactionEnd proto.InternalMessageInfo + +func (m *TransactionEnd) GetExecuteTime() int64 { + if m != nil { + return m.ExecuteTime + } + return 0 +} + +func (m *TransactionEnd) GetTransactionId() string { + if m != nil { + return m.TransactionId + } + return "" +} + +func (m *TransactionEnd) GetProps() []*Pair { + if m != nil { + return m.Props + } + return nil +} + +//*预留扩展* +type Pair struct { + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Pair) Reset() { *m = Pair{} } +func (m *Pair) String() string { return proto.CompactTextString(m) } +func (*Pair) ProtoMessage() {} +func (*Pair) Descriptor() ([]byte, []int) { + return fileDescriptor_237ce6ff565bd62b, []int{7} +} +func (m *Pair) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Pair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Pair.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 *Pair) XXX_Merge(src proto.Message) { + xxx_messageInfo_Pair.Merge(m, src) +} +func (m *Pair) XXX_Size() int { + return m.Size() +} +func (m *Pair) XXX_DiscardUnknown() { + xxx_messageInfo_Pair.DiscardUnknown(m) +} + +var xxx_messageInfo_Pair proto.InternalMessageInfo + +func (m *Pair) GetKey() string { + if m != nil { + return m.Key + } + return "" +} + +func (m *Pair) GetValue() string { + if m != nil { + return m.Value + } + return "" +} + +func init() { + proto.RegisterEnum("com.alibaba.otter.canal.protocol.EntryType", EntryType_name, EntryType_value) + proto.RegisterEnum("com.alibaba.otter.canal.protocol.EventType", EventType_name, EventType_value) + proto.RegisterEnum("com.alibaba.otter.canal.protocol.Type", Type_name, Type_value) + proto.RegisterType((*Entry)(nil), "com.alibaba.otter.canal.protocol.Entry") + proto.RegisterType((*Header)(nil), "com.alibaba.otter.canal.protocol.Header") + proto.RegisterType((*Column)(nil), "com.alibaba.otter.canal.protocol.Column") + proto.RegisterType((*RowData)(nil), "com.alibaba.otter.canal.protocol.RowData") + proto.RegisterType((*RowChange)(nil), "com.alibaba.otter.canal.protocol.RowChange") + proto.RegisterType((*TransactionBegin)(nil), "com.alibaba.otter.canal.protocol.TransactionBegin") + proto.RegisterType((*TransactionEnd)(nil), "com.alibaba.otter.canal.protocol.TransactionEnd") + proto.RegisterType((*Pair)(nil), "com.alibaba.otter.canal.protocol.Pair") +} + +func init() { proto.RegisterFile("EntryProtocol.proto", fileDescriptor_237ce6ff565bd62b) } + +var fileDescriptor_237ce6ff565bd62b = []byte{ + // 1071 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x56, 0xcd, 0x6e, 0xdb, 0x46, + 0x10, 0x16, 0xf5, 0xcf, 0xd1, 0x8f, 0x37, 0x1b, 0x23, 0x60, 0x83, 0xd6, 0x10, 0x04, 0xa3, 0x70, + 0x5d, 0x40, 0x07, 0xf7, 0xda, 0x43, 0x48, 0x6a, 0x61, 0x09, 0x96, 0x28, 0x65, 0xbd, 0x4e, 0xed, + 0x53, 0x41, 0x8b, 0x6b, 0x5b, 0x28, 0x45, 0x2a, 0x24, 0xe5, 0xc4, 0x87, 0xbe, 0x46, 0xd1, 0xe7, + 0x28, 0xfa, 0x10, 0x05, 0x7a, 0xe9, 0xa5, 0xf7, 0xc2, 0x45, 0x8e, 0x7d, 0x87, 0x62, 0x97, 0x7f, + 0x52, 0xdb, 0x40, 0x8e, 0x7b, 0xc9, 0x6d, 0x66, 0xb4, 0xdf, 0xec, 0x37, 0x33, 0xdf, 0x2c, 0x05, + 0x4f, 0x89, 0x17, 0x05, 0x77, 0xd3, 0xc0, 0x8f, 0xfc, 0x99, 0xef, 0xf6, 0x96, 0xc2, 0xc0, 0x9d, + 0x99, 0xbf, 0xe8, 0xd9, 0xee, 0xfc, 0xd2, 0xbe, 0xb4, 0x7b, 0x7e, 0x14, 0xf1, 0xa0, 0x37, 0xb3, + 0x3d, 0x3b, 0xf9, 0x79, 0xe6, 0xbb, 0xdd, 0x5f, 0x15, 0xa8, 0x48, 0x24, 0x7e, 0x01, 0xd5, 0x1b, + 0x6e, 0x3b, 0x3c, 0xd0, 0x94, 0x8e, 0x72, 0xd0, 0x38, 0x3a, 0xe8, 0x6d, 0x03, 0xf7, 0x06, 0xf2, + 0x3c, 0x4d, 0x70, 0xf8, 0x04, 0x54, 0x2e, 0x52, 0xb1, 0xbb, 0x25, 0xd7, 0x8a, 0x1d, 0xe5, 0xa0, + 0x7d, 0xf4, 0xe5, 0xf6, 0x24, 0x24, 0x85, 0x0c, 0x0a, 0x34, 0xc7, 0xe3, 0x3d, 0x80, 0x30, 0xf2, + 0x03, 0xfe, 0xca, 0x76, 0x57, 0x5c, 0x2b, 0x75, 0x94, 0x83, 0x26, 0x5d, 0x8b, 0x18, 0x4f, 0xe1, + 0x49, 0x76, 0xf8, 0xdb, 0x65, 0xc0, 0x43, 0xee, 0x45, 0xdd, 0xdf, 0xcb, 0x50, 0x8d, 0x49, 0xe1, + 0xe7, 0x50, 0xbb, 0xe5, 0x41, 0x38, 0xf7, 0x3d, 0x59, 0x4f, 0x65, 0x50, 0xa0, 0x69, 0x00, 0x77, + 0xa0, 0xe1, 0xfa, 0xd7, 0x57, 0x73, 0x97, 0x5b, 0xf6, 0x22, 0xa6, 0xaa, 0xd2, 0xf5, 0x10, 0xde, + 0x87, 0x56, 0xe2, 0x4e, 0xae, 0xae, 0x42, 0x1e, 0x49, 0x02, 0x25, 0xba, 0x19, 0xc4, 0xcf, 0xa1, + 0x1e, 0xf2, 0xe0, 0x96, 0x07, 0x43, 0x47, 0x2b, 0xcb, 0x03, 0x99, 0x8f, 0xbb, 0xd0, 0x8c, 0x6d, + 0xee, 0x99, 0xbe, 0xc3, 0xb5, 0x8a, 0xbc, 0x64, 0x23, 0x26, 0x78, 0xf0, 0xb7, 0x7c, 0xb6, 0x8a, + 0x38, 0x9b, 0x2f, 0xb8, 0x56, 0x95, 0x29, 0xd6, 0x43, 0x78, 0x00, 0x10, 0xfa, 0xab, 0x60, 0xc6, + 0x65, 0x4f, 0x6b, 0xb2, 0xa7, 0x9f, 0x6f, 0xef, 0xa9, 0x6c, 0xa7, 0x42, 0xd7, 0xb0, 0xb2, 0x9f, + 0xb3, 0x1b, 0xbe, 0xb0, 0x65, 0xc9, 0x75, 0xc9, 0x66, 0x2d, 0x82, 0x3f, 0x05, 0x35, 0xb2, 0x2f, + 0x93, 0x8e, 0xa8, 0xf2, 0xe7, 0x3c, 0x20, 0x99, 0xde, 0x72, 0x2f, 0x1a, 0x71, 0xef, 0x3a, 0xba, + 0xd1, 0x20, 0x61, 0x9a, 0x87, 0xe4, 0xf0, 0x85, 0x2b, 0x89, 0x36, 0x1e, 0x3c, 0xfc, 0x14, 0x32, + 0x28, 0xd2, 0x1c, 0x8f, 0xbf, 0x86, 0xca, 0x32, 0xf0, 0x97, 0xa1, 0xd6, 0xec, 0x94, 0x0e, 0x1a, + 0x0f, 0xa9, 0x78, 0x6a, 0xcf, 0x03, 0x1a, 0x83, 0x30, 0x86, 0xf2, 0x75, 0x34, 0x77, 0xb4, 0x96, + 0xac, 0x42, 0xda, 0xc6, 0x13, 0xd8, 0x49, 0xa6, 0x9f, 0x8a, 0xc5, 0xd8, 0x05, 0x9c, 0xf7, 0x27, + 0x8b, 0x0a, 0x5d, 0xa5, 0x3c, 0x32, 0x5d, 0xfd, 0x54, 0x84, 0xaa, 0xe9, 0xbb, 0xab, 0x85, 0x87, + 0x77, 0xa1, 0x32, 0xf7, 0x1c, 0xfe, 0x36, 0x56, 0x15, 0x8d, 0x1d, 0xac, 0x41, 0x2d, 0x7c, 0xed, + 0x66, 0xc2, 0xaf, 0xd0, 0xd4, 0x15, 0x64, 0x3c, 0xd1, 0xd2, 0x52, 0x4c, 0x46, 0xd8, 0x32, 0x47, + 0x78, 0xc2, 0xef, 0xa4, 0x68, 0xea, 0x34, 0x76, 0x44, 0x8e, 0xd5, 0xd2, 0xb1, 0x23, 0xee, 0x48, + 0xb1, 0xd4, 0x69, 0xea, 0x62, 0x0d, 0xaa, 0xf3, 0xd0, 0x5a, 0xb9, 0xae, 0x94, 0x48, 0x7d, 0x50, + 0xa0, 0x89, 0x9f, 0x37, 0xaa, 0xf6, 0x98, 0x46, 0xed, 0x42, 0xe5, 0x56, 0xae, 0x57, 0x2c, 0x87, + 0xd8, 0xc1, 0xcf, 0xa0, 0xea, 0xc6, 0x63, 0x56, 0x65, 0x29, 0x89, 0x27, 0x14, 0xb2, 0xb8, 0x4b, + 0xab, 0x84, 0x58, 0x21, 0x59, 0xc0, 0x40, 0xd0, 0x8e, 0x39, 0x65, 0x4d, 0x7b, 0xa7, 0x40, 0x8d, + 0xfa, 0x6f, 0xfa, 0x76, 0x64, 0x63, 0x0b, 0x5a, 0x97, 0xfc, 0xca, 0x0f, 0x78, 0xdc, 0xc5, 0x50, + 0x53, 0x24, 0xdf, 0x07, 0xbc, 0x31, 0x31, 0x80, 0x6e, 0xc2, 0xf1, 0x08, 0x9a, 0xf6, 0x55, 0xc4, + 0x83, 0x34, 0x5d, 0xf1, 0x03, 0xd3, 0x6d, 0xa0, 0xf3, 0x2e, 0x96, 0x1e, 0xd1, 0xc5, 0xee, 0x5f, + 0x45, 0x50, 0xa9, 0xff, 0xc6, 0xbc, 0xb1, 0xbd, 0x6b, 0x2e, 0xa6, 0x28, 0xd7, 0x66, 0xe8, 0x48, + 0x85, 0x94, 0x68, 0xea, 0x6e, 0x6e, 0x48, 0xf1, 0xc3, 0x37, 0xa4, 0xb0, 0xbe, 0x21, 0xcf, 0x84, + 0x84, 0xfa, 0x8e, 0x2b, 0x07, 0x51, 0x1f, 0x28, 0x34, 0x76, 0x31, 0x82, 0x52, 0xf8, 0xda, 0x95, + 0x0b, 0xa8, 0x52, 0x61, 0x62, 0x02, 0xf5, 0x20, 0x9e, 0x42, 0xba, 0x4e, 0x5f, 0x6c, 0xbf, 0x35, + 0x99, 0x1b, 0xcd, 0xa0, 0x79, 0x8f, 0x5a, 0x8f, 0x51, 0xda, 0x3e, 0xb4, 0x1c, 0xc7, 0x3d, 0xcd, + 0x1f, 0xa0, 0xb6, 0x24, 0xb8, 0x19, 0xfc, 0xcf, 0xdd, 0x33, 0x76, 0xa0, 0x25, 0x4b, 0xcb, 0x74, + 0xf5, 0xb3, 0x02, 0x88, 0x05, 0xb6, 0x17, 0xda, 0xb3, 0x68, 0xee, 0x7b, 0x06, 0xbf, 0x9e, 0x7b, + 0xff, 0x7c, 0x4a, 0x95, 0x7f, 0x3f, 0xa5, 0xfb, 0xd0, 0x8a, 0x72, 0xd4, 0xd0, 0x49, 0x9e, 0xfd, + 0xcd, 0xe0, 0xff, 0x93, 0x82, 0xf8, 0x20, 0x44, 0x37, 0x01, 0xb7, 0x9d, 0xfc, 0x83, 0x90, 0xfa, + 0xdd, 0x1f, 0x14, 0x68, 0xaf, 0xd1, 0x26, 0x9e, 0xf3, 0x71, 0x90, 0xee, 0xf6, 0xa0, 0x2c, 0x5c, + 0x21, 0x9d, 0xef, 0xf8, 0x9d, 0x64, 0xa1, 0x52, 0x61, 0xe6, 0xef, 0x43, 0x71, 0xed, 0x7d, 0x38, + 0xfc, 0x1e, 0xd4, 0xec, 0x9b, 0x8d, 0x3f, 0x83, 0x4f, 0x88, 0xc5, 0xe8, 0x05, 0xbb, 0x98, 0x12, + 0x73, 0x32, 0x9e, 0xea, 0x6c, 0x68, 0x8c, 0xc8, 0x94, 0x4e, 0xd8, 0xe4, 0x08, 0x15, 0xf0, 0x2e, + 0x20, 0x46, 0x75, 0xeb, 0x54, 0x37, 0xd9, 0x70, 0x62, 0x19, 0xe4, 0x78, 0x68, 0x21, 0x05, 0x37, + 0xa0, 0x46, 0x27, 0xdf, 0xf4, 0x75, 0xa6, 0xa3, 0x22, 0xc6, 0xd0, 0x5e, 0x3b, 0x42, 0xac, 0x3e, + 0x2a, 0xe1, 0x16, 0xa8, 0x03, 0xa2, 0x53, 0x66, 0x10, 0x9d, 0xa1, 0xb2, 0x38, 0x7f, 0xcc, 0x86, + 0xfd, 0xd1, 0xe4, 0x18, 0x55, 0x0e, 0xdf, 0x29, 0xa0, 0x66, 0x4b, 0x21, 0xef, 0x7f, 0x45, 0x2c, + 0xf6, 0x9e, 0xfb, 0x01, 0xaa, 0x43, 0xeb, 0x94, 0x50, 0x86, 0x14, 0x61, 0x9f, 0x4d, 0xfb, 0x3a, + 0x23, 0xa8, 0x28, 0xec, 0x3e, 0x19, 0x11, 0x46, 0x50, 0x49, 0xd8, 0x26, 0x25, 0x22, 0x5e, 0xc6, + 0x2a, 0x54, 0xf4, 0x11, 0x23, 0x14, 0x55, 0x84, 0x49, 0xa8, 0x7e, 0x4a, 0x50, 0x55, 0x98, 0x2f, + 0xcf, 0x08, 0xbd, 0x40, 0x35, 0xdc, 0x84, 0x3a, 0xa3, 0x67, 0x96, 0x29, 0x8e, 0xd7, 0x05, 0x94, + 0x12, 0x4b, 0x1f, 0x13, 0xa4, 0xca, 0x34, 0x43, 0xab, 0x4f, 0xce, 0x11, 0xc8, 0xf4, 0xb1, 0xdd, + 0xc0, 0x75, 0x28, 0x0b, 0xf2, 0xa8, 0x29, 0xb0, 0xe7, 0xba, 0x39, 0x19, 0x8f, 0x87, 0x0c, 0xb5, + 0x70, 0x1b, 0xe0, 0x5c, 0xa7, 0x93, 0xd1, 0xc8, 0xd0, 0xcd, 0x13, 0xd4, 0x16, 0xfe, 0x38, 0x2f, + 0x7a, 0xe7, 0xd0, 0x80, 0xb2, 0xac, 0x50, 0x83, 0xdd, 0xf7, 0x17, 0x37, 0xa1, 0xba, 0x39, 0x22, + 0x48, 0x11, 0x14, 0xc7, 0x17, 0xa7, 0x2f, 0x47, 0xa8, 0x28, 0xcc, 0xe9, 0xb1, 0x30, 0x4b, 0xc6, + 0x8b, 0x5f, 0xee, 0xf7, 0x94, 0xdf, 0xee, 0xf7, 0x94, 0x3f, 0xee, 0xf7, 0x94, 0x1f, 0xff, 0xdc, + 0x2b, 0xc0, 0xd6, 0x7f, 0x84, 0x06, 0x98, 0xc2, 0x97, 0x13, 0x1e, 0x28, 0x97, 0x55, 0x19, 0xff, + 0xea, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc5, 0x2a, 0xd4, 0xa2, 0x5f, 0x0a, 0x00, 0x00, +} + +func (m *Entry) 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 *Entry) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Entry) 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.StoreValue) > 0 { + i -= len(m.StoreValue) + copy(dAtA[i:], m.StoreValue) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.StoreValue))) + i-- + dAtA[i] = 0x1a + } + if m.EntryTypePresent != nil { + { + size := m.EntryTypePresent.Size() + i -= size + if _, err := m.EntryTypePresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.Header != nil { + { + size, err := m.Header.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Entry_EntryType) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Entry_EntryType) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.EntryType)) + i-- + dAtA[i] = 0x10 + return len(dAtA) - i, nil +} +func (m *Header) 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 *Header) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Header) 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.Gtid) > 0 { + i -= len(m.Gtid) + copy(dAtA[i:], m.Gtid) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Gtid))) + i-- + dAtA[i] = 0x6a + } + if len(m.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + } + if m.EventTypePresent != nil { + { + size := m.EventTypePresent.Size() + i -= size + if _, err := m.EventTypePresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.EventLength != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.EventLength)) + i-- + dAtA[i] = 0x50 + } + if len(m.TableName) > 0 { + i -= len(m.TableName) + copy(dAtA[i:], m.TableName) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.TableName))) + i-- + dAtA[i] = 0x4a + } + if len(m.SchemaName) > 0 { + i -= len(m.SchemaName) + copy(dAtA[i:], m.SchemaName) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.SchemaName))) + i-- + dAtA[i] = 0x42 + } + if m.SourceTypePresent != nil { + { + size := m.SourceTypePresent.Size() + i -= size + if _, err := m.SourceTypePresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.ExecuteTime != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.ExecuteTime)) + i-- + dAtA[i] = 0x30 + } + if len(m.ServerenCode) > 0 { + i -= len(m.ServerenCode) + copy(dAtA[i:], m.ServerenCode) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.ServerenCode))) + i-- + dAtA[i] = 0x2a + } + if m.ServerId != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.ServerId)) + i-- + dAtA[i] = 0x20 + } + if m.LogfileOffset != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.LogfileOffset)) + i-- + dAtA[i] = 0x18 + } + if len(m.LogfileName) > 0 { + i -= len(m.LogfileName) + copy(dAtA[i:], m.LogfileName) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.LogfileName))) + i-- + dAtA[i] = 0x12 + } + if m.VersionPresent != nil { + { + size := m.VersionPresent.Size() + i -= size + if _, err := m.VersionPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Header_Version) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Header_Version) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *Header_SourceType) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Header_SourceType) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.SourceType)) + i-- + dAtA[i] = 0x38 + return len(dAtA) - i, nil +} +func (m *Header_EventType) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Header_EventType) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.EventType)) + i-- + dAtA[i] = 0x58 + 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.MysqlType) > 0 { + i -= len(m.MysqlType) + copy(dAtA[i:], m.MysqlType) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.MysqlType))) + i-- + dAtA[i] = 0x52 + } + if m.Length != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.Length)) + i-- + dAtA[i] = 0x48 + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x42 + } + if len(m.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + } + } + if m.IsNullPresent != nil { + { + size := m.IsNullPresent.Size() + i -= size + if _, err := m.IsNullPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.Updated { + i-- + if m.Updated { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } + if m.IsKey { + i-- + if m.IsKey { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x20 + } + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0x1a + } + if m.SqlType != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.SqlType)) + i-- + dAtA[i] = 0x10 + } + if m.Index != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.Index)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Column_IsNull) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Column_IsNull) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i-- + if m.IsNull { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + return len(dAtA) - i, nil +} +func (m *RowData) 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 *RowData) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowData) 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.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.AfterColumns) > 0 { + for iNdEx := len(m.AfterColumns) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.AfterColumns[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.BeforeColumns) > 0 { + for iNdEx := len(m.BeforeColumns) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.BeforeColumns[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func (m *RowChange) 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 *RowChange) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowChange) 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.DdlSchemaName) > 0 { + i -= len(m.DdlSchemaName) + copy(dAtA[i:], m.DdlSchemaName) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.DdlSchemaName))) + i-- + dAtA[i] = 0x72 + } + if len(m.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x6a + } + } + if len(m.RowDatas) > 0 { + for iNdEx := len(m.RowDatas) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.RowDatas[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x62 + } + } + if len(m.Sql) > 0 { + i -= len(m.Sql) + copy(dAtA[i:], m.Sql) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Sql))) + i-- + dAtA[i] = 0x5a + } + if m.IsDdlPresent != nil { + { + size := m.IsDdlPresent.Size() + i -= size + if _, err := m.IsDdlPresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.EventTypePresent != nil { + { + size := m.EventTypePresent.Size() + i -= size + if _, err := m.EventTypePresent.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.TableId != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.TableId)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *RowChange_EventType) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowChange_EventType) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.EventType)) + i-- + dAtA[i] = 0x10 + return len(dAtA) - i, nil +} +func (m *RowChange_IsDdl) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *RowChange_IsDdl) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i-- + if m.IsDdl { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + return len(dAtA) - i, nil +} +func (m *TransactionBegin) 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 *TransactionBegin) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TransactionBegin) 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.ThreadId != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.ThreadId)) + i-- + dAtA[i] = 0x20 + } + if len(m.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.TransactionId) > 0 { + i -= len(m.TransactionId) + copy(dAtA[i:], m.TransactionId) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.TransactionId))) + i-- + dAtA[i] = 0x12 + } + if m.ExecuteTime != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.ExecuteTime)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *TransactionEnd) 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 *TransactionEnd) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TransactionEnd) 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.Props) > 0 { + for iNdEx := len(m.Props) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Props[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEntryProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.TransactionId) > 0 { + i -= len(m.TransactionId) + copy(dAtA[i:], m.TransactionId) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.TransactionId))) + i-- + dAtA[i] = 0x12 + } + if m.ExecuteTime != 0 { + i = encodeVarintEntryProtocol(dAtA, i, uint64(m.ExecuteTime)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Pair) 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 *Pair) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Pair) 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 = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x12 + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintEntryProtocol(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintEntryProtocol(dAtA []byte, offset int, v uint64) int { + offset -= sovEntryProtocol(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Entry) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Header != nil { + l = m.Header.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.EntryTypePresent != nil { + n += m.EntryTypePresent.Size() + } + l = len(m.StoreValue) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Entry_EntryType) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovEntryProtocol(uint64(m.EntryType)) + return n +} +func (m *Header) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.VersionPresent != nil { + n += m.VersionPresent.Size() + } + l = len(m.LogfileName) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.LogfileOffset != 0 { + n += 1 + sovEntryProtocol(uint64(m.LogfileOffset)) + } + if m.ServerId != 0 { + n += 1 + sovEntryProtocol(uint64(m.ServerId)) + } + l = len(m.ServerenCode) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.ExecuteTime != 0 { + n += 1 + sovEntryProtocol(uint64(m.ExecuteTime)) + } + if m.SourceTypePresent != nil { + n += m.SourceTypePresent.Size() + } + l = len(m.SchemaName) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + l = len(m.TableName) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.EventLength != 0 { + n += 1 + sovEntryProtocol(uint64(m.EventLength)) + } + if m.EventTypePresent != nil { + n += m.EventTypePresent.Size() + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + l = len(m.Gtid) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Header_Version) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovEntryProtocol(uint64(m.Version)) + return n +} +func (m *Header_SourceType) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovEntryProtocol(uint64(m.SourceType)) + return n +} +func (m *Header_EventType) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovEntryProtocol(uint64(m.EventType)) + return n +} +func (m *Column) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Index != 0 { + n += 1 + sovEntryProtocol(uint64(m.Index)) + } + if m.SqlType != 0 { + n += 1 + sovEntryProtocol(uint64(m.SqlType)) + } + l = len(m.Name) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.IsKey { + n += 2 + } + if m.Updated { + n += 2 + } + if m.IsNullPresent != nil { + n += m.IsNullPresent.Size() + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.Length != 0 { + n += 1 + sovEntryProtocol(uint64(m.Length)) + } + l = len(m.MysqlType) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Column_IsNull) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 2 + return n +} +func (m *RowData) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.BeforeColumns) > 0 { + for _, e := range m.BeforeColumns { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if len(m.AfterColumns) > 0 { + for _, e := range m.AfterColumns { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *RowChange) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.TableId != 0 { + n += 1 + sovEntryProtocol(uint64(m.TableId)) + } + if m.EventTypePresent != nil { + n += m.EventTypePresent.Size() + } + if m.IsDdlPresent != nil { + n += m.IsDdlPresent.Size() + } + l = len(m.Sql) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if len(m.RowDatas) > 0 { + for _, e := range m.RowDatas { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + l = len(m.DdlSchemaName) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *RowChange_EventType) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovEntryProtocol(uint64(m.EventType)) + return n +} +func (m *RowChange_IsDdl) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 2 + return n +} +func (m *TransactionBegin) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecuteTime != 0 { + n += 1 + sovEntryProtocol(uint64(m.ExecuteTime)) + } + l = len(m.TransactionId) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if m.ThreadId != 0 { + n += 1 + sovEntryProtocol(uint64(m.ThreadId)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *TransactionEnd) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ExecuteTime != 0 { + n += 1 + sovEntryProtocol(uint64(m.ExecuteTime)) + } + l = len(m.TransactionId) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if len(m.Props) > 0 { + for _, e := range m.Props { + l = e.Size() + n += 1 + l + sovEntryProtocol(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Pair) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovEntryProtocol(uint64(l)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovEntryProtocol(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozEntryProtocol(x uint64) (n int) { + return sovEntryProtocol(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Entry) 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 ErrIntOverflowEntryProtocol + } + 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: Entry: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Entry: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Header == nil { + m.Header = &Header{} + } + if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EntryType", wireType) + } + var v EntryType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= EntryType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EntryTypePresent = &Entry_EntryType{v} + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreValue", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StoreValue = append(m.StoreValue[:0], dAtA[iNdEx:postIndex]...) + if m.StoreValue == nil { + m.StoreValue = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *Header) 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 ErrIntOverflowEntryProtocol + } + 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: Header: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Header: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.VersionPresent = &Header_Version{v} + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogfileName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogfileName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LogfileOffset", wireType) + } + m.LogfileOffset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LogfileOffset |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ServerId", wireType) + } + m.ServerId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ServerId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ServerenCode", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ServerenCode = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecuteTime", wireType) + } + m.ExecuteTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExecuteTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SourceType", wireType) + } + var v Type + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= Type(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SourceTypePresent = &Header_SourceType{v} + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SchemaName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SchemaName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TableName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TableName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EventLength", wireType) + } + m.EventLength = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EventLength |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 11: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EventType", wireType) + } + var v EventType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= EventType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EventTypePresent = &Header_EventType{v} + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Gtid", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Gtid = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 ErrIntOverflowEntryProtocol + } + 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 != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SqlType", wireType) + } + m.SqlType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SqlType |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + 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 ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsKey", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IsKey = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Updated", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Updated = bool(v != 0) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsNull", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.IsNullPresent = &Column_IsNull{b} + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Length", wireType) + } + m.Length = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Length |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field MysqlType", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.MysqlType = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *RowData) 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 ErrIntOverflowEntryProtocol + } + 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: RowData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RowData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BeforeColumns", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BeforeColumns = append(m.BeforeColumns, &Column{}) + if err := m.BeforeColumns[len(m.BeforeColumns)-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 AfterColumns", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AfterColumns = append(m.AfterColumns, &Column{}) + if err := m.AfterColumns[len(m.AfterColumns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *RowChange) 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 ErrIntOverflowEntryProtocol + } + 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: RowChange: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RowChange: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableId", wireType) + } + m.TableId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TableId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EventType", wireType) + } + var v EventType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= EventType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.EventTypePresent = &RowChange_EventType{v} + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IsDdl", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + b := bool(v != 0) + m.IsDdlPresent = &RowChange_IsDdl{b} + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Sql", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Sql = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RowDatas", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RowDatas = append(m.RowDatas, &RowData{}) + if err := m.RowDatas[len(m.RowDatas)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DdlSchemaName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DdlSchemaName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *TransactionBegin) 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 ErrIntOverflowEntryProtocol + } + 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: TransactionBegin: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TransactionBegin: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecuteTime", wireType) + } + m.ExecuteTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExecuteTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ThreadId", wireType) + } + m.ThreadId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ThreadId |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *TransactionEnd) 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 ErrIntOverflowEntryProtocol + } + 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: TransactionEnd: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TransactionEnd: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExecuteTime", wireType) + } + m.ExecuteTime = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExecuteTime |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TransactionId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TransactionId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Props", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Props = append(m.Props, &Pair{}) + if err := m.Props[len(m.Props)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 *Pair) 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 ErrIntOverflowEntryProtocol + } + 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: Pair: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Pair: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEntryProtocol + } + 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 ErrInvalidLengthEntryProtocol + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEntryProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEntryProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthEntryProtocol + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthEntryProtocol + } + 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 skipEntryProtocol(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, ErrIntOverflowEntryProtocol + } + 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, ErrIntOverflowEntryProtocol + } + 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, ErrIntOverflowEntryProtocol + } + 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, ErrInvalidLengthEntryProtocol + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupEntryProtocol + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthEntryProtocol + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthEntryProtocol = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowEntryProtocol = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupEntryProtocol = fmt.Errorf("proto: unexpected end of group") +) diff --git a/proto/generate-proto.sh b/proto/generate-proto.sh new file mode 100755 index 00000000000..a7357a8fe1f --- /dev/null +++ b/proto/generate-proto.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +echo "generate canal protocol code..." + +[ ! -d ./canal ] && mkdir ./canal + +protoc --gofast_out=./canal EntryProtocol.proto +protoc --gofast_out=./canal CanalProtocol.proto diff --git a/scripts/check-copyright.sh b/scripts/check-copyright.sh index b00f189eaac..f58e3a37c5e 100755 --- a/scripts/check-copyright.sh +++ b/scripts/check-copyright.sh @@ -1,6 +1,6 @@ copyright="// Copyright $(date '+%Y') PingCAP, Inc." -result=$(find ./ -name "*.go" ! -path ".//vendor/*" | while read file_path; do +result=$(find ./ -name "*.go" ! -path ".//vendor/*" | grep -v '.pb.go' | while read file_path; do head=`cat "${file_path}" | head -n 1` if [ "$head" != "$copyright" ];then echo "${file_path}"