diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index b7fbb6901a7..e1aeaa6a760 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -5,9 +5,8 @@ package binlogdata import ( fmt "fmt" - math "math" - proto "github.com/golang/protobuf/proto" + math "math" query "vitess.io/vitess/go/vt/proto/query" topodata "vitess.io/vitess/go/vt/proto/topodata" vtrpc "vitess.io/vitess/go/vt/proto/vtrpc" @@ -87,6 +86,7 @@ const ( VEventType_VGTID VEventType = 15 VEventType_JOURNAL VEventType = 16 VEventType_VERSION VEventType = 17 + VEventType_LASTPK VEventType = 18 ) var VEventType_name = map[int32]string{ @@ -108,6 +108,7 @@ var VEventType_name = map[int32]string{ 15: "VGTID", 16: "JOURNAL", 17: "VERSION", + 18: "LASTPK", } var VEventType_value = map[string]int32{ @@ -129,6 +130,7 @@ var VEventType_value = map[string]int32{ "VGTID": 15, "JOURNAL": 16, "VERSION": 17, + "LASTPK": 18, } func (x VEventType) String() string { @@ -1001,12 +1003,13 @@ func (m *FieldEvent) GetFields() []*query.Field { // of a shard. It's also used in a Journal to indicate the // list of targets and shard positions to migrate to. type ShardGtid struct { - Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` - Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` - Gtid string `protobuf:"bytes,3,opt,name=gtid,proto3" json:"gtid,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Keyspace string `protobuf:"bytes,1,opt,name=keyspace,proto3" json:"keyspace,omitempty"` + Shard string `protobuf:"bytes,2,opt,name=shard,proto3" json:"shard,omitempty"` + Gtid string `protobuf:"bytes,3,opt,name=gtid,proto3" json:"gtid,omitempty"` + TablePKs []*TableLastPK `protobuf:"bytes,4,rep,name=table_p_ks,json=tablePKs,proto3" json:"table_p_ks,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ShardGtid) Reset() { *m = ShardGtid{} } @@ -1055,6 +1058,13 @@ func (m *ShardGtid) GetGtid() string { return "" } +func (m *ShardGtid) GetTablePKs() []*TableLastPK { + if m != nil { + return m.TablePKs + } + return nil +} + // A VGtid is a list of ShardGtids. type VGtid struct { ShardGtids []*ShardGtid `protobuf:"bytes,1,rep,name=shard_gtids,json=shardGtids,proto3" json:"shard_gtids,omitempty"` @@ -1278,10 +1288,12 @@ type VEvent struct { Dml string `protobuf:"bytes,9,opt,name=dml,proto3" json:"dml,omitempty"` // CurrentTime specifies the current time when the message was sent. // This can be used to compenssate for clock skew. - CurrentTime int64 `protobuf:"varint,20,opt,name=current_time,json=currentTime,proto3" json:"current_time,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + CurrentTime int64 `protobuf:"varint,20,opt,name=current_time,json=currentTime,proto3" json:"current_time,omitempty"` + // LastPK is the last PK for a table + LastPKEvent *LastPKEvent `protobuf:"bytes,21,opt,name=last_p_k_event,json=lastPKEvent,proto3" json:"last_p_k_event,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *VEvent) Reset() { *m = VEvent{} } @@ -1379,6 +1391,13 @@ func (m *VEvent) GetCurrentTime() int64 { return 0 } +func (m *VEvent) GetLastPKEvent() *LastPKEvent { + if m != nil { + return m.LastPKEvent + } + return nil +} + type MinimalTable struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` Fields []*query.Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields,omitempty"` @@ -1480,6 +1499,7 @@ type VStreamRequest struct { Target *query.Target `protobuf:"bytes,3,opt,name=target,proto3" json:"target,omitempty"` Position string `protobuf:"bytes,4,opt,name=position,proto3" json:"position,omitempty"` Filter *Filter `protobuf:"bytes,5,opt,name=filter,proto3" json:"filter,omitempty"` + TableLastPKs []*TableLastPK `protobuf:"bytes,6,rep,name=table_last_p_ks,json=tableLastPKs,proto3" json:"table_last_p_ks,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -1545,6 +1565,13 @@ func (m *VStreamRequest) GetFilter() *Filter { return nil } +func (m *VStreamRequest) GetTableLastPKs() []*TableLastPK { + if m != nil { + return m.TableLastPKs + } + return nil +} + // VStreamResponse is the response from VStreamer type VStreamResponse struct { Events []*VEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` @@ -1729,6 +1756,100 @@ func (m *VStreamRowsResponse) GetLastpk() *query.Row { return nil } +type LastPKEvent struct { + TableLastPK *TableLastPK `protobuf:"bytes,1,opt,name=table_last_p_k,json=tableLastPK,proto3" json:"table_last_p_k,omitempty"` + Completed bool `protobuf:"varint,2,opt,name=completed,proto3" json:"completed,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LastPKEvent) Reset() { *m = LastPKEvent{} } +func (m *LastPKEvent) String() string { return proto.CompactTextString(m) } +func (*LastPKEvent) ProtoMessage() {} +func (*LastPKEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_5fd02bcb2e350dad, []int{23} +} + +func (m *LastPKEvent) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_LastPKEvent.Unmarshal(m, b) +} +func (m *LastPKEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_LastPKEvent.Marshal(b, m, deterministic) +} +func (m *LastPKEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_LastPKEvent.Merge(m, src) +} +func (m *LastPKEvent) XXX_Size() int { + return xxx_messageInfo_LastPKEvent.Size(m) +} +func (m *LastPKEvent) XXX_DiscardUnknown() { + xxx_messageInfo_LastPKEvent.DiscardUnknown(m) +} + +var xxx_messageInfo_LastPKEvent proto.InternalMessageInfo + +func (m *LastPKEvent) GetTableLastPK() *TableLastPK { + if m != nil { + return m.TableLastPK + } + return nil +} + +func (m *LastPKEvent) GetCompleted() bool { + if m != nil { + return m.Completed + } + return false +} + +type TableLastPK struct { + TableName string `protobuf:"bytes,1,opt,name=table_name,json=tableName,proto3" json:"table_name,omitempty"` + Lastpk *query.QueryResult `protobuf:"bytes,3,opt,name=lastpk,proto3" json:"lastpk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TableLastPK) Reset() { *m = TableLastPK{} } +func (m *TableLastPK) String() string { return proto.CompactTextString(m) } +func (*TableLastPK) ProtoMessage() {} +func (*TableLastPK) Descriptor() ([]byte, []int) { + return fileDescriptor_5fd02bcb2e350dad, []int{24} +} + +func (m *TableLastPK) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_TableLastPK.Unmarshal(m, b) +} +func (m *TableLastPK) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_TableLastPK.Marshal(b, m, deterministic) +} +func (m *TableLastPK) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableLastPK.Merge(m, src) +} +func (m *TableLastPK) XXX_Size() int { + return xxx_messageInfo_TableLastPK.Size(m) +} +func (m *TableLastPK) XXX_DiscardUnknown() { + xxx_messageInfo_TableLastPK.DiscardUnknown(m) +} + +var xxx_messageInfo_TableLastPK proto.InternalMessageInfo + +func (m *TableLastPK) GetTableName() string { + if m != nil { + return m.TableName + } + return "" +} + +func (m *TableLastPK) GetLastpk() *query.QueryResult { + if m != nil { + return m.Lastpk + } + return nil +} + // VStreamResultsRequest is the payload for VStreamResults // The ids match VStreamRows, in case we decide to merge the two. // The ids match VStreamRows, in case we decide to merge the two. @@ -1746,7 +1867,7 @@ func (m *VStreamResultsRequest) Reset() { *m = VStreamResultsRequest{} } func (m *VStreamResultsRequest) String() string { return proto.CompactTextString(m) } func (*VStreamResultsRequest) ProtoMessage() {} func (*VStreamResultsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_5fd02bcb2e350dad, []int{23} + return fileDescriptor_5fd02bcb2e350dad, []int{25} } func (m *VStreamResultsRequest) XXX_Unmarshal(b []byte) error { @@ -1810,7 +1931,7 @@ func (m *VStreamResultsResponse) Reset() { *m = VStreamResultsResponse{} func (m *VStreamResultsResponse) String() string { return proto.CompactTextString(m) } func (*VStreamResultsResponse) ProtoMessage() {} func (*VStreamResultsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_5fd02bcb2e350dad, []int{24} + return fileDescriptor_5fd02bcb2e350dad, []int{26} } func (m *VStreamResultsResponse) XXX_Unmarshal(b []byte) error { @@ -1882,6 +2003,8 @@ func init() { proto.RegisterType((*VStreamResponse)(nil), "binlogdata.VStreamResponse") proto.RegisterType((*VStreamRowsRequest)(nil), "binlogdata.VStreamRowsRequest") proto.RegisterType((*VStreamRowsResponse)(nil), "binlogdata.VStreamRowsResponse") + proto.RegisterType((*LastPKEvent)(nil), "binlogdata.LastPKEvent") + proto.RegisterType((*TableLastPK)(nil), "binlogdata.TableLastPK") proto.RegisterType((*VStreamResultsRequest)(nil), "binlogdata.VStreamResultsRequest") proto.RegisterType((*VStreamResultsResponse)(nil), "binlogdata.VStreamResultsResponse") } @@ -1889,117 +2012,124 @@ func init() { func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_5fd02bcb2e350dad) } var fileDescriptor_5fd02bcb2e350dad = []byte{ - // 1779 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x58, 0xcd, 0x72, 0xdb, 0xc8, - 0x11, 0x36, 0xff, 0xc9, 0x86, 0x44, 0x41, 0x23, 0x59, 0x61, 0x5c, 0xd9, 0x2d, 0x2d, 0x2a, 0x5e, - 0x6b, 0x55, 0x15, 0x6a, 0xc3, 0x24, 0xce, 0x69, 0xb3, 0xe1, 0x0f, 0x24, 0xd3, 0x02, 0x49, 0x79, - 0x08, 0xcb, 0xa9, 0xbd, 0xa0, 0x20, 0x70, 0x24, 0x21, 0xc2, 0x9f, 0x81, 0xa1, 0xb4, 0x7c, 0x80, - 0x54, 0x1e, 0x20, 0x4f, 0x91, 0x73, 0xae, 0xc9, 0x35, 0x4f, 0x91, 0xca, 0x2d, 0xa7, 0x9c, 0xf2, - 0x06, 0xa9, 0xf9, 0x01, 0x08, 0xc8, 0x5b, 0xb6, 0xbc, 0x55, 0x39, 0x64, 0x2f, 0xac, 0x9e, 0x9e, - 0xee, 0x9e, 0xee, 0x6f, 0xba, 0x1b, 0x3d, 0x04, 0xf5, 0xc2, 0x0d, 0xbc, 0xf0, 0x6a, 0x61, 0x53, - 0xbb, 0x1b, 0xc5, 0x21, 0x0d, 0x11, 0xac, 0x39, 0x4f, 0x94, 0x5b, 0x1a, 0x47, 0x8e, 0xd8, 0x78, - 0xa2, 0xbc, 0x5d, 0x92, 0x78, 0x25, 0x17, 0x6d, 0x1a, 0x46, 0xe1, 0x5a, 0x4b, 0x9b, 0x40, 0x63, - 0x78, 0x6d, 0xc7, 0x09, 0xa1, 0x68, 0x0f, 0xea, 0x8e, 0xe7, 0x92, 0x80, 0x76, 0x4a, 0xfb, 0xa5, - 0x83, 0x1a, 0x96, 0x2b, 0x84, 0xa0, 0xea, 0x84, 0x41, 0xd0, 0x29, 0x73, 0x2e, 0xa7, 0x99, 0x6c, - 0x42, 0xe2, 0x5b, 0x12, 0x77, 0x2a, 0x42, 0x56, 0xac, 0xb4, 0x7f, 0x55, 0x60, 0x7b, 0xc0, 0xfd, - 0x30, 0x63, 0x3b, 0x48, 0x6c, 0x87, 0xba, 0x61, 0x80, 0x4e, 0x00, 0x12, 0x6a, 0x53, 0xe2, 0x93, - 0x80, 0x26, 0x9d, 0xd2, 0x7e, 0xe5, 0x40, 0xe9, 0x3d, 0xeb, 0xe6, 0x22, 0x78, 0x47, 0xa5, 0x3b, - 0x4f, 0xe5, 0x71, 0x4e, 0x15, 0xf5, 0x40, 0x21, 0xb7, 0x24, 0xa0, 0x16, 0x0d, 0x6f, 0x48, 0xd0, - 0xa9, 0xee, 0x97, 0x0e, 0x94, 0xde, 0x76, 0x57, 0x04, 0xa8, 0xb3, 0x1d, 0x93, 0x6d, 0x60, 0x20, - 0x19, 0xfd, 0xe4, 0xef, 0x65, 0x68, 0x65, 0xd6, 0x90, 0x01, 0x4d, 0xc7, 0xa6, 0xe4, 0x2a, 0x8c, - 0x57, 0x3c, 0xcc, 0x76, 0xef, 0xcb, 0x07, 0x3a, 0xd2, 0x1d, 0x4a, 0x3d, 0x9c, 0x59, 0x40, 0x3f, - 0x83, 0x86, 0x23, 0xd0, 0xe3, 0xe8, 0x28, 0xbd, 0x9d, 0xbc, 0x31, 0x09, 0x2c, 0x4e, 0x65, 0x90, - 0x0a, 0x95, 0xe4, 0xad, 0xc7, 0x21, 0xdb, 0xc0, 0x8c, 0xd4, 0xfe, 0x5c, 0x82, 0x66, 0x6a, 0x17, - 0xed, 0xc0, 0xd6, 0xc0, 0xb0, 0x5e, 0x4f, 0xb1, 0x3e, 0x9c, 0x9d, 0x4c, 0xc7, 0xdf, 0xe8, 0x23, - 0xf5, 0x11, 0xda, 0x80, 0xe6, 0xc0, 0xb0, 0x06, 0xfa, 0xc9, 0x78, 0xaa, 0x96, 0xd0, 0x26, 0xb4, - 0x06, 0x86, 0x35, 0x9c, 0x4d, 0x26, 0x63, 0x53, 0x2d, 0xa3, 0x2d, 0x50, 0x06, 0x86, 0x85, 0x67, - 0x86, 0x31, 0xe8, 0x0f, 0x4f, 0xd5, 0x0a, 0x7a, 0x0c, 0xdb, 0x03, 0xc3, 0x1a, 0x4d, 0x0c, 0x6b, - 0xa4, 0x9f, 0x61, 0x7d, 0xd8, 0x37, 0xf5, 0x91, 0x5a, 0x45, 0x00, 0x75, 0xc6, 0x1e, 0x19, 0x6a, - 0x4d, 0xd2, 0x73, 0xdd, 0x54, 0xeb, 0xd2, 0xdc, 0x78, 0x3a, 0xd7, 0xb1, 0xa9, 0x36, 0xe4, 0xf2, - 0xf5, 0xd9, 0xa8, 0x6f, 0xea, 0x6a, 0x53, 0x2e, 0x47, 0xba, 0xa1, 0x9b, 0xba, 0xda, 0x7a, 0x59, - 0x6d, 0x96, 0xd5, 0xca, 0xcb, 0x6a, 0xb3, 0xa2, 0x56, 0xb5, 0x3f, 0x95, 0xe0, 0xf1, 0x9c, 0xc6, - 0xc4, 0xf6, 0x4f, 0xc9, 0x0a, 0xdb, 0xc1, 0x15, 0xc1, 0xe4, 0xed, 0x92, 0x24, 0x14, 0x3d, 0x81, - 0x66, 0x14, 0x26, 0x2e, 0xc3, 0x8e, 0x03, 0xdc, 0xc2, 0xd9, 0x1a, 0x1d, 0x41, 0xeb, 0x86, 0xac, - 0xac, 0x98, 0xc9, 0x4b, 0xc0, 0x50, 0x37, 0x4b, 0xc8, 0xcc, 0x52, 0xf3, 0x46, 0x52, 0x79, 0x7c, - 0x2b, 0x1f, 0xc6, 0x57, 0xbb, 0x84, 0xbd, 0xfb, 0x4e, 0x25, 0x51, 0x18, 0x24, 0x04, 0x19, 0x80, - 0x84, 0xa2, 0x45, 0xd7, 0x77, 0xcb, 0xfd, 0x53, 0x7a, 0x9f, 0xbc, 0x37, 0x01, 0xf0, 0xf6, 0xc5, - 0x7d, 0x96, 0xf6, 0x2d, 0xec, 0x88, 0x73, 0x4c, 0xfb, 0xc2, 0x23, 0xc9, 0x43, 0x42, 0xdf, 0x83, - 0x3a, 0xe5, 0xc2, 0x9d, 0xf2, 0x7e, 0xe5, 0xa0, 0x85, 0xe5, 0xea, 0x63, 0x23, 0x5c, 0xc0, 0x6e, - 0xf1, 0xe4, 0xff, 0x49, 0x7c, 0xbf, 0x84, 0x2a, 0x5e, 0x7a, 0x04, 0xed, 0x42, 0xcd, 0xb7, 0xa9, - 0x73, 0x2d, 0xa3, 0x11, 0x0b, 0x16, 0xca, 0xa5, 0xeb, 0x51, 0x12, 0xf3, 0x2b, 0x6c, 0x61, 0xb9, - 0xd2, 0xfe, 0x52, 0x82, 0xfa, 0x31, 0x27, 0xd1, 0xe7, 0x50, 0x8b, 0x97, 0x2c, 0x58, 0x51, 0xeb, - 0x6a, 0xde, 0x03, 0x66, 0x19, 0x8b, 0x6d, 0x34, 0x86, 0xf6, 0xa5, 0x4b, 0xbc, 0x05, 0x2f, 0xdd, - 0x49, 0xb8, 0x10, 0x59, 0xd1, 0xee, 0x7d, 0x96, 0x57, 0x10, 0x36, 0xbb, 0xc7, 0x05, 0x41, 0x7c, - 0x4f, 0x51, 0x7b, 0x0e, 0xed, 0xa2, 0x04, 0x2b, 0x27, 0x1d, 0x63, 0x6b, 0x36, 0xb5, 0x26, 0xe3, - 0xf9, 0xa4, 0x6f, 0x0e, 0x5f, 0xa8, 0x8f, 0x78, 0xc5, 0xe8, 0x73, 0xd3, 0xd2, 0x8f, 0x8f, 0x67, - 0xd8, 0x54, 0x4b, 0xda, 0xbf, 0xcb, 0xb0, 0x21, 0x40, 0x99, 0x87, 0xcb, 0xd8, 0x21, 0xec, 0x16, - 0x6f, 0xc8, 0x2a, 0x89, 0x6c, 0x87, 0xa4, 0xb7, 0x98, 0xae, 0x19, 0x20, 0xc9, 0xb5, 0x1d, 0x2f, - 0x64, 0xe4, 0x62, 0x81, 0x7e, 0x05, 0x0a, 0xbf, 0x4d, 0x6a, 0xd1, 0x55, 0x44, 0xf8, 0x3d, 0xb6, - 0x7b, 0xbb, 0xeb, 0xc4, 0xe6, 0x77, 0x45, 0xcd, 0x55, 0x44, 0x30, 0xd0, 0x8c, 0x2e, 0x56, 0x43, - 0xf5, 0x01, 0xd5, 0xb0, 0xce, 0xa1, 0x5a, 0x21, 0x87, 0x0e, 0xb3, 0x0b, 0xa9, 0x4b, 0x2b, 0xef, - 0xa0, 0x97, 0x5e, 0x12, 0xea, 0x42, 0x3d, 0x0c, 0xac, 0xc5, 0xc2, 0xeb, 0x34, 0xb8, 0x9b, 0x3f, - 0xca, 0xcb, 0xce, 0x82, 0xd1, 0xc8, 0xe8, 0x8b, 0xb4, 0xa8, 0x85, 0xc1, 0x68, 0xe1, 0xa1, 0xa7, - 0xd0, 0x26, 0xdf, 0x52, 0x12, 0x07, 0xb6, 0x67, 0xf9, 0x2b, 0xd6, 0xbd, 0x9a, 0x3c, 0xf4, 0xcd, - 0x94, 0x3b, 0x61, 0x4c, 0xf4, 0x39, 0x6c, 0x25, 0x34, 0x8c, 0x2c, 0xfb, 0x92, 0x92, 0xd8, 0x72, - 0xc2, 0x68, 0xd5, 0x69, 0xed, 0x97, 0x0e, 0x9a, 0x78, 0x93, 0xb1, 0xfb, 0x8c, 0x3b, 0x0c, 0xa3, - 0x95, 0xf6, 0x0a, 0x5a, 0x38, 0xbc, 0x1b, 0x5e, 0xf3, 0x78, 0x34, 0xa8, 0x5f, 0x90, 0xcb, 0x30, - 0x26, 0x32, 0x51, 0x41, 0x36, 0x72, 0x1c, 0xde, 0x61, 0xb9, 0x83, 0xf6, 0xa1, 0xc6, 0x6d, 0xca, - 0x76, 0x91, 0x17, 0x11, 0x1b, 0x9a, 0x0d, 0x4d, 0x1c, 0xde, 0xf1, 0x6b, 0x47, 0x9f, 0x80, 0x00, - 0xd8, 0x0a, 0x6c, 0x3f, 0xbd, 0xbd, 0x16, 0xe7, 0x4c, 0x6d, 0x9f, 0xa0, 0xe7, 0xa0, 0xc4, 0xe1, - 0x9d, 0xe5, 0xf0, 0xe3, 0x45, 0x25, 0x2a, 0xbd, 0xc7, 0x85, 0xe4, 0x4c, 0x9d, 0xc3, 0x10, 0xa7, - 0x64, 0xa2, 0xbd, 0x02, 0x58, 0xe7, 0xd6, 0x87, 0x0e, 0xf9, 0x29, 0xbb, 0x0d, 0xe2, 0x2d, 0x52, - 0xfb, 0x1b, 0xd2, 0x65, 0x6e, 0x01, 0xcb, 0x3d, 0x06, 0xc4, 0x9c, 0x25, 0xcf, 0x09, 0x75, 0x17, - 0xdf, 0x23, 0xe5, 0x10, 0x54, 0xaf, 0xa8, 0xbb, 0xe0, 0xb9, 0xd6, 0xc2, 0x9c, 0xd6, 0xbe, 0x86, - 0xda, 0x39, 0x37, 0xf7, 0x1c, 0x14, 0x2e, 0x65, 0x31, 0x76, 0x5a, 0x83, 0x85, 0x30, 0xb3, 0xa3, - 0x31, 0x24, 0x29, 0x99, 0x68, 0x7d, 0xd8, 0x3c, 0x95, 0xc7, 0x72, 0x81, 0x8f, 0xf7, 0x4b, 0xfb, - 0x6b, 0x19, 0x1a, 0x2f, 0xc3, 0x25, 0x4b, 0x0c, 0xd4, 0x86, 0xb2, 0xbb, 0xe0, 0x7a, 0x15, 0x5c, - 0x76, 0x17, 0xe8, 0xb7, 0xd0, 0xf6, 0xdd, 0xab, 0xd8, 0x66, 0xe9, 0x25, 0x2a, 0x45, 0x14, 0xfb, - 0x8f, 0xf3, 0x9e, 0x4d, 0x52, 0x09, 0x5e, 0x2e, 0x9b, 0x7e, 0x7e, 0x99, 0x2b, 0x80, 0x4a, 0xa1, - 0x00, 0x9e, 0x42, 0xdb, 0x0b, 0x1d, 0xdb, 0xb3, 0xb2, 0xf6, 0x5b, 0x15, 0x49, 0xca, 0xb9, 0x67, - 0x69, 0x0f, 0xbe, 0x87, 0x4b, 0xed, 0x81, 0xb8, 0xa0, 0xaf, 0x60, 0x23, 0xb2, 0x63, 0xea, 0x3a, - 0x6e, 0x64, 0xb3, 0x01, 0xa6, 0xce, 0x15, 0x0b, 0x6e, 0x17, 0x70, 0xc3, 0x05, 0x71, 0xf4, 0x05, - 0xa8, 0x09, 0x6f, 0x2d, 0xd6, 0x5d, 0x18, 0xdf, 0x5c, 0x7a, 0xe1, 0x5d, 0xd2, 0x69, 0x70, 0xff, - 0xb7, 0x04, 0xff, 0x4d, 0xca, 0xd6, 0xfe, 0x53, 0x86, 0xfa, 0xb9, 0xc8, 0xb2, 0x43, 0xa8, 0x72, - 0x8c, 0xc4, 0x90, 0xb2, 0x97, 0x3f, 0x4c, 0x48, 0x70, 0x80, 0xb8, 0x0c, 0xfa, 0x09, 0xb4, 0xa8, - 0xeb, 0x93, 0x84, 0xda, 0x7e, 0xc4, 0x41, 0xad, 0xe0, 0x35, 0xe3, 0xbb, 0x72, 0x85, 0x4d, 0x22, - 0xac, 0x07, 0x08, 0x98, 0x18, 0x89, 0x7e, 0x0e, 0x2d, 0x56, 0x1b, 0x7c, 0x70, 0xea, 0xd4, 0x78, - 0xb1, 0xed, 0xde, 0xab, 0x0c, 0x7e, 0x2c, 0x6e, 0xc6, 0x69, 0xb5, 0xfd, 0x1a, 0x14, 0x9e, 0xcd, - 0x52, 0x49, 0x34, 0x9f, 0xbd, 0x62, 0xf3, 0x49, 0xab, 0x06, 0xc3, 0xba, 0x5f, 0xa3, 0x67, 0x50, - 0xbb, 0xe5, 0x2e, 0x35, 0xe4, 0x00, 0x97, 0x0f, 0x8e, 0xc3, 0x2f, 0xf6, 0xd9, 0xd7, 0xf1, 0xf7, - 0x22, 0x9b, 0x78, 0xdb, 0xb9, 0xf7, 0x75, 0x94, 0x89, 0x86, 0x53, 0x19, 0x1e, 0x95, 0xef, 0xf1, - 0xce, 0xc3, 0xa2, 0xf2, 0x3d, 0xf4, 0x19, 0x6c, 0x38, 0xcb, 0x38, 0xe6, 0x23, 0xa3, 0xeb, 0x93, - 0xce, 0x2e, 0x07, 0x47, 0x91, 0x3c, 0xd3, 0xf5, 0x89, 0x76, 0x0d, 0x1b, 0x13, 0x37, 0x70, 0x7d, - 0xdb, 0xe3, 0x7d, 0x9a, 0xc1, 0x95, 0x2b, 0x6c, 0x4e, 0x3f, 0xac, 0xa6, 0xd1, 0xa7, 0xa0, 0x44, - 0xd6, 0x8d, 0xe5, 0x84, 0xde, 0xd2, 0x0f, 0x44, 0x8e, 0x56, 0x70, 0x2b, 0x3a, 0x1d, 0x0a, 0x06, - 0xab, 0x2f, 0x79, 0xd2, 0xdc, 0xb9, 0x26, 0xbe, 0x8d, 0xbe, 0xcc, 0xf2, 0x59, 0xd4, 0x68, 0xa7, - 0x58, 0x09, 0x6b, 0xa7, 0xd2, 0x4c, 0xd7, 0xfe, 0x58, 0x86, 0xf6, 0xb9, 0x98, 0x00, 0xd2, 0xa9, - 0xe3, 0x6b, 0xd8, 0x21, 0x97, 0x97, 0xc4, 0xa1, 0xee, 0x2d, 0xb1, 0x1c, 0xdb, 0xf3, 0x48, 0x6c, - 0xc9, 0xba, 0x53, 0x7a, 0x5b, 0x5d, 0xf1, 0x12, 0x18, 0x72, 0xfe, 0x78, 0x84, 0xb7, 0x33, 0x59, - 0xc9, 0x5a, 0x20, 0x1d, 0x76, 0x5c, 0xdf, 0x27, 0x0b, 0xd7, 0xa6, 0x79, 0x03, 0xa2, 0xe1, 0x3e, - 0x96, 0x91, 0x9e, 0x9b, 0x27, 0x36, 0x25, 0x6b, 0x33, 0x99, 0x46, 0x66, 0xe6, 0x29, 0x0b, 0x26, - 0xbe, 0xca, 0x06, 0x99, 0x4d, 0xa9, 0x69, 0x72, 0x26, 0x96, 0x9b, 0x85, 0x21, 0xa9, 0x7a, 0x6f, - 0x48, 0x5a, 0x7f, 0xc8, 0x6a, 0x1f, 0xfa, 0x90, 0x69, 0x5f, 0xc1, 0x56, 0x06, 0x84, 0x1c, 0x82, - 0x0e, 0xa1, 0xce, 0x33, 0x31, 0x85, 0x13, 0xbd, 0x5b, 0x34, 0x58, 0x4a, 0x68, 0x7f, 0x28, 0x03, - 0x4a, 0xf5, 0xc3, 0xbb, 0xe4, 0xff, 0x14, 0xcc, 0x5d, 0xa8, 0x71, 0xbe, 0x44, 0x52, 0x2c, 0x18, - 0x0e, 0x9e, 0x9d, 0xd0, 0xe8, 0x26, 0x83, 0x51, 0x28, 0xbf, 0x62, 0xbf, 0x98, 0x24, 0x4b, 0x8f, - 0x62, 0x29, 0xa1, 0xfd, 0xad, 0x04, 0x3b, 0x05, 0x1c, 0x24, 0x96, 0xeb, 0x8c, 0x2f, 0xbd, 0x27, - 0xe3, 0x0f, 0xa0, 0x19, 0xdd, 0xbc, 0xa7, 0x32, 0xb2, 0xdd, 0xef, 0x6c, 0x42, 0x9f, 0x42, 0x35, - 0x66, 0xcd, 0xb0, 0xca, 0x35, 0xf3, 0x9f, 0x76, 0xce, 0x67, 0xf3, 0x41, 0x21, 0x8e, 0xc2, 0x7c, - 0x20, 0xfd, 0xff, 0x47, 0x09, 0x1e, 0xaf, 0xf3, 0x60, 0xe9, 0xd1, 0x1f, 0xd4, 0x55, 0x6a, 0x31, - 0xec, 0xdd, 0x8f, 0xee, 0xa3, 0x2e, 0xe8, 0x7b, 0xc0, 0x7e, 0xf8, 0x1b, 0x50, 0x72, 0x83, 0x20, - 0x7b, 0x2f, 0x8e, 0x4f, 0xa6, 0x33, 0xac, 0xab, 0x8f, 0x50, 0x13, 0xaa, 0x73, 0x73, 0x76, 0xa6, - 0x96, 0x18, 0xa5, 0xff, 0x4e, 0x1f, 0x8a, 0x37, 0x28, 0xa3, 0x2c, 0x29, 0x54, 0x39, 0xfc, 0x67, - 0x09, 0x60, 0xfd, 0x89, 0x42, 0x0a, 0x34, 0x5e, 0x4f, 0x4f, 0xa7, 0xb3, 0x37, 0x53, 0x61, 0xe0, - 0xc4, 0x1c, 0x8f, 0xd4, 0x12, 0x6a, 0x41, 0x4d, 0x3c, 0x6a, 0xcb, 0xec, 0x04, 0xf9, 0xa2, 0xad, - 0xb0, 0xe7, 0x6e, 0xf6, 0x9c, 0xad, 0xa2, 0x06, 0x54, 0xb2, 0x47, 0xab, 0x7c, 0xa5, 0xd6, 0x99, - 0x41, 0xac, 0x9f, 0x19, 0xfd, 0xa1, 0xae, 0x36, 0xd8, 0x46, 0xf6, 0x5e, 0x05, 0xa8, 0xa7, 0x8f, - 0x55, 0xa6, 0xc9, 0x9e, 0xb8, 0xc0, 0xce, 0x99, 0x99, 0x2f, 0x74, 0xac, 0x2a, 0x8c, 0x87, 0x67, - 0x6f, 0xd4, 0x0d, 0xc6, 0x3b, 0x1e, 0xeb, 0xc6, 0x48, 0xdd, 0x64, 0x6f, 0xdc, 0x17, 0x7a, 0x1f, - 0x9b, 0x03, 0xbd, 0x6f, 0xaa, 0x6d, 0xb6, 0x73, 0xce, 0x1d, 0xdc, 0x62, 0xc7, 0xbc, 0x9c, 0xbd, - 0xc6, 0xd3, 0xbe, 0xa1, 0xaa, 0x6c, 0x71, 0xae, 0xe3, 0xf9, 0x78, 0x36, 0x55, 0xb7, 0x0f, 0x9f, - 0xb1, 0x3e, 0x9e, 0x9f, 0x4b, 0x00, 0xea, 0x66, 0x7f, 0x60, 0xe8, 0x73, 0xf5, 0x11, 0xa3, 0xe7, - 0x2f, 0xfa, 0x78, 0x34, 0x57, 0x4b, 0x83, 0x2f, 0xbe, 0x79, 0x76, 0xeb, 0x52, 0x92, 0x24, 0x5d, - 0x37, 0x3c, 0x12, 0xd4, 0xd1, 0x55, 0x78, 0x74, 0x4b, 0x8f, 0xf8, 0x9f, 0x2f, 0x47, 0xeb, 0xf6, - 0x74, 0x51, 0xe7, 0x9c, 0x5f, 0xfc, 0x37, 0x00, 0x00, 0xff, 0xff, 0x1b, 0xb2, 0xd4, 0x99, 0xd8, - 0x11, 0x00, 0x00, + // 1901 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x58, 0x4b, 0x73, 0xe3, 0xc6, + 0x11, 0x5e, 0xbe, 0xc9, 0x06, 0x45, 0x41, 0xa3, 0x47, 0x98, 0xad, 0xd8, 0x25, 0xa3, 0x62, 0xaf, + 0xac, 0xaa, 0x50, 0x0e, 0x13, 0x6f, 0x2e, 0xb1, 0x1d, 0x3e, 0xb0, 0x5a, 0xae, 0xf8, 0xd0, 0x0e, + 0xb1, 0x5a, 0x97, 0x2f, 0x28, 0x08, 0x1c, 0x49, 0x88, 0x00, 0x02, 0x0b, 0x0c, 0x25, 0xf3, 0x07, + 0xa4, 0x2a, 0xf7, 0xfc, 0x8a, 0x9c, 0x72, 0xc8, 0x35, 0xb9, 0x26, 0x7f, 0x22, 0xd7, 0x5c, 0x92, + 0x3f, 0x91, 0x9a, 0x07, 0x5e, 0x5a, 0x7b, 0xa5, 0x75, 0x55, 0x0e, 0xc9, 0x85, 0x35, 0xd3, 0xd3, + 0xdd, 0xd3, 0xaf, 0xaf, 0xd1, 0x1c, 0x50, 0xcf, 0x9d, 0xa5, 0xeb, 0x5f, 0x2e, 0x2c, 0x6a, 0x75, + 0x82, 0xd0, 0xa7, 0x3e, 0x82, 0x94, 0xf2, 0x58, 0xb9, 0xa1, 0x61, 0x60, 0x8b, 0x83, 0xc7, 0xca, + 0x9b, 0x15, 0x09, 0xd7, 0x72, 0xd3, 0xa2, 0x7e, 0xe0, 0xa7, 0x52, 0xda, 0x04, 0x6a, 0x83, 0x2b, + 0x2b, 0x8c, 0x08, 0x45, 0x7b, 0x50, 0xb5, 0x5d, 0x87, 0x2c, 0x69, 0xbb, 0xb0, 0x5f, 0x38, 0xa8, + 0x60, 0xb9, 0x43, 0x08, 0xca, 0xb6, 0xbf, 0x5c, 0xb6, 0x8b, 0x9c, 0xca, 0xd7, 0x8c, 0x37, 0x22, + 0xe1, 0x0d, 0x09, 0xdb, 0x25, 0xc1, 0x2b, 0x76, 0xda, 0x3f, 0x4b, 0xb0, 0xd5, 0xe7, 0x76, 0x18, + 0xa1, 0xb5, 0x8c, 0x2c, 0x9b, 0x3a, 0xfe, 0x12, 0x1d, 0x03, 0x44, 0xd4, 0xa2, 0xc4, 0x23, 0x4b, + 0x1a, 0xb5, 0x0b, 0xfb, 0xa5, 0x03, 0xa5, 0xfb, 0xa4, 0x93, 0xf1, 0xe0, 0x2d, 0x91, 0xce, 0x3c, + 0xe6, 0xc7, 0x19, 0x51, 0xd4, 0x05, 0x85, 0xdc, 0x90, 0x25, 0x35, 0xa9, 0x7f, 0x4d, 0x96, 0xed, + 0xf2, 0x7e, 0xe1, 0x40, 0xe9, 0x6e, 0x75, 0x84, 0x83, 0x3a, 0x3b, 0x31, 0xd8, 0x01, 0x06, 0x92, + 0xac, 0x1f, 0xff, 0xad, 0x08, 0x8d, 0x44, 0x1b, 0x1a, 0x43, 0xdd, 0xb6, 0x28, 0xb9, 0xf4, 0xc3, + 0x35, 0x77, 0xb3, 0xd5, 0xfd, 0xec, 0x81, 0x86, 0x74, 0x06, 0x52, 0x0e, 0x27, 0x1a, 0xd0, 0xcf, + 0xa0, 0x66, 0x8b, 0xe8, 0xf1, 0xe8, 0x28, 0xdd, 0xed, 0xac, 0x32, 0x19, 0x58, 0x1c, 0xf3, 0x20, + 0x15, 0x4a, 0xd1, 0x1b, 0x97, 0x87, 0xac, 0x89, 0xd9, 0x52, 0xfb, 0x63, 0x01, 0xea, 0xb1, 0x5e, + 0xb4, 0x0d, 0x9b, 0xfd, 0xb1, 0xf9, 0x6a, 0x8a, 0xf5, 0xc1, 0xec, 0x78, 0x3a, 0xfa, 0x46, 0x1f, + 0xaa, 0x8f, 0x50, 0x13, 0xea, 0xfd, 0xb1, 0xd9, 0xd7, 0x8f, 0x47, 0x53, 0xb5, 0x80, 0x36, 0xa0, + 0xd1, 0x1f, 0x9b, 0x83, 0xd9, 0x64, 0x32, 0x32, 0xd4, 0x22, 0xda, 0x04, 0xa5, 0x3f, 0x36, 0xf1, + 0x6c, 0x3c, 0xee, 0xf7, 0x06, 0x27, 0x6a, 0x09, 0xed, 0xc2, 0x56, 0x7f, 0x6c, 0x0e, 0x27, 0x63, + 0x73, 0xa8, 0x9f, 0x62, 0x7d, 0xd0, 0x33, 0xf4, 0xa1, 0x5a, 0x46, 0x00, 0x55, 0x46, 0x1e, 0x8e, + 0xd5, 0x8a, 0x5c, 0xcf, 0x75, 0x43, 0xad, 0x4a, 0x75, 0xa3, 0xe9, 0x5c, 0xc7, 0x86, 0x5a, 0x93, + 0xdb, 0x57, 0xa7, 0xc3, 0x9e, 0xa1, 0xab, 0x75, 0xb9, 0x1d, 0xea, 0x63, 0xdd, 0xd0, 0xd5, 0xc6, + 0x8b, 0x72, 0xbd, 0xa8, 0x96, 0x5e, 0x94, 0xeb, 0x25, 0xb5, 0xac, 0xfd, 0xa1, 0x00, 0xbb, 0x73, + 0x1a, 0x12, 0xcb, 0x3b, 0x21, 0x6b, 0x6c, 0x2d, 0x2f, 0x09, 0x26, 0x6f, 0x56, 0x24, 0xa2, 0xe8, + 0x31, 0xd4, 0x03, 0x3f, 0x72, 0x58, 0xec, 0x78, 0x80, 0x1b, 0x38, 0xd9, 0xa3, 0x23, 0x68, 0x5c, + 0x93, 0xb5, 0x19, 0x32, 0x7e, 0x19, 0x30, 0xd4, 0x49, 0x0a, 0x32, 0xd1, 0x54, 0xbf, 0x96, 0xab, + 0x6c, 0x7c, 0x4b, 0xf7, 0xc7, 0x57, 0xbb, 0x80, 0xbd, 0xbb, 0x46, 0x45, 0x81, 0xbf, 0x8c, 0x08, + 0x1a, 0x03, 0x12, 0x82, 0x26, 0x4d, 0x73, 0xcb, 0xed, 0x53, 0xba, 0x1f, 0xbc, 0xb3, 0x00, 0xf0, + 0xd6, 0xf9, 0x5d, 0x92, 0xf6, 0x2d, 0x6c, 0x8b, 0x7b, 0x0c, 0xeb, 0xdc, 0x25, 0xd1, 0x43, 0x5c, + 0xdf, 0x83, 0x2a, 0xe5, 0xcc, 0xed, 0xe2, 0x7e, 0xe9, 0xa0, 0x81, 0xe5, 0xee, 0x7d, 0x3d, 0x5c, + 0xc0, 0x4e, 0xfe, 0xe6, 0xff, 0x8a, 0x7f, 0xbf, 0x84, 0x32, 0x5e, 0xb9, 0x04, 0xed, 0x40, 0xc5, + 0xb3, 0xa8, 0x7d, 0x25, 0xbd, 0x11, 0x1b, 0xe6, 0xca, 0x85, 0xe3, 0x52, 0x12, 0xf2, 0x14, 0x36, + 0xb0, 0xdc, 0x69, 0x7f, 0x2e, 0x40, 0xf5, 0x19, 0x5f, 0xa2, 0x4f, 0xa0, 0x12, 0xae, 0x98, 0xb3, + 0x02, 0xeb, 0x6a, 0xd6, 0x02, 0xa6, 0x19, 0x8b, 0x63, 0x34, 0x82, 0xd6, 0x85, 0x43, 0xdc, 0x05, + 0x87, 0xee, 0xc4, 0x5f, 0x88, 0xaa, 0x68, 0x75, 0x3f, 0xca, 0x0a, 0x08, 0x9d, 0x9d, 0x67, 0x39, + 0x46, 0x7c, 0x47, 0x50, 0x7b, 0x0a, 0xad, 0x3c, 0x07, 0x83, 0x93, 0x8e, 0xb1, 0x39, 0x9b, 0x9a, + 0x93, 0xd1, 0x7c, 0xd2, 0x33, 0x06, 0xcf, 0xd5, 0x47, 0x1c, 0x31, 0xfa, 0xdc, 0x30, 0xf5, 0x67, + 0xcf, 0x66, 0xd8, 0x50, 0x0b, 0xda, 0xbf, 0x8b, 0xd0, 0x14, 0x41, 0x99, 0xfb, 0xab, 0xd0, 0x26, + 0x2c, 0x8b, 0xd7, 0x64, 0x1d, 0x05, 0x96, 0x4d, 0xe2, 0x2c, 0xc6, 0x7b, 0x16, 0x90, 0xe8, 0xca, + 0x0a, 0x17, 0xd2, 0x73, 0xb1, 0x41, 0x9f, 0x83, 0xc2, 0xb3, 0x49, 0x4d, 0xba, 0x0e, 0x08, 0xcf, + 0x63, 0xab, 0xbb, 0x93, 0x16, 0x36, 0xcf, 0x15, 0x35, 0xd6, 0x01, 0xc1, 0x40, 0x93, 0x75, 0x1e, + 0x0d, 0xe5, 0x07, 0xa0, 0x21, 0xad, 0xa1, 0x4a, 0xae, 0x86, 0x0e, 0x93, 0x84, 0x54, 0xa5, 0x96, + 0xb7, 0xa2, 0x17, 0x27, 0x09, 0x75, 0xa0, 0xea, 0x2f, 0xcd, 0xc5, 0xc2, 0x6d, 0xd7, 0xb8, 0x99, + 0x3f, 0xca, 0xf2, 0xce, 0x96, 0xc3, 0xe1, 0xb8, 0x27, 0xca, 0xa2, 0xe2, 0x2f, 0x87, 0x0b, 0x17, + 0x7d, 0x0c, 0x2d, 0xf2, 0x2d, 0x25, 0xe1, 0xd2, 0x72, 0x4d, 0x6f, 0xcd, 0xba, 0x57, 0x9d, 0xbb, + 0xbe, 0x11, 0x53, 0x27, 0x8c, 0x88, 0x3e, 0x81, 0xcd, 0x88, 0xfa, 0x81, 0x69, 0x5d, 0x50, 0x12, + 0x9a, 0xb6, 0x1f, 0xac, 0xdb, 0x8d, 0xfd, 0xc2, 0x41, 0x1d, 0x6f, 0x30, 0x72, 0x8f, 0x51, 0x07, + 0x7e, 0xb0, 0xd6, 0x5e, 0x42, 0x03, 0xfb, 0xb7, 0x83, 0x2b, 0xee, 0x8f, 0x06, 0xd5, 0x73, 0x72, + 0xe1, 0x87, 0x44, 0x16, 0x2a, 0xc8, 0x46, 0x8e, 0xfd, 0x5b, 0x2c, 0x4f, 0xd0, 0x3e, 0x54, 0xb8, + 0x4e, 0xd9, 0x2e, 0xb2, 0x2c, 0xe2, 0x40, 0xb3, 0xa0, 0x8e, 0xfd, 0x5b, 0x9e, 0x76, 0xf4, 0x01, + 0x88, 0x00, 0x9b, 0x4b, 0xcb, 0x8b, 0xb3, 0xd7, 0xe0, 0x94, 0xa9, 0xe5, 0x11, 0xf4, 0x14, 0x94, + 0xd0, 0xbf, 0x35, 0x6d, 0x7e, 0xbd, 0x40, 0xa2, 0xd2, 0xdd, 0xcd, 0x15, 0x67, 0x6c, 0x1c, 0x86, + 0x30, 0x5e, 0x46, 0xda, 0x4b, 0x80, 0xb4, 0xb6, 0xee, 0xbb, 0xe4, 0xa7, 0x2c, 0x1b, 0xc4, 0x5d, + 0xc4, 0xfa, 0x9b, 0xd2, 0x64, 0xae, 0x01, 0xcb, 0x33, 0xed, 0xf7, 0x05, 0x68, 0xcc, 0x59, 0xf5, + 0x1c, 0x53, 0x67, 0xf1, 0x03, 0x6a, 0x0e, 0x41, 0xf9, 0x92, 0x3a, 0x0b, 0x5e, 0x6c, 0x0d, 0xcc, + 0xd7, 0xe8, 0xf3, 0xd8, 0xb0, 0xc0, 0xbc, 0x8e, 0xda, 0x65, 0x7e, 0x7b, 0x2e, 0xbf, 0xbc, 0x10, + 0xc7, 0x56, 0x44, 0x4f, 0x4f, 0x70, 0x9d, 0xb3, 0x9e, 0x9e, 0x44, 0xda, 0x57, 0x50, 0x39, 0xe3, + 0x56, 0x3c, 0x05, 0x85, 0x2b, 0x37, 0x99, 0xb6, 0x18, 0xbb, 0xb9, 0xf0, 0x24, 0x16, 0x63, 0x88, + 0xe2, 0x65, 0xa4, 0xf5, 0x60, 0xe3, 0x44, 0x5a, 0xcb, 0x19, 0xde, 0xdf, 0x1d, 0xed, 0x2f, 0x45, + 0xa8, 0xbd, 0xf0, 0x57, 0xac, 0xa0, 0x50, 0x0b, 0x8a, 0xce, 0x82, 0xcb, 0x95, 0x70, 0xd1, 0x59, + 0xa0, 0xdf, 0x40, 0xcb, 0x73, 0x2e, 0x43, 0x8b, 0x95, 0xa5, 0x40, 0x98, 0x68, 0x12, 0x3f, 0xce, + 0x5a, 0x36, 0x89, 0x39, 0x38, 0xcc, 0x36, 0xbc, 0xec, 0x36, 0x03, 0x9c, 0x52, 0x0e, 0x38, 0x1f, + 0x43, 0xcb, 0xf5, 0x6d, 0xcb, 0x35, 0x93, 0xb6, 0x5d, 0x16, 0xc5, 0xcd, 0xa9, 0xa7, 0x71, 0xef, + 0xbe, 0x13, 0x97, 0xca, 0x03, 0xe3, 0x82, 0xbe, 0x80, 0x66, 0x60, 0x85, 0xd4, 0xb1, 0x9d, 0xc0, + 0x62, 0x83, 0x4f, 0x95, 0x0b, 0xe6, 0xcc, 0xce, 0xc5, 0x0d, 0xe7, 0xd8, 0xd1, 0xa7, 0xa0, 0x46, + 0xbc, 0x25, 0x99, 0xb7, 0x7e, 0x78, 0x7d, 0xe1, 0xfa, 0xb7, 0x51, 0xbb, 0xc6, 0xed, 0xdf, 0x14, + 0xf4, 0xd7, 0x31, 0x59, 0xfb, 0x53, 0x09, 0xaa, 0x67, 0xa2, 0x3a, 0x0f, 0xa1, 0xcc, 0x63, 0x24, + 0x86, 0x9b, 0xbd, 0xec, 0x65, 0x82, 0x83, 0x07, 0x88, 0xf3, 0xa0, 0x9f, 0x40, 0x83, 0x3a, 0x1e, + 0x89, 0xa8, 0xe5, 0x05, 0x3c, 0xa8, 0x25, 0x9c, 0x12, 0xbe, 0xb3, 0xc4, 0x54, 0x28, 0xb1, 0xde, + 0x21, 0xc2, 0xc4, 0x96, 0xe8, 0xe7, 0xd0, 0x60, 0x98, 0xe2, 0x03, 0x57, 0xbb, 0xc2, 0x41, 0xba, + 0x73, 0x07, 0x51, 0xfc, 0x5a, 0x5c, 0x0f, 0x63, 0x94, 0xfe, 0x0a, 0x14, 0x8e, 0x02, 0x29, 0x24, + 0x9a, 0xd6, 0x5e, 0xbe, 0x69, 0xc5, 0x68, 0xc3, 0x90, 0xf6, 0x79, 0xf4, 0x04, 0x2a, 0x37, 0xdc, + 0xa4, 0x9a, 0x1c, 0xfc, 0xb2, 0xce, 0xf1, 0xf0, 0x8b, 0x73, 0xf6, 0x55, 0xfd, 0xad, 0xa8, 0x26, + 0xde, 0xae, 0xee, 0x7c, 0x55, 0x65, 0xa1, 0xe1, 0x98, 0x87, 0x7b, 0xe5, 0xb9, 0xbc, 0x63, 0x31, + 0xaf, 0x3c, 0x17, 0x7d, 0x04, 0x4d, 0x7b, 0x15, 0x86, 0x7c, 0xd4, 0x74, 0x3c, 0xd2, 0xde, 0xe1, + 0xc1, 0x51, 0x24, 0xcd, 0x70, 0x3c, 0x82, 0x7e, 0x0d, 0x2d, 0xd7, 0x8a, 0x28, 0x03, 0x9b, 0x74, + 0x64, 0x97, 0x5f, 0x95, 0x43, 0x9c, 0x00, 0x9b, 0xf0, 0x44, 0x71, 0xd3, 0x8d, 0x76, 0x05, 0xcd, + 0x89, 0xb3, 0x74, 0x3c, 0xcb, 0xe5, 0xa0, 0x64, 0xc1, 0xce, 0xb4, 0x13, 0xbe, 0x7e, 0x58, 0x27, + 0x41, 0x1f, 0x82, 0xc2, 0x4c, 0xb0, 0x7d, 0x77, 0xe5, 0x2d, 0x45, 0x85, 0x97, 0x70, 0x23, 0x38, + 0x19, 0x08, 0x02, 0x43, 0xa7, 0xbc, 0x69, 0x6e, 0x5f, 0x11, 0xcf, 0x42, 0x9f, 0x25, 0x68, 0x10, + 0x08, 0x6f, 0xe7, 0x71, 0x94, 0x1a, 0x15, 0xe3, 0x44, 0xfb, 0x7b, 0x11, 0x5a, 0x67, 0x62, 0xee, + 0x88, 0x67, 0x9d, 0xaf, 0x60, 0x9b, 0x5c, 0x5c, 0x10, 0x9b, 0x3a, 0x37, 0xc4, 0xb4, 0x2d, 0xd7, + 0x25, 0xa1, 0x29, 0x51, 0xab, 0x74, 0x37, 0x3b, 0xe2, 0xff, 0xc7, 0x80, 0xd3, 0x47, 0x43, 0xbc, + 0x95, 0xf0, 0x4a, 0xd2, 0x02, 0xe9, 0xb0, 0xed, 0x78, 0x1e, 0x59, 0x38, 0x16, 0xcd, 0x2a, 0x10, + 0x6d, 0x7e, 0x57, 0x7a, 0x7a, 0x66, 0x1c, 0x5b, 0x94, 0xa4, 0x6a, 0x12, 0x89, 0x44, 0xcd, 0xc7, + 0xcc, 0x99, 0xf0, 0x32, 0x19, 0x9f, 0x36, 0xa4, 0xa4, 0xc1, 0x89, 0x58, 0x1e, 0xe6, 0x46, 0xb3, + 0xf2, 0x9d, 0xd1, 0x2c, 0xfd, 0x7c, 0x56, 0xee, 0xfd, 0x7c, 0x7e, 0x09, 0x9b, 0xa2, 0xc5, 0xc6, + 0xa9, 0x8f, 0x51, 0xfd, 0xbd, 0x7d, 0xb6, 0x49, 0xd3, 0x4d, 0xa4, 0x7d, 0x01, 0x9b, 0x49, 0x20, + 0xe5, 0xe8, 0x76, 0x08, 0x55, 0x5e, 0x3e, 0x71, 0x3a, 0xd0, 0xdb, 0x90, 0xc5, 0x92, 0x43, 0xfb, + 0x5d, 0x11, 0x50, 0x2c, 0xef, 0xdf, 0x46, 0xff, 0xa3, 0xc9, 0xd8, 0x81, 0x0a, 0xa7, 0xcb, 0x4c, + 0x88, 0x0d, 0x8b, 0x03, 0x0b, 0x6a, 0x70, 0x9d, 0xa4, 0x41, 0x08, 0xbf, 0x64, 0xbf, 0x98, 0x44, + 0x2b, 0x97, 0x62, 0xc9, 0xa1, 0xfd, 0xb5, 0x00, 0xdb, 0xb9, 0x38, 0xc8, 0x58, 0xa6, 0x88, 0x29, + 0xbc, 0x03, 0x31, 0x07, 0x50, 0x0f, 0xae, 0xdf, 0x81, 0xac, 0xe4, 0xf4, 0x3b, 0x5b, 0xe0, 0x87, + 0x50, 0x0e, 0x59, 0x2b, 0x16, 0xdf, 0xd7, 0xec, 0x40, 0xc2, 0xe9, 0x6c, 0xaa, 0xc9, 0xf9, 0x91, + 0x9b, 0x6a, 0xa4, 0xfd, 0x0e, 0x28, 0x99, 0xce, 0xc0, 0x5a, 0x49, 0xbe, 0xaa, 0x64, 0xea, 0xbe, + 0xb7, 0xa8, 0x94, 0x4c, 0x51, 0xb1, 0x2e, 0x6e, 0xfb, 0x5e, 0xe0, 0x12, 0x4a, 0x44, 0xca, 0xea, + 0x38, 0x25, 0x68, 0x5f, 0x83, 0x92, 0x91, 0xbc, 0x6f, 0x78, 0x49, 0x93, 0x50, 0xba, 0x37, 0x09, + 0xff, 0x28, 0xc0, 0x6e, 0x5a, 0xcc, 0x2b, 0x97, 0xfe, 0x5f, 0xd5, 0xa3, 0x16, 0xc2, 0xde, 0x5d, + 0xef, 0xde, 0xab, 0xca, 0x7e, 0x40, 0xed, 0x1c, 0x7e, 0x09, 0x4a, 0x66, 0x06, 0x67, 0x7f, 0xd5, + 0x47, 0xc7, 0xd3, 0x19, 0xd6, 0xd5, 0x47, 0xa8, 0x0e, 0xe5, 0xb9, 0x31, 0x3b, 0x55, 0x0b, 0x6c, + 0xa5, 0x7f, 0xad, 0x0f, 0xc4, 0xdf, 0x7f, 0xb6, 0x32, 0x25, 0x53, 0xe9, 0xf0, 0x5f, 0x05, 0x80, + 0xf4, 0x2b, 0x8f, 0x14, 0xa8, 0xbd, 0x9a, 0x9e, 0x4c, 0x67, 0xaf, 0xa7, 0x42, 0xc1, 0xb1, 0x31, + 0x1a, 0xaa, 0x05, 0xd4, 0x80, 0x8a, 0x78, 0x4f, 0x28, 0xb2, 0x1b, 0xe4, 0x63, 0x42, 0x09, 0x35, + 0xa1, 0x9e, 0xbc, 0x24, 0x94, 0x51, 0x0d, 0x4a, 0xc9, 0x7b, 0x81, 0x7c, 0x20, 0xa8, 0x32, 0x85, + 0x58, 0x3f, 0x1d, 0xf7, 0x06, 0xba, 0x5a, 0x63, 0x07, 0xc9, 0x53, 0x01, 0x40, 0x35, 0x7e, 0x27, + 0x60, 0x92, 0x73, 0xdd, 0x50, 0x81, 0xdd, 0x33, 0x33, 0x9e, 0xeb, 0x58, 0x55, 0x18, 0x0d, 0xcf, + 0x5e, 0xab, 0x4d, 0x46, 0x7b, 0x36, 0xd2, 0xc7, 0x43, 0x75, 0x03, 0x6d, 0x40, 0xe3, 0xb9, 0xde, + 0xc3, 0x46, 0x5f, 0xef, 0x19, 0x6a, 0x8b, 0x9d, 0x9c, 0x71, 0x03, 0x37, 0xd9, 0x35, 0x2f, 0x66, + 0xaf, 0xf0, 0xb4, 0x37, 0x56, 0x55, 0xb6, 0x39, 0xd3, 0xf1, 0x7c, 0x34, 0x9b, 0xaa, 0x5b, 0xec, + 0x9e, 0x71, 0x6f, 0x6e, 0x9c, 0x9e, 0xa8, 0xe8, 0xf0, 0x09, 0xfb, 0xb0, 0x65, 0xc7, 0x3c, 0x80, + 0xaa, 0xd1, 0xeb, 0x8f, 0xf5, 0xb9, 0xfa, 0x88, 0xad, 0xe7, 0xcf, 0x7b, 0x78, 0x38, 0x57, 0x0b, + 0xfd, 0x4f, 0xbf, 0x79, 0x72, 0xe3, 0x50, 0x12, 0x45, 0x1d, 0xc7, 0x3f, 0x12, 0xab, 0xa3, 0x4b, + 0xff, 0xe8, 0x86, 0x1e, 0xf1, 0x37, 0xb0, 0xa3, 0x14, 0x64, 0xe7, 0x55, 0x4e, 0xf9, 0xc5, 0x7f, + 0x02, 0x00, 0x00, 0xff, 0xff, 0xb6, 0x0c, 0x0f, 0x53, 0x5f, 0x13, 0x00, 0x00, } diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index ed5c45425f7..14072726f5e 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -462,8 +462,8 @@ func (itc *internalTabletConn) StreamHealth(ctx context.Context, callback func(* } // VStream is part of queryservice.QueryService. -func (itc *internalTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - err := itc.tablet.qsc.QueryService().VStream(ctx, target, startPos, filter, send) +func (itc *internalTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, tableLastPKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + err := itc.tablet.qsc.QueryService().VStream(ctx, target, startPos, tableLastPKs, filter, send) return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) } diff --git a/go/vt/vtgate/endtoend/vstream_test.go b/go/vt/vtgate/endtoend/vstream_test.go index f38e0c47e07..301cda7a753 100644 --- a/go/vt/vtgate/endtoend/vstream_test.go +++ b/go/vt/vtgate/endtoend/vstream_test.go @@ -19,10 +19,15 @@ package endtoend import ( "context" "fmt" + "io" + "sync" "testing" "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/require" "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/log" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" "vitess.io/vitess/go/vt/proto/query" querypb "vitess.io/vitess/go/vt/proto/query" @@ -30,25 +35,32 @@ import ( "vitess.io/vitess/go/vt/vtgate/vtgateconn" ) -func TestVStream(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - +func initialize(ctx context.Context, t *testing.T) (*vtgateconn.VTGateConn, *mysql.Conn, *mysql.Conn, func()) { gconn, err := vtgateconn.Dial(ctx, grpcAddress) if err != nil { t.Fatal(err) } - defer gconn.Close() conn, err := mysql.Connect(ctx, &vtParams) if err != nil { t.Fatal(err) } - defer conn.Close() mconn, err := mysql.Connect(ctx, &mysqlParams) if err != nil { t.Fatal(err) } - defer conn.Close() + close := func() { + gconn.Close() + conn.Close() + mconn.Close() + } + return gconn, conn, mconn, close +} +func TestVStream(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + gconn, conn, mconn, closeConnections := initialize(ctx, t) + defer closeConnections() mpos, err := mconn.MasterPosition() if err != nil { @@ -128,3 +140,88 @@ func TestVStream(t *testing.T) { } cancel() } + +func TestVStreamCopyBasic(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + gconn, conn, mconn, closeConnections := initialize(ctx, t) + defer closeConnections() + + _, err := conn.ExecuteFetch("insert into t1(id1,id2) values(1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8)", 1, false) + if err != nil { + t.Fatal(err) + } + + lastPK := sqltypes.Result{ + Fields: []*query.Field{{Name: "id1", Type: query.Type_INT32}}, + Rows: [][]sqltypes.Value{{sqltypes.NewInt32(4)}}, + } + qr := sqltypes.ResultToProto3(&lastPK) + tablePKs := []*binlogdatapb.TableLastPK{{ + TableName: "t1", + Lastpk: qr, + }} + var shardGtids []*binlogdatapb.ShardGtid + var vgtid = &binlogdatapb.VGtid{} + shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{ + Keyspace: "ks", + Shard: "-80", + Gtid: "", + TablePKs: tablePKs, + }) + shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{ + Keyspace: "ks", + Shard: "80-", + Gtid: "", + TablePKs: tablePKs, + }) + vgtid.ShardGtids = shardGtids + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1", + }}, + } + reader, err := gconn.VStream(ctx, topodatapb.TabletType_MASTER, vgtid, filter) + _, _ = conn, mconn + if err != nil { + t.Fatal(err) + } + numExpectedEvents := 2 /* num shards */ * (7 /* begin/field/vgtid:pos/2 rowevents avg/vgitd: lastpk/commit) */ + 3 /* begin/vgtid/commit for completed table */) + require.NotNil(t, reader) + var evs []*binlogdatapb.VEvent + for { + e, err := reader.Recv() + switch err { + case nil: + evs = append(evs, e...) + if len(evs) == numExpectedEvents { + t.Logf("TestVStreamCopyBasic was successful") + return + } + printEvents(evs) // for debugging ci failures + case io.EOF: + log.Infof("stream ended\n") + cancel() + default: + log.Errorf("Returned err %v", err) + t.Fatalf("remote error: %v\n", err) + } + } +} + +var printMu sync.Mutex + +func printEvents(evs []*binlogdatapb.VEvent) { + printMu.Lock() + defer printMu.Unlock() + if len(evs) == 0 { + return + } + s := "\n===START===" + "\n" + for i, ev := range evs { + s += fmt.Sprintf("Event %d; %v\n", i, ev) + } + s += "===END===" + "\n" + log.Infof("%s", s) +} diff --git a/go/vt/vtgate/vstream_manager.go b/go/vt/vtgate/vstream_manager.go index 81ee3765e8b..21813cb1c85 100644 --- a/go/vt/vtgate/vstream_manager.go +++ b/go/vt/vtgate/vstream_manager.go @@ -150,6 +150,8 @@ func (vsm *vstreamManager) resolveParams(ctx context.Context, tabletType topodat newvgtid.ShardGtids = append(newvgtid.ShardGtids, sgtid) } } + //TODO add tablepk validations + return newvgtid, filter, nil } @@ -212,7 +214,7 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unexpected number or shards: %v", rss) } // Safe to access sgtid.Gtid here (because it can't change until streaming begins). - err = rss[0].Gateway.VStream(ctx, rss[0].Target, sgtid.Gtid, vs.filter, func(events []*binlogdatapb.VEvent) error { + err = rss[0].Gateway.VStream(ctx, rss[0].Target, sgtid.Gtid, sgtid.TablePKs, vs.filter, func(events []*binlogdatapb.VEvent) error { // We received a valid event. Reset error count. errCount = 0 @@ -319,6 +321,33 @@ func (vs *vstream) sendAll(sgtid *binlogdatapb.ShardGtid, eventss [][]*binlogdat Type: binlogdatapb.VEventType_VGTID, Vgtid: proto.Clone(vs.vgtid).(*binlogdatapb.VGtid), } + } else if event.Type == binlogdatapb.VEventType_LASTPK { + var foundIndex = -1 + eventTablePK := event.LastPKEvent.TableLastPK + for idx, pk := range sgtid.TablePKs { + if pk.TableName == eventTablePK.TableName { + foundIndex = idx + break + } + } + if foundIndex == -1 { + if !event.LastPKEvent.Completed { + sgtid.TablePKs = append(sgtid.TablePKs, eventTablePK) + } + } else { + if event.LastPKEvent.Completed { + // remove tablepk from sgtid + sgtid.TablePKs[foundIndex] = sgtid.TablePKs[len(sgtid.TablePKs)-1] + sgtid.TablePKs[len(sgtid.TablePKs)-1] = nil + sgtid.TablePKs = sgtid.TablePKs[:len(sgtid.TablePKs)-1] + } else { + sgtid.TablePKs[foundIndex] = eventTablePK + } + } + events[j] = &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_VGTID, + Vgtid: proto.Clone(vs.vgtid).(*binlogdatapb.VGtid), + } } } if err := vs.send(events); err != nil { diff --git a/go/vt/vttablet/endtoend/vstreamer_test.go b/go/vt/vttablet/endtoend/vstreamer_test.go index 9528a4018de..dcd961cd899 100644 --- a/go/vt/vttablet/endtoend/vstreamer_test.go +++ b/go/vt/vttablet/endtoend/vstreamer_test.go @@ -77,7 +77,7 @@ func TestHistorianSchemaUpdate(t *testing.T) { return nil } go func() { - if err := tsv.VStream(ctx, target, "current", filter, send); err != nil { + if err := tsv.VStream(ctx, target, "current", nil, filter, send); err != nil { fmt.Printf("Error in tsv.VStream: %v", err) t.Error(err) } @@ -198,7 +198,7 @@ func TestSchemaVersioning(t *testing.T) { } go func() { defer close(eventCh) - if err := tsv.VStream(ctx, target, "current", filter, send); err != nil { + if err := tsv.VStream(ctx, target, "current", nil, filter, send); err != nil { fmt.Printf("Error in tsv.VStream: %v", err) t.Error(err) } @@ -248,7 +248,7 @@ func TestSchemaVersioning(t *testing.T) { } go func() { defer close(eventCh) - if err := tsv.VStream(ctx, target, startPos, filter, send); err != nil { + if err := tsv.VStream(ctx, target, startPos, nil, filter, send); err != nil { fmt.Printf("Error in tsv.VStream: %v", err) t.Error(err) } @@ -317,7 +317,7 @@ func TestSchemaVersioning(t *testing.T) { } go func() { defer close(eventCh) - if err := tsv.VStream(ctx, target, startPos, filter, send); err != nil { + if err := tsv.VStream(ctx, target, startPos, nil, filter, send); err != nil { fmt.Printf("Error in tsv.VStream: %v", err) t.Error(err) } diff --git a/go/vt/vttablet/grpcqueryservice/server.go b/go/vt/vttablet/grpcqueryservice/server.go index e7979e14972..65f903a6ea5 100644 --- a/go/vt/vttablet/grpcqueryservice/server.go +++ b/go/vt/vttablet/grpcqueryservice/server.go @@ -341,7 +341,7 @@ func (q *query) VStream(request *binlogdatapb.VStreamRequest, stream queryservic request.EffectiveCallerId, request.ImmediateCallerId, ) - err = q.server.VStream(ctx, request.Target, request.Position, request.Filter, func(events []*binlogdatapb.VEvent) error { + err = q.server.VStream(ctx, request.Target, request.Position, request.TableLastPKs, request.Filter, func(events []*binlogdatapb.VEvent) error { return stream.Send(&binlogdatapb.VStreamResponse{ Events: events, }) diff --git a/go/vt/vttablet/grpctabletconn/conn.go b/go/vt/vttablet/grpctabletconn/conn.go index 3c3a62cb32b..8c87a49f580 100644 --- a/go/vt/vttablet/grpctabletconn/conn.go +++ b/go/vt/vttablet/grpctabletconn/conn.go @@ -598,7 +598,7 @@ func (conn *gRPCQueryClient) StreamHealth(ctx context.Context, callback func(*qu } // VStream starts a VReplication stream. -func (conn *gRPCQueryClient) VStream(ctx context.Context, target *querypb.Target, position string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (conn *gRPCQueryClient) VStream(ctx context.Context, target *querypb.Target, position string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { stream, err := func() (queryservicepb.Query_VStreamClient, error) { conn.mu.RLock() defer conn.mu.RUnlock() @@ -612,6 +612,7 @@ func (conn *gRPCQueryClient) VStream(ctx context.Context, target *querypb.Target ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx), Position: position, Filter: filter, + TableLastPKs: tablePKs, } stream, err := conn.c.VStream(ctx, req) if err != nil { diff --git a/go/vt/vttablet/queryservice/queryservice.go b/go/vt/vttablet/queryservice/queryservice.go index 2f6a9b68ba9..2dcf5ebab81 100644 --- a/go/vt/vttablet/queryservice/queryservice.go +++ b/go/vt/vttablet/queryservice/queryservice.go @@ -97,7 +97,7 @@ type QueryService interface { MessageAck(ctx context.Context, target *querypb.Target, name string, ids []*querypb.Value) (count int64, err error) // VStream streams VReplication events based on the specified filter. - VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + VStream(ctx context.Context, target *querypb.Target, startPos string, tableLastPKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error // VStreamRows streams rows of a table from the specified starting point. VStreamRows(ctx context.Context, target *querypb.Target, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error diff --git a/go/vt/vttablet/queryservice/wrapped.go b/go/vt/vttablet/queryservice/wrapped.go index 2ac41122d4d..c0af4a273c8 100644 --- a/go/vt/vttablet/queryservice/wrapped.go +++ b/go/vt/vttablet/queryservice/wrapped.go @@ -236,9 +236,9 @@ func (ws *wrappedService) MessageAck(ctx context.Context, target *querypb.Target return count, err } -func (ws *wrappedService) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (ws *wrappedService) VStream(ctx context.Context, target *querypb.Target, startPos string, tableLastPKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { return ws.wrapper(ctx, target, ws.impl, "VStream", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) { - innerErr := conn.VStream(ctx, target, startPos, filter, send) + innerErr := conn.VStream(ctx, target, startPos, tableLastPKs, filter, send) return false, innerErr }) } diff --git a/go/vt/vttablet/sandboxconn/sandboxconn.go b/go/vt/vttablet/sandboxconn/sandboxconn.go index a37d50ae556..a4fd397028b 100644 --- a/go/vt/vttablet/sandboxconn/sandboxconn.go +++ b/go/vt/vttablet/sandboxconn/sandboxconn.go @@ -344,7 +344,7 @@ func (sbc *SandboxConn) AddVStreamEvents(events []*binlogdatapb.VEvent, err erro } // VStream is part of the QueryService interface. -func (sbc *SandboxConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (sbc *SandboxConn) VStream(ctx context.Context, target *querypb.Target, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { if sbc.StartPos != "" && sbc.StartPos != startPos { return fmt.Errorf("startPos(%v): %v, want %v", target, startPos, sbc.StartPos) } diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go index f49776f6164..2d5993f8f7a 100644 --- a/go/vt/vttablet/tabletconntest/fakequeryservice.go +++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go @@ -666,14 +666,16 @@ var TestStreamHealthStreamHealthResponse = &querypb.StreamHealthResponse{ Shard: "test_shard", TabletType: topodatapb.TabletType_RDONLY, }, - Serving: true, + Serving: true, + TabletExternallyReparentedTimestamp: 1234589, + RealtimeStats: &querypb.RealtimeStats{ + CpuUsage: 1.0, HealthError: "random error", SecondsBehindMaster: 234, BinlogPlayersCount: 1, SecondsBehindMasterFilteredReplication: 2, - CpuUsage: 1.0, }, } @@ -697,7 +699,7 @@ func (f *FakeQueryService) StreamHealth(ctx context.Context, callback func(*quer } // VStream is part of the queryservice.QueryService interface -func (f *FakeQueryService) VStream(ctx context.Context, target *querypb.Target, position string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (f *FakeQueryService) VStream(ctx context.Context, target *querypb.Target, position string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { panic("not implemented") } diff --git a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go index bbbca25688a..d991cb4a319 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go +++ b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go @@ -46,7 +46,7 @@ type VStreamerClient interface { Close(context.Context) error // VStream streams VReplication events based on the specified filter. - VStream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error // VStreamRows streams rows of a table from the specified starting point. VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error @@ -124,8 +124,8 @@ func (c *mysqlConnector) Close(ctx context.Context) error { return nil } -func (c *mysqlConnector) VStream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - return c.vstreamer.Stream(ctx, startPos, filter, send) +func (c *mysqlConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + return c.vstreamer.Stream(ctx, startPos, tablePKs, filter, send) } func (c *mysqlConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { @@ -169,8 +169,8 @@ func (tc *tabletConnector) Close(ctx context.Context) error { return tc.qs.Close(ctx) } -func (tc *tabletConnector) VStream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - return tc.qs.VStream(ctx, tc.target, startPos, filter, send) +func (tc *tabletConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + return tc.qs.VStream(ctx, tc.target, startPos, tablePKs, filter, send) } func (tc *tabletConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error { diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index e35481e7f75..aa98039c311 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -224,7 +224,7 @@ func (ftc *fakeTabletConn) StreamHealth(ctx context.Context, callback func(*quer var vstreamHook func(ctx context.Context) // VStream directly calls into the pre-initialized engine. -func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { if target.Keyspace != "vttest" { <-ctx.Done() return io.EOF @@ -232,7 +232,7 @@ func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, if vstreamHook != nil { vstreamHook(ctx) } - return streamerEngine.Stream(ctx, startPos, filter, send) + return streamerEngine.Stream(ctx, startPos, tablePKs, filter, send) } // vstreamRowsHook allows you to do work just before calling VStreamRows. diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 3ed3c0f7ee8..8225db57312 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -136,7 +136,7 @@ func (vp *vplayer) fetchAndApply(ctx context.Context) (err error) { streamErr := make(chan error, 1) go func() { - streamErr <- vp.vr.sourceVStreamer.VStream(ctx, mysql.EncodePosition(vp.startPos), vp.replicatorPlan.VStreamFilter, func(events []*binlogdatapb.VEvent) error { + streamErr <- vp.vr.sourceVStreamer.VStream(ctx, mysql.EncodePosition(vp.startPos), nil, vp.replicatorPlan.VStreamFilter, func(events []*binlogdatapb.VEvent) error { return relay.Send(events) }) }() diff --git a/go/vt/vttablet/tabletserver/messager/engine.go b/go/vt/vttablet/tabletserver/messager/engine.go index 0d275d7bdc7..3a84acea713 100644 --- a/go/vt/vttablet/tabletserver/messager/engine.go +++ b/go/vt/vttablet/tabletserver/messager/engine.go @@ -44,7 +44,7 @@ type TabletService interface { // VStreamer defines the functions of VStreamer // that the messager needs. type VStreamer interface { - Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error StreamResults(ctx context.Context, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error } diff --git a/go/vt/vttablet/tabletserver/messager/message_manager.go b/go/vt/vttablet/tabletserver/messager/message_manager.go index 88c949672e5..4cd0ffe69d2 100644 --- a/go/vt/vttablet/tabletserver/messager/message_manager.go +++ b/go/vt/vttablet/tabletserver/messager/message_manager.go @@ -649,7 +649,7 @@ func (mm *messageManager) runOneVStream(ctx context.Context) error { var curPos string var fields []*querypb.Field - err := mm.vs.Stream(ctx, "current", mm.vsFilter, func(events []*binlogdatapb.VEvent) error { + err := mm.vs.Stream(ctx, "current", nil, mm.vsFilter, func(events []*binlogdatapb.VEvent) error { mm.streamMu.Lock() defer mm.streamMu.Unlock() diff --git a/go/vt/vttablet/tabletserver/messager/message_manager_test.go b/go/vt/vttablet/tabletserver/messager/message_manager_test.go index a47569d01f6..778ecb80ec4 100644 --- a/go/vt/vttablet/tabletserver/messager/message_manager_test.go +++ b/go/vt/vttablet/tabletserver/messager/message_manager_test.go @@ -891,7 +891,7 @@ func (fv *fakeVStreamer) setPollerResponse(pr []*binlogdatapb.VStreamResultsResp fv.pollerResponse = pr } -func (fv *fakeVStreamer) Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (fv *fakeVStreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { fv.streamInvocations.Add(1) for { fv.mu.Lock() diff --git a/go/vt/vttablet/tabletserver/replication_watcher.go b/go/vt/vttablet/tabletserver/replication_watcher.go index 485a7dfdfb3..178bdeab8d6 100644 --- a/go/vt/vttablet/tabletserver/replication_watcher.go +++ b/go/vt/vttablet/tabletserver/replication_watcher.go @@ -31,7 +31,7 @@ import ( // VStreamer defines the functions of VStreamer // that the replicationWatcher needs. type VStreamer interface { - Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error } // ReplicationWatcher is a tabletserver service that watches the @@ -89,7 +89,7 @@ func (rpw *ReplicationWatcher) Process(ctx context.Context) { var gtid string for { // The tracker will reload the schema and save it into _vt.schema_tracking when the vstream encounters a DDL. - err := rpw.vs.Stream(ctx, "current", filter, func(events []*binlogdatapb.VEvent) error { + err := rpw.vs.Stream(ctx, "current", nil, filter, func(events []*binlogdatapb.VEvent) error { for _, event := range events { if event.Type == binlogdatapb.VEventType_GTID { gtid = event.Gtid diff --git a/go/vt/vttablet/tabletserver/replication_watcher_test.go b/go/vt/vttablet/tabletserver/replication_watcher_test.go index b4f645b2773..67637b9cf34 100644 --- a/go/vt/vttablet/tabletserver/replication_watcher_test.go +++ b/go/vt/vttablet/tabletserver/replication_watcher_test.go @@ -149,7 +149,7 @@ func (f *fakeEnv) Stats() *tabletenv.Stats { func (f *fakeEnv) LogError() { } -func (f *fakeVstreamer) Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (f *fakeVstreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { f.called = true for _, events := range f.events { err := send(events) diff --git a/go/vt/vttablet/tabletserver/schema/engine.go b/go/vt/vttablet/tabletserver/schema/engine.go index b304fdb6541..528b8787fb1 100644 --- a/go/vt/vttablet/tabletserver/schema/engine.go +++ b/go/vt/vttablet/tabletserver/schema/engine.go @@ -255,6 +255,7 @@ func (se *Engine) reload(ctx context.Context) error { if _, ok := se.tables[tableName]; ok && createTime < se.lastChange { continue } + log.V(2).Infof("Reading schema for table: %s", tableName) table, err := LoadTable(conn, tableName, row[1].ToString(), row[3].ToString()) if err != nil { rec.RecordError(err) diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index dcf1951f937..34246108c87 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1353,11 +1353,11 @@ func (tsv *TabletServer) execDML(ctx context.Context, target *querypb.Target, qu } // VStream streams VReplication events. -func (tsv *TabletServer) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (tsv *TabletServer) VStream(ctx context.Context, target *querypb.Target, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { if err := tsv.verifyTarget(ctx, target); err != nil { return err } - return tsv.vstreamer.Stream(ctx, startPos, filter, send) + return tsv.vstreamer.Stream(ctx, startPos, tablePKs, filter, send) } // VStreamRows streams rows from the specified starting point. diff --git a/go/vt/vttablet/tabletserver/vstreamer/copy.go b/go/vt/vttablet/tabletserver/vstreamer/copy.go new file mode 100644 index 00000000000..22e73cbd67a --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/copy.go @@ -0,0 +1,279 @@ +/* +Copyright 2020 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vstreamer + +import ( + "context" + "fmt" + "io" + "math" + "time" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/log" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" +) + +// starts the copy phase for the first table in the (sorted) list. +// can be continuing the copy of a partially completed table or start a new one +func (uvs *uvstreamer) copy(ctx context.Context) error { + for len(uvs.tablesToCopy) > 0 { + tableName := uvs.tablesToCopy[0] + log.V(2).Infof("Copystate not empty starting catchupAndCopy on table %s", tableName) + if err := uvs.catchupAndCopy(ctx, tableName); err != nil { + return err + } + } + log.Info("No tables left to copy") + return nil +} + +// first does a catchup for tables already fully or partially copied (upto last pk) +func (uvs *uvstreamer) catchupAndCopy(ctx context.Context, tableName string) error { + log.Infof("catchupAndCopy for %s", tableName) + if !uvs.pos.IsZero() { + if err := uvs.catchup(ctx); err != nil { + log.Infof("catchupAndCopy: catchup returned %v", err) + return err + } + } + log.Infof("catchupAndCopy: before copyTable %s", tableName) + uvs.fields = nil + return uvs.copyTable(ctx, tableName) +} + +// catchup on events for tables already fully or partially copied (upto last pk) until replication lag is small +func (uvs *uvstreamer) catchup(ctx context.Context) error { + log.Infof("starting catchup ...") + uvs.setSecondsBehindMaster(math.MaxInt64) + ctx, cancel := context.WithCancel(ctx) + defer cancel() + errch := make(chan error, 1) + go func() { + startPos := mysql.EncodePosition(uvs.pos) + vs := newVStreamer(ctx, uvs.cp, uvs.se, uvs.sh, startPos, "", uvs.filter, uvs.vschema, uvs.send2) + errch <- vs.Stream() + uvs.vs = nil + log.Infof("catchup vs.stream returned with vs.pos %s", vs.pos.String()) + }() + + // Wait for catchup. + tkr := time.NewTicker(uvs.config.CatchupRetryTime) + defer tkr.Stop() + seconds := int64(uvs.config.MaxReplicationLag / time.Second) + for { + sbm := uvs.getSecondsBehindMaster() + if sbm <= seconds { + log.Infof("Canceling context because lag is %d:%d", sbm, seconds) + cancel() + // Make sure vplayer returns before returning. + <-errch + return nil + } + select { + case err := <-errch: + if err != nil { + return err + } + return io.EOF + case <-ctx.Done(): + // Make sure vplayer returns before returning. + <-errch + return io.EOF + case <-tkr.C: + } + } +} + +// field event is sent for every new rowevent or set of rowevents +func (uvs *uvstreamer) sendFieldEvent(ctx context.Context, gtid string, fieldEvent *binlogdatapb.FieldEvent) error { + evs := []*binlogdatapb.VEvent{{ + Type: binlogdatapb.VEventType_BEGIN, + }, { + Type: binlogdatapb.VEventType_FIELD, + FieldEvent: fieldEvent, + }} + log.V(2).Infof("Sending field event %v, gtid is %s", fieldEvent, gtid) + uvs.send(evs) + if err := uvs.setPosition(gtid, true); err != nil { + log.Infof("setPosition returned error %v", err) + return err + } + return nil +} + +// send one RowEvent per row, followed by a LastPK (merged in VTGate with vgtid) +func (uvs *uvstreamer) sendEventsForRows(ctx context.Context, tableName string, rows *binlogdatapb.VStreamRowsResponse, qr *querypb.QueryResult) error { + var evs []*binlogdatapb.VEvent + for _, row := range rows.Rows { + ev := &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_ROW, + RowEvent: &binlogdatapb.RowEvent{ + TableName: tableName, + RowChanges: []*binlogdatapb.RowChange{{ + Before: nil, + After: row, + }}, + }, + } + evs = append(evs, ev) + } + lastPKEvent := &binlogdatapb.LastPKEvent{ + TableLastPK: &binlogdatapb.TableLastPK{ + TableName: tableName, + Lastpk: qr, + }, + Completed: false, + } + + ev := &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_LASTPK, + LastPKEvent: lastPKEvent, + } + evs = append(evs, ev) + evs = append(evs, &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_COMMIT, + }) + if err := uvs.send(evs); err != nil { + log.Infof("send returned error %v", err) + return err + } + return nil +} + +// converts lastpk from proto to value +func getLastPKFromQR(qr *querypb.QueryResult) []sqltypes.Value { + if qr == nil { + return nil + } + var lastPK []sqltypes.Value + r := sqltypes.Proto3ToResult(qr) + if len(r.Rows) != 1 { + log.Errorf("unexpected lastpk input: %v", qr) + return nil + } + lastPK = r.Rows[0] + return lastPK +} + +// converts lastpk from value to proto +func getQRFromLastPK(fields []*querypb.Field, lastPK []sqltypes.Value) *querypb.QueryResult { + row := sqltypes.RowToProto3(lastPK) + qr := &querypb.QueryResult{ + Fields: fields, + Rows: []*querypb.Row{row}, + } + return qr +} + +// gets batch of rows to copy. size of batch is determined by max packetsize +func (uvs *uvstreamer) copyTable(ctx context.Context, tableName string) error { + ctx, cancel := context.WithCancel(ctx) + defer cancel() + var newLastPK *sqltypes.Result + lastPK := getLastPKFromQR(uvs.plans[tableName].tablePK.Lastpk) + filter := uvs.plans[tableName].rule.Filter + + log.Infof("Starting copyTable for %s, PK %v", tableName, lastPK) + uvs.sendTestEvent(fmt.Sprintf("Copy Start %s", tableName)) + + err := uvs.vse.StreamRows(ctx, filter, lastPK, func(rows *binlogdatapb.VStreamRowsResponse) error { + select { + case <-ctx.Done(): + log.Infof("Returning io.EOF in StreamRows") + return io.EOF + default: + } + if uvs.fields == nil { + if len(rows.Fields) == 0 { + return fmt.Errorf("expecting field event first, got: %v", rows) + } + pos, _ := mysql.DecodePosition(rows.Gtid) + if !uvs.pos.IsZero() && !uvs.pos.AtLeast(pos) { + if err := uvs.fastForward(rows.Gtid); err != nil { + log.Infof("fastForward returned error %v", err) + return err + } + if mysql.EncodePosition(uvs.pos) != rows.Gtid { + return fmt.Errorf("position after fastforward was %s but stopPos was %s", uvs.pos, rows.Gtid) + } + if err := uvs.setPosition(rows.Gtid, false); err != nil { + return err + } + } else { + log.V(2).Infof("Not starting fastforward pos is %s, uvs.pos is %s, rows.gtid %s", pos, uvs.pos, rows.Gtid) + } + + fieldEvent := &binlogdatapb.FieldEvent{ + TableName: tableName, + Fields: rows.Fields, + } + uvs.fields = rows.Fields + uvs.pkfields = rows.Pkfields + if err := uvs.sendFieldEvent(ctx, rows.Gtid, fieldEvent); err != nil { + log.Infof("sendFieldEvent returned error %v", err) + return err + } + } + if len(rows.Rows) == 0 { + log.V(2).Infof("0 rows returned for table %s", tableName) + return nil + } + + newLastPK = sqltypes.CustomProto3ToResult(uvs.pkfields, &querypb.QueryResult{ + Fields: rows.Fields, + Rows: []*querypb.Row{rows.Lastpk}, + }) + qrLastPK := sqltypes.ResultToProto3(newLastPK) + log.V(2).Infof("Calling sendEventForRows with gtid %s", rows.Gtid) + if err := uvs.sendEventsForRows(ctx, tableName, rows, qrLastPK); err != nil { + log.Infof("sendEventsForRows returned error %v", err) + return err + } + + uvs.setCopyState(tableName, qrLastPK) + log.V(2).Infof("NewLastPK: %v", qrLastPK) + return nil + }) + if err != nil { + return err + } + + select { + case <-ctx.Done(): + log.Infof("Context done: Copy of %v stopped at lastpk: %v", tableName, newLastPK) + return ctx.Err() + default: + } + + log.Infof("Copy of %v finished at lastpk: %v", tableName, newLastPK) + if err := uvs.copyComplete(tableName); err != nil { + return err + } + return nil +} + +// processes events between when a table was caught up and when a snapshot is taken for streaming a batch of rows +func (uvs *uvstreamer) fastForward(stopPos string) error { + log.Infof("starting fastForward from %s upto pos %s", mysql.EncodePosition(uvs.pos), stopPos) + uvs.stopPos, _ = mysql.DecodePosition(stopPos) + vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, uvs.sh, mysql.EncodePosition(uvs.pos), "", uvs.filter, uvs.vschema, uvs.send2) + uvs.vs = vs + return vs.Stream() +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index ce362fd181a..c49e5d6e5af 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -49,7 +49,7 @@ type Engine struct { // wg is incremented for every Stream, and decremented on end. // Close waits for all current streams to end by waiting on wg. wg sync.WaitGroup - streamers map[int]*vstreamer + streamers map[int]*uvstreamer rowStreamers map[int]*rowStreamer resultStreamers map[int]*resultStreamer streamIdx int @@ -78,7 +78,7 @@ type Engine struct { func NewEngine(env tabletenv.Env, ts srvtopo.Server, se *schema.Engine, sh schema.Historian) *Engine { vse := &Engine{ env: env, - streamers: make(map[int]*vstreamer), + streamers: make(map[int]*uvstreamer), rowStreamers: make(map[int]*rowStreamer), resultStreamers: make(map[int]*resultStreamer), lvschema: &localVSchema{vschema: &vindexes.VSchema{}}, @@ -145,20 +145,20 @@ func (vse *Engine) vschema() *vindexes.VSchema { } // Stream starts a new stream. -func (vse *Engine) Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { // Ensure vschema is initialized and the watcher is started. // Starting of the watcher has to be delayed till the first call to Stream // because this overhead should be incurred only if someone uses this feature. vse.watcherOnce.Do(vse.setWatch) // Create stream and add it to the map. - streamer, idx, err := func() (*vstreamer, int, error) { + streamer, idx, err := func() (*uvstreamer, int, error) { vse.mu.Lock() defer vse.mu.Unlock() if !vse.isOpen { return nil, 0, errors.New("VStreamer is not open") } - streamer := newVStreamer(ctx, vse.env.Config().DB.AppWithDB(), vse.se, vse.sh, startPos, filter, vse.lvschema, send) + streamer := newUVStreamer(ctx, vse, vse.env.Config().DB.AppWithDB(), vse.se, vse.sh, startPos, tablePKs, filter, vse.lvschema, send) idx := vse.streamIdx vse.streamers[idx] = streamer vse.streamIdx++ diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go index f2241b1221a..64c71114bb0 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go @@ -88,7 +88,7 @@ func TestUpdateVSchema(t *testing.T) { }}, } // Stream should terminate immediately due to canceled context. - _ = engine.Stream(ctx, "current", filter, func(_ []*binlogdatapb.VEvent) error { + _ = engine.Stream(ctx, "current", nil, filter, func(_ []*binlogdatapb.VEvent) error { return nil }) diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go index e6fa88c4989..366036a92c9 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go @@ -81,6 +81,7 @@ func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, sh schema.Histo } func (rs *rowStreamer) Cancel() { + log.Info("Rowstreamer Cancel() called") rs.cancel() } @@ -137,6 +138,7 @@ func (rs *rowStreamer) buildPlan() error { if err != nil { return err } + log.Infof("Rowstreamer, table plan %v, pkColumns %v, fields %v", rs.plan, rs.pkColumns, rs.plan.fields()) return err } @@ -234,8 +236,10 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V lastpk := make([]sqltypes.Value, len(rs.pkColumns)) byteCount := 0 for { + //log.Infof("StreamResponse for loop iteration starts") select { case <-rs.ctx.Done(): + log.Infof("Stream ended because of ctx.Done") return fmt.Errorf("stream ended: %v", rs.ctx.Err()) default: } @@ -247,7 +251,7 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V if row == nil { break } - // Compute lastpk here, because we'll neeed it + // Compute lastpk here, because we'll need it // at the end after the loop exits. for i, pk := range rs.pkColumns { lastpk[i] = row[pk] @@ -268,6 +272,7 @@ func (rs *rowStreamer) streamQuery(conn *snapshotConn, send func(*binlogdatapb.V response.Lastpk = sqltypes.RowToProto3(lastpk) err = send(response) if err != nil { + log.Infof("Rowstreamer send returned error %v", err) return err } // empty the rows so we start over, but we keep the diff --git a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go index 2fe437186da..79bf3fba3cf 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go +++ b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go @@ -78,6 +78,7 @@ func (conn *snapshotConn) startSnapshot(ctx context.Context, table string) (gtid log.Infof("Locking table %s for copying", table) if _, err := lockConn.ExecuteFetch(fmt.Sprintf("lock tables %s read", tableIdent), 1, false); err != nil { + log.Infof("Error locking table %s to read", tableIdent) return "", err } mpos, err := lockConn.MasterPosition() diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go new file mode 100644 index 00000000000..0ceb013534b --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go @@ -0,0 +1,467 @@ +/* +Copyright 2020 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vstreamer + +import ( + "context" + "fmt" + "io" + "regexp" + "sort" + "strings" + "sync" + "time" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/key" + "vitess.io/vitess/go/vt/log" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" +) + +var uvstreamerTestMode = false // Only used for testing + +type tablePlan struct { + tablePK *binlogdatapb.TableLastPK + rule *binlogdatapb.Rule +} + +type uvstreamer struct { + ctx context.Context + cancel func() + + // input parameters + vse *Engine + send func([]*binlogdatapb.VEvent) error + cp dbconfigs.Connector + se *schema.Engine + sh schema.Historian + startPos string + filter *binlogdatapb.Filter + inTablePKs []*binlogdatapb.TableLastPK + vschema *localVSchema + + // map holds tables remaining to be fully copied, it is depleted as each table gets completely copied + plans map[string]*tablePlan + tablesToCopy []string + + // changes for each table being copied + fields []*querypb.Field + pkfields []*querypb.Field + + // current position in the binlog for this streamer + pos mysql.Position + + // fast forward uses this to stop replicating upto the point of the last snapshot + stopPos mysql.Position + + // lastTimestampNs is the last timestamp seen so far. + lastTimestampNs int64 + secondsBehindMaster int64 + mu sync.Mutex + + config *uvstreamerConfig + + vs *vstreamer //last vstreamer created in uvstreamer: FIXME currently used only for setting vschema, find another way? +} + +type uvstreamerConfig struct { + MaxReplicationLag time.Duration + CatchupRetryTime time.Duration +} + +func newUVStreamer(ctx context.Context, vse *Engine, cp dbconfigs.Connector, se *schema.Engine, sh schema.Historian, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, vschema *localVSchema, send func([]*binlogdatapb.VEvent) error) *uvstreamer { + ctx, cancel := context.WithCancel(ctx) + config := &uvstreamerConfig{ + MaxReplicationLag: 1 * time.Nanosecond, + CatchupRetryTime: 1 * time.Second, + } + uvs := &uvstreamer{ + ctx: ctx, + cancel: cancel, + vse: vse, + send: send, + cp: cp, + se: se, + sh: sh, + startPos: startPos, + filter: filter, + vschema: vschema, + config: config, + inTablePKs: tablePKs, + } + + return uvs +} + +// buildTablePlan identifies the tables for the copy phase and creates the plans which consist of the lastPK seen +// for a table and its Rule (for filtering purposes by the vstreamer engine) +// it can be called +// the first time, with just the filter and an empty pos +// during a restart, with both the filter and list of TableLastPK from the vgtid +func (uvs *uvstreamer) buildTablePlan() error { + + uvs.plans = make(map[string]*tablePlan) + tableLastPKs := make(map[string]*binlogdatapb.TableLastPK) + for _, tablePK := range uvs.inTablePKs { + tableLastPKs[tablePK.TableName] = tablePK + } + if err := uvs.se.Reload(uvs.ctx); err != nil { + return err + } + tables := uvs.se.GetSchema() + for range tables { + for _, rule := range uvs.filter.Rules { + if !strings.HasPrefix(rule.Match, "/") { + _, ok := tables[rule.Match] + if !ok { + return fmt.Errorf("table %s is not present in the database", rule.Match) + } + } + } + } + for tableName := range tables { + rule, err := matchTable(tableName, uvs.filter, tables) + if err != nil { + return err + } + if rule == nil { + continue + } + plan := &tablePlan{ + tablePK: nil, + rule: &binlogdatapb.Rule{ + Filter: rule.Filter, + Match: rule.Match, + }, + } + tablePK, ok := tableLastPKs[tableName] + if !ok { + tablePK = &binlogdatapb.TableLastPK{ + TableName: tableName, + Lastpk: nil, + } + } + plan.tablePK = tablePK + uvs.plans[tableName] = plan + uvs.tablesToCopy = append(uvs.tablesToCopy, tableName) + + } + sort.Strings(uvs.tablesToCopy) + return nil +} + +// check which rule matches table, validate table is in schema +func matchTable(tableName string, filter *binlogdatapb.Filter, tables map[string]*schema.Table) (*binlogdatapb.Rule, error) { + if tableName == "dual" { + return nil, nil + } + found := false + for _, rule := range filter.Rules { + + switch { + case tableName == rule.Match: + found = true + case strings.HasPrefix(rule.Match, "/"): + expr := strings.Trim(rule.Match, "/") + result, err := regexp.MatchString(expr, tableName) + if err != nil { + return nil, err + } + if !result { + continue + } + found = true + } + if found { + return &binlogdatapb.Rule{ + Match: tableName, + Filter: getQuery(tableName, rule.Filter), + }, nil + } + } + + return nil, nil +} + +// generate equivalent select statement if filter is empty or a keyrange. +func getQuery(tableName string, filter string) string { + query := filter + switch { + case filter == "": + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select * from %v", sqlparser.NewTableIdent(tableName)) + query = buf.String() + case key.IsKeyRange(filter): + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select * from %v where in_keyrange(%v)", sqlparser.NewTableIdent(tableName), sqlparser.NewStrVal([]byte(filter))) + query = buf.String() + } + return query +} + +func (uvs *uvstreamer) Cancel() { + log.Infof("uvstreamer context is being cancelled") + uvs.cancel() +} + +// during copy phase only send streaming events (during catchup/fastforward) for pks already seen +func (uvs *uvstreamer) filterEvents(evs []*binlogdatapb.VEvent) []*binlogdatapb.VEvent { + if len(uvs.plans) == 0 { + return evs + } + var evs2 []*binlogdatapb.VEvent + var tableName string + var shouldSend bool + + for _, ev := range evs { + shouldSend = false + tableName = "" + switch ev.Type { + case binlogdatapb.VEventType_ROW: + tableName = ev.RowEvent.TableName + case binlogdatapb.VEventType_FIELD: + tableName = ev.FieldEvent.TableName + case binlogdatapb.VEventType_HEARTBEAT: + shouldSend = false + default: + shouldSend = true + } + if !shouldSend && tableName != "" { + shouldSend = true + _, ok := uvs.plans[tableName] + if ok { + shouldSend = false + } + } + if shouldSend { + evs2 = append(evs2, ev) + } + } + return evs2 +} + +// wraps the send parameter and filters events. called by fastforward/catchup +func (uvs *uvstreamer) send2(evs []*binlogdatapb.VEvent) error { + if len(evs) == 0 { + return nil + } + ev := evs[len(evs)-1] + if ev.Timestamp != 0 { + uvs.lastTimestampNs = ev.Timestamp * 1e9 + } + behind := time.Now().UnixNano() - uvs.lastTimestampNs + uvs.setSecondsBehindMaster(behind / 1e9) + //log.Infof("sbm set to %d", uvs.secondsBehindMaster) + var evs2 []*binlogdatapb.VEvent + if len(uvs.plans) > 0 { + evs2 = uvs.filterEvents(evs) + } + err := uvs.send(evs2) + if err != nil && err != io.EOF { + return err + } + for _, ev := range evs2 { + if ev.Type == binlogdatapb.VEventType_GTID { + uvs.pos, _ = mysql.DecodePosition(ev.Gtid) + if !uvs.stopPos.IsZero() && uvs.pos.AtLeast(uvs.stopPos) { + err = io.EOF + } + } + } + return err +} + +func (uvs *uvstreamer) sendEventsForCurrentPos() error { + log.Infof("sendEventsForCurrentPos") + vevents := []*binlogdatapb.VEvent{{ + Type: binlogdatapb.VEventType_GTID, + Gtid: mysql.EncodePosition(uvs.pos), + }, { + Type: binlogdatapb.VEventType_OTHER, + }} + if err := uvs.send(vevents); err != nil { + return wrapError(err, uvs.pos) + } + return nil +} + +func (uvs *uvstreamer) setStreamStartPosition() error { + curPos, err := uvs.currentPosition() + if err != nil { + return vterrors.Wrap(err, "could not obtain current position") + } + if uvs.startPos == "current" { + uvs.pos = curPos + if err := uvs.sendEventsForCurrentPos(); err != nil { + return err + } + return nil + } + pos, err := mysql.DecodePosition(uvs.startPos) + if err != nil { + return vterrors.Wrap(err, "could not decode position") + } + if !curPos.AtLeast(pos) { + return fmt.Errorf("requested position %v is ahead of current position %v", mysql.EncodePosition(pos), mysql.EncodePosition(curPos)) + } + uvs.pos = pos + return nil +} + +func (uvs *uvstreamer) currentPosition() (mysql.Position, error) { + conn, err := uvs.cp.Connect(uvs.ctx) + if err != nil { + return mysql.Position{}, err + } + defer conn.Close() + return conn.MasterPosition() +} + +func (uvs *uvstreamer) init() error { + if uvs.startPos != "" { + if err := uvs.setStreamStartPosition(); err != nil { + return err + } + } else if uvs.startPos == "" || len(uvs.inTablePKs) > 0 { + if err := uvs.buildTablePlan(); err != nil { + return err + } + } + + if uvs.pos.IsZero() && (len(uvs.plans) == 0) { + return fmt.Errorf("stream needs a position or a table to copy") + } + return nil +} + +// Stream streams binlog events. +func (uvs *uvstreamer) Stream() error { + log.Info("Stream() called") + if err := uvs.init(); err != nil { + return err + } + if len(uvs.plans) > 0 { + log.Info("TablePKs is not nil: starting vs.copy()") + if err := uvs.copy(uvs.ctx); err != nil { + log.Infof("uvstreamer.Stream() copy returned with err %s", err) + return err + } + uvs.sendTestEvent("Copy Done") + } + log.V(2).Infof("Starting replicate in uvstreamer.Stream()") + vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, uvs.sh, mysql.EncodePosition(uvs.pos), mysql.EncodePosition(uvs.stopPos), uvs.filter, uvs.vschema, uvs.send) + uvs.vs = vs + return vs.Stream() +} + +// SetVSchema updates the vstreamer against the new vschema. +func (uvs *uvstreamer) SetVSchema(vschema *localVSchema) { + uvs.vschema = vschema + if uvs.vs != nil { + uvs.vs.SetVSchema(vschema) + } +} + +func (uvs *uvstreamer) setCopyState(tableName string, qr *querypb.QueryResult) { + uvs.plans[tableName].tablePK.Lastpk = qr +} + +// dummy event sent only in test mode +func (uvs *uvstreamer) sendTestEvent(msg string) { + if !uvstreamerTestMode { + return + } + ev := &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_OTHER, + Gtid: msg, + } + if err := uvs.send([]*binlogdatapb.VEvent{ev}); err != nil { + return + } +} + +func (uvs *uvstreamer) copyComplete(tableName string) error { + evs := []*binlogdatapb.VEvent{ + {Type: binlogdatapb.VEventType_BEGIN}, + { + Type: binlogdatapb.VEventType_LASTPK, + LastPKEvent: &binlogdatapb.LastPKEvent{ + TableLastPK: &binlogdatapb.TableLastPK{ + TableName: tableName, + Lastpk: nil, + }, + Completed: true, + }, + }, + {Type: binlogdatapb.VEventType_COMMIT}, + } + if err := uvs.send(evs); err != nil { + return err + } + + delete(uvs.plans, tableName) + uvs.tablesToCopy = uvs.tablesToCopy[1:] + return nil +} + +func (uvs *uvstreamer) setPosition(gtid string, isInTx bool) error { + if gtid == "" { + return fmt.Errorf("empty gtid passed to setPosition") + } + pos, err := mysql.DecodePosition(gtid) + if err != nil { + return err + } + if pos.Equal(uvs.pos) { + return nil + } + gtidEvent := &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_GTID, + Gtid: gtid, + } + + var evs []*binlogdatapb.VEvent + if !isInTx { + evs = append(evs, &binlogdatapb.VEvent{Type: binlogdatapb.VEventType_BEGIN}) + } + evs = append(evs, gtidEvent) + if !isInTx { + evs = append(evs, &binlogdatapb.VEvent{Type: binlogdatapb.VEventType_COMMIT}) + } + if err := uvs.send(evs); err != nil { + return err + } + uvs.pos = pos + return nil +} + +func (uvs *uvstreamer) getSecondsBehindMaster() int64 { + uvs.mu.Lock() + defer uvs.mu.Unlock() + return uvs.secondsBehindMaster +} + +func (uvs *uvstreamer) setSecondsBehindMaster(sbm int64) { + uvs.mu.Lock() + defer uvs.mu.Unlock() + uvs.secondsBehindMaster = sbm +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_test.go new file mode 100644 index 00000000000..0ba97a68153 --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_test.go @@ -0,0 +1,508 @@ +/* +Copyright 2020 The Vitess Authors. + +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. +*/ + +/* +TestVStreamCopyCompleteFlow tests a complete happy VStream Copy flow: copy/catchup/fastforward/replicate +Three tables t1, t2, t3 are copied. Initially 10 (numInitialRows) rows are inserted into each. +To avoid races in testing we send additional events when *uvstreamerTestMode* is set to true. These are used in +conjunction with callbacks to do additional crud at precise points of the flow to test the different paths +We intercept the vstreamer send callback to look for specific events and invoke these test callbacks. +Fast forward requires tables to be locked briefly to get a snapshot: the test uses this knowledge to hold a lock +on the table in order to insert rows for fastforward to find. + +The flow is as follows: + t1: copy phase, 10 rows. + The lastpk event is intercepted + A row is inserted into t1 to be found in catchup + A row is inserted into t2 which will be an empty transaction during t1 catchup but will be found in t2 copy + t1/t2: catchup phase finds inserted row in t1 + t2: copy phase to start. Test event is sent, intercepted, we lock t2 to block t2's copy, and a row is inserted into t1 and then unlock + t2: fastforward finds t1 event + t2: copy starts + t2: copy complete + t3: copy phase to start. Test event is sent, intercepted, we lock t3 to block t3's copy, and two rows are, one each into t1 and t2 and then unlock + t3: fastforward finds t1 and t2 events + t3: copy starts + t3: copy complete, all tables copied, Copy Complete test event sent, insert 3 rows, one each into t1/t2/t3 + replicate: finds the 3 inserts, context is cancelled after last expected row callback and its commit, vstream context is cancelled +*/ + +package vstreamer + +import ( + "context" + "fmt" + "regexp" + "strings" + "testing" + "time" + + "vitess.io/vitess/go/vt/dbconfigs" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/proto/query" + + "github.com/stretchr/testify/require" + "vitess.io/vitess/go/vt/log" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +const ( + createTableQuery = "create table %s(id%d1 int, id%d2 int, primary key(id%d1))" + bulkInsertQuery = "insert into %s (id%d1, id%d2) values " + insertQuery = "insert into %s (id%d1, id%d2) values (%d, %d)" + numInitialRows = 10 +) + +type state struct { + tables []string +} + +var testState = &state{} + +var positions map[string]string +var allEvents []*binlogdatapb.VEvent + +var callbacks map[string]func() + +func TestVStreamCopyFilterValidations(t *testing.T) { + if testing.Short() { + t.Skip() + } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + defer execStatements(t, []string{ + "drop table t1", + "drop table t2a", + "drop table t2b", + }) + + execStatements(t, []string{ + "create table t1(id11 int, id12 int, primary key(id11))", + "create table t2a(id21 int, id22 int, primary key(id21))", + "create table t2b(id21 int, id22 int, primary key(id21))", + }) + + var getUVStreamer = func(filter *binlogdatapb.Filter, tablePKs []*binlogdatapb.TableLastPK) *uvstreamer { + uvs := &uvstreamer{ + ctx: ctx, + cancel: cancel, + vse: nil, + send: nil, + cp: dbconfigs.Connector{}, + se: engine.se, + sh: engine.sh, + startPos: "", + filter: filter, + vschema: nil, + config: nil, + inTablePKs: tablePKs, + } + return uvs + } + var testFilter = func(rules []*binlogdatapb.Rule, tablePKs []*binlogdatapb.TableLastPK, expected []string, err string) { + uvs := getUVStreamer(&binlogdatapb.Filter{Rules: rules}, tablePKs) + if err == "" { + require.NoError(t, uvs.init()) + } else { + require.Error(t, uvs.init(), err) + return + } + require.Equal(t, len(expected), len(uvs.plans)) + for _, tableName := range expected { + require.True(t, uvs.plans[tableName].tablePK.TableName == tableName) + if tablePKs == nil { + require.Nil(t, uvs.plans[tableName].tablePK.Lastpk) + } + } + for _, pk := range tablePKs { + require.Equal(t, uvs.plans[pk.TableName].tablePK, pk) + } + } + + type TestCase struct { + rules []*binlogdatapb.Rule + tablePKs []*binlogdatapb.TableLastPK + expected []string + err string + } + + var testCases []*TestCase + + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "t1"}}, nil, []string{"t1"}, ""}) + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "t2a"}, {Match: "t1"}}, nil, []string{"t1", "t2a"}, ""}) + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "/.*"}}, nil, []string{"t1", "t2a", "t2b"}, ""}) + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "/t2.*"}}, nil, []string{"t2a", "t2b"}, ""}) + + tablePKs := []*binlogdatapb.TableLastPK{{ + TableName: "t1", + Lastpk: getQRFromLastPK([]*query.Field{{Name: "id11", Type: query.Type_INT32}}, []sqltypes.Value{sqltypes.NewInt32(10)}), + }} + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "t1"}}, tablePKs, []string{"t1"}, ""}) + + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "/.*"}, {Match: "xyz"}}, nil, []string{""}, "table xyz is not present in the database"}) + testCases = append(testCases, &TestCase{[]*binlogdatapb.Rule{{Match: "/x.*"}}, nil, []string{""}, "stream needs a position or a table to copy"}) + + for _, tc := range testCases { + log.Infof("Running %v", tc.rules) + testFilter(tc.rules, tc.tablePKs, tc.expected, tc.err) + } +} + +func TestVStreamCopyCompleteFlow(t *testing.T) { + if testing.Short() { + t.Skip() + } + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + defer execStatements(t, []string{ + "drop table t1", + "drop table t2", + "drop table t3", + }) + + uvstreamerTestMode = true + defer func() { uvstreamerTestMode = false }() + initialize(t) + if err := engine.se.Reload(context.Background()); err != nil { + t.Fatal("Error reloading schema") + } + + var rules []*binlogdatapb.Rule + var tablePKs []*binlogdatapb.TableLastPK + for i, table := range testState.tables { + rules = append(rules, getRule(table)) + tablePKs = append(tablePKs, getTablePK(table, i+1)) + } + filter := &binlogdatapb.Filter{ + Rules: rules, + } + + // Test event called after t1 copy is complete + callbacks["OTHER.*Copy Start t2"] = func() { + conn, err := env.Mysqld.GetDbaConnection(ctx) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + log.Info("Inserting row for fast forward to find, locking t2") + conn.ExecuteFetch("lock tables t2 write", 1, false) + insertRow(t, "t1", 1, numInitialRows+2) + log.Infof("Position after second insert into t1: %s", masterPosition(t)) + conn.ExecuteFetch("unlock tables", 1, false) + log.Info("Inserted row for fast forward to find, unlocked tables") + + } + + callbacks["OTHER.*Copy Start t3"] = func() { + conn, err := env.Mysqld.GetDbaConnection(ctx) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + log.Info("Inserting row for fast forward to find, locking t3") + conn.ExecuteFetch("lock tables t3 write", 1, false) + insertRow(t, "t1", 1, numInitialRows+3) + insertRow(t, "t2", 2, numInitialRows+2) + log.Infof("Position after third insert into t1: %s", masterPosition(t)) + conn.ExecuteFetch("unlock tables", 1, false) + log.Info("Inserted rows for fast forward to find, unlocked tables") + + } + + callbacks["OTHER.*Copy Done"] = func() { + log.Info("Copy done, inserting events to stream") + insertRow(t, "t1", 1, numInitialRows+4) + insertRow(t, "t2", 2, numInitialRows+3) + insertRow(t, "t3", 3, numInitialRows+2) + insertRow(t, "t3", 3, numInitialRows+3) + } + + numCopyEvents := 3 /*t1,t2,t3*/ * (numInitialRows + 1 /*FieldEvent*/ + 1 /*LastPKEvent*/ + 1 /*TestEvent: Copy Start*/ + 2 /*begin,commit*/ + 3 /* LastPK Completed*/) + numCopyEvents += 2 /* GTID + Test event after all copy is done */ + numCatchupEvents := 3 * 5 /*2 t1, 1 t2 : BEGIN+FIELD+ROW+GTID+COMMIT*/ + numFastForwardEvents := 5 /*t1:FIELD+ROW*/ + numIgnored := 3 // empty events + numMisc := 1 /* t2 insert during t1 catchup that comes in t2 copy */ + numReplicateEvents := 3*5 /* insert into t1/t2/t3 */ + 4 /* second insert into t3, no FieldEvent */ + numExpectedEvents := numCopyEvents + numCatchupEvents + numFastForwardEvents + numIgnored + numMisc + numReplicateEvents + + var lastRowEventSeen bool + + callbacks["ROW.*t3.*13390"] = func() { + log.Infof("Saw last row event") + lastRowEventSeen = true + } + + callbacks["COMMIT"] = func() { + log.Infof("Got commit, lastRowSeen is %t", lastRowEventSeen) + if lastRowEventSeen { + log.Infof("Found last row event, canceling context") + cancel() + } + } + + startVStreamCopy(ctx, t, filter, tablePKs) + + select { + case <-time.After(5 * time.Second): + printAllEvents("Timed out") + t.Fatal("Timed out waiting for events") + case <-ctx.Done(): + log.Infof("Received context.Done, ending test") + } + + printAllEvents("End of test") + if len(allEvents) != numExpectedEvents { + log.Errorf("Received %d events, expected %d", len(allEvents), numExpectedEvents) + for _, ev := range allEvents { + log.Errorf("\t%s", ev) + } + t.Fatalf("Received %d events, expected %d", len(allEvents), numExpectedEvents) + } else { + log.Infof("Successfully received %d events", numExpectedEvents) + } + validateReceivedEvents(t) + +} + +func validateReceivedEvents(t *testing.T) { + if len(allEvents) != len(expectedEvents) { + t.Fatalf("Received events not equal to expected events, wanted %d, got %d", len(expectedEvents), len(allEvents)) + } + for i, ev := range allEvents { + ev.Timestamp = 0 + got := ev.String() + want := expectedEvents[i] + if !strings.HasPrefix(got, want) { + t.Fatalf("Event %d did not match, want %s, got %s", i, want, got) + } + } +} + +func insertMultipleRows(t *testing.T, table string, idx int, numRows int) { + query1 := fmt.Sprintf(bulkInsertQuery, table, idx, idx) + s := "" + for i := 1; i <= numRows; i++ { + if s != "" { + s += "," + } + s += fmt.Sprintf("(%d,%d)", i, i*idx*10) + } + query1 += s + execStatement(t, query1) +} + +func initTables(t *testing.T, tables []string) { + var idx int + positions["start"] = masterPosition(t) + for i, table := range tables { + idx = i + 1 + execStatement(t, fmt.Sprintf(createTableQuery, table, idx, idx, idx)) + } + for i, table := range tables { + tableName := table + idx = i + 1 + insertMultipleRows(t, table, idx, numInitialRows) + positions[fmt.Sprintf("%sBulkInsert", table)] = masterPosition(t) + + callbacks[fmt.Sprintf("LASTPK.*%s.*%d", table, numInitialRows)] = func() { + if tableName == "t1" { + insertRow(t, "t1", 1, numInitialRows+1) + //should result in empty commit ignored during catchup since t2 copy has not started + insertRow(t, "t2", 2, numInitialRows+1) + log.Infof("Position after first insert into t1 (and t2/t3): %s", masterPosition(t)) + } + } + } + positions["afterInitialInsert"] = masterPosition(t) +} + +func initialize(t *testing.T) { + callbacks = make(map[string]func()) + testState.tables = []string{"t1", "t2", "t3"} + positions = make(map[string]string) + initTables(t, testState.tables) + callbacks["gtid.*"+positions["afterInitialInsert"]] = func() { + log.Infof("Callback: afterInitialInsert") + } +} + +func getRule(table string) *binlogdatapb.Rule { + return &binlogdatapb.Rule{ + Match: table, + Filter: fmt.Sprintf("select * from %s", table), + } +} + +func getTablePK(table string, idx int) *binlogdatapb.TableLastPK { + fields := []*query.Field{{Name: fmt.Sprintf("id%d1", idx), Type: query.Type_INT32}} + + lastPK := []sqltypes.Value{sqltypes.NewInt32(0)} + return &binlogdatapb.TableLastPK{ + TableName: table, + Lastpk: getQRFromLastPK(fields, lastPK), + } +} + +func insertRow(t *testing.T, table string, idx int, id int) { + execStatement(t, fmt.Sprintf(insertQuery, table, idx, idx, id, id*idx*10)) +} + +func printAllEvents(msg string) { + log.Infof("%s: Received %d events", msg, len(allEvents)) + for _, ev := range allEvents { + log.Infof("\t%s", ev) + } +} + +func getEventCallback(event *binlogdatapb.VEvent) func() { + s := fmt.Sprintf("%v", event) + for key, cb := range callbacks { + match := regexp.MustCompile(".*" + key + ".*") + if key == s || match.MatchString(s) { + return cb + } + } + return nil +} + +func startVStreamCopy(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter, tablePKs []*binlogdatapb.TableLastPK) { + pos := "" + go func() { + err := engine.Stream(ctx, pos, tablePKs, filter, func(evs []*binlogdatapb.VEvent) error { + //t.Logf("Received events: %v", evs) + for _, ev := range evs { + if ev.Type == binlogdatapb.VEventType_HEARTBEAT { + continue + } + cb := getEventCallback(ev) + if cb != nil { + cb() + } + allEvents = append(allEvents, ev) + } + return nil + }) + require.Nil(t, err) + }() +} + +var expectedEvents = []string{ + "type:OTHER gtid:\"Copy Start t1\"", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:GTID", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT", + "type:BEGIN", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", //empty commit for insert into t3 + "type:GTID", + "type:COMMIT", + "type:OTHER gtid:\"Copy Start t2\"", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT", + "type:BEGIN", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT", + "type:OTHER gtid:\"Copy Start t3\"", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT", + "type:BEGIN", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT", + "type:OTHER gtid:\"Copy Done\"", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", + "type:BEGIN", + "type:ROW row_event: > > ", + "type:GTID", + "type:COMMIT", +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index 2373f79cc46..96c603fa634 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -31,7 +31,6 @@ import ( "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -73,10 +72,14 @@ type vstreamer struct { versionTableID uint64 // format and pos are updated by parseEvent. - format mysql.BinlogFormat - pos mysql.Position + format mysql.BinlogFormat + pos mysql.Position + stopPos string } +// CopyState contains the last PK for tables to be copied +type CopyState map[string][]*sqltypes.Result + // streamerPlan extends the original plan to also include // the TableMap, which comes from the binlog. It's used // to extract values from the ROW events. @@ -103,9 +106,9 @@ type streamerPlan struct { // Other constructs like joins, group by, etc. are not supported. // vschema: the current vschema. This value can later be changed through the SetVSchema method. // send: callback function to send events. -func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, sh schema.Historian, startPos string, - filter *binlogdatapb.Filter, vschema *localVSchema, send func([]*binlogdatapb.VEvent) error) *vstreamer { +func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, sh schema.Historian, startPos string, stopPos string, filter *binlogdatapb.Filter, vschema *localVSchema, send func([]*binlogdatapb.VEvent) error) *vstreamer { ctx, cancel := context.WithCancel(ctx) + //init copy state return &vstreamer{ ctx: ctx, cancel: cancel, @@ -113,6 +116,7 @@ func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine se: se, sh: sh, startPos: startPos, + stopPos: stopPos, filter: filter, send: send, vevents: make(chan *localVSchema, 1), @@ -127,48 +131,37 @@ func (vs *vstreamer) SetVSchema(vschema *localVSchema) { // that thread, which helps us avoid mutexes to update the plans. select { case vs.vevents <- vschema: + log.Infof("VSchema sent to vs.vevents") case <-vs.ctx.Done(): + log.Infof("ctx.Done() in setVSchema") } } // Cancel stops the streaming. func (vs *vstreamer) Cancel() { + log.Infof("vstreamer context is being cancelled") vs.cancel() } // Stream streams binlog events. func (vs *vstreamer) Stream() error { - defer vs.cancel() - - // Validate the request against the current position. - curPos, err := vs.currentPosition() + //defer vs.cancel() + ctx := context.Background() + defer ctx.Done() + log.Infof("Starting Stream() with startPos %s", vs.startPos) + pos, err := mysql.DecodePosition(vs.startPos) if err != nil { - return vterrors.Wrap(err, "could not obtain current position") - } - if vs.startPos == "current" { - vs.pos = curPos - vevents := []*binlogdatapb.VEvent{{ - Type: binlogdatapb.VEventType_GTID, - Gtid: mysql.EncodePosition(vs.pos), - }, { - Type: binlogdatapb.VEventType_OTHER, - }} - if err := vs.send(vevents); err != nil { - return wrapError(err, vs.pos) - } - } else { - pos, err := mysql.DecodePosition(vs.startPos) - if err != nil { - return vterrors.Wrap(err, "could not decode position") - } - if !curPos.AtLeast(pos) { - return fmt.Errorf("requested position %v is ahead of current position %v", mysql.EncodePosition(pos), mysql.EncodePosition(curPos)) - } - vs.pos = pos + return err } + vs.pos = pos + return vs.replicate(ctx) +} +// Stream streams binlog events. +func (vs *vstreamer) replicate(ctx context.Context) error { + log.Infof("In replicate with pos %s", vs.pos) // Ensure sh is Open. If vttablet came up in a non_serving role, - // the schema engine may not have been initialized. + // the historian may not have been initialized. if err := vs.sh.Open(); err != nil { return wrapError(err, vs.pos) } @@ -187,17 +180,9 @@ func (vs *vstreamer) Stream() error { return wrapError(err, vs.pos) } -func (vs *vstreamer) currentPosition() (mysql.Position, error) { - conn, err := vs.cp.Connect(vs.ctx) - if err != nil { - return mysql.Position{}, err - } - defer conn.Close() - return conn.MasterPosition() -} - // parseEvents parses and sends events. func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.BinlogEvent) error { + log.Infof("In parse events") // bufferAndTransmit uses bufferedEvents and curSize to buffer events. var ( bufferedEvents []*binlogdatapb.VEvent @@ -294,6 +279,7 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog } vevents, err := vs.parseEvent(ev) if err != nil { + log.Infof("parseEvent returned error %v", err) return err } for _, vevent := range vevents { @@ -305,7 +291,9 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog } } case vs.vschema = <-vs.vevents: + log.Infof("Received vschema in vs.vevents") if err := vs.rebuildPlans(); err != nil { + log.Infof("Error rebuilding plans %v", err) return err } // Increment this counter for testing. @@ -381,7 +369,7 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e Type: binlogdatapb.VEventType_BEGIN, }) } - vs.pos = mysql.AppendGTID(vs.pos, gtid) + vs.pos = mysql.AppendGTID(vs.pos, gtid) //TODO: #sugu why Append? case ev.IsXID(): vevents = append(vevents, &binlogdatapb.VEvent{ Type: binlogdatapb.VEventType_GTID, diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index a318aca681e..83ecf289db8 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -19,17 +19,20 @@ package vstreamer import ( "context" "fmt" + "io" "strconv" "strings" + "sync" "testing" "time" + "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" + + "vitess.io/vitess/go/vt/log" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" - binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) @@ -73,10 +76,217 @@ func TestVersion(t *testing.T) { `gtid`, `commit`}}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) assert.Equal(t, 1, numVersionEventsReceived) } +func insertLotsOfData(t *testing.T, numRows int) { + query1 := "insert into t1 (id11, id12) values" + s := "" + for i := 1; i <= numRows; i++ { + if s != "" { + s += "," + } + s += fmt.Sprintf("(%d,%d)", i, i*10) + } + query1 += s + query2 := "insert into t2 (id21, id22) values" + s = "" + for i := 1; i <= numRows; i++ { + if s != "" { + s += "," + } + s += fmt.Sprintf("(%d,%d)", i, i*20) + } + query2 += s + execStatements(t, []string{ + query1, + query2, + }) +} + +func TestVStreamCopySimpleFlow(t *testing.T) { + if testing.Short() { + t.Skip() + } + execStatements(t, []string{ + "create table t1(id11 int, id12 int, primary key(id11))", + "create table t2(id21 int, id22 int, primary key(id21))", + }) + log.Infof("Pos before bulk insert: %s", masterPosition(t)) + insertLotsOfData(t, 10) + log.Infof("Pos after bulk insert: %s", masterPosition(t)) + defer execStatements(t, []string{ + "drop table t1", + "drop table t2", + }) + engine.se.Reload(context.Background()) + ctx := context.Background() + qr, err := env.Mysqld.FetchSuperQuery(ctx, "SELECT count(*) as cnt from t1, t2 where t1.id11 = t2.id21") + if err != nil { + t.Fatal("Query failed") + } + require.Equal(t, "[[INT64(10)]]", fmt.Sprintf("%v", qr.Rows)) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1", + }, { + Match: "t2", + Filter: "select * from t2", + }}, + } + + var tablePKs []*binlogdatapb.TableLastPK + tablePKs = append(tablePKs, getTablePK("t1", 1)) + tablePKs = append(tablePKs, getTablePK("t2", 2)) + + t1FieldEvent := []string{"begin", "type:FIELD field_event: fields: > "} + t2FieldEvent := []string{"begin", "type:FIELD field_event: fields: > "} + t1Events := []string{} + t2Events := []string{} + for i := 1; i <= 10; i++ { + t1Events = append(t1Events, + fmt.Sprintf("type:ROW row_event: > > ", len(strconv.Itoa(i)), len(strconv.Itoa(i*10)), i, i*10)) + t2Events = append(t2Events, + fmt.Sprintf("type:ROW row_event: > > ", len(strconv.Itoa(i)), len(strconv.Itoa(i*20)), i, i*20)) + } + t1Events = append(t1Events, "lastpk", "commit") + t2Events = append(t2Events, "lastpk", "commit") + + insertEvents1 := []string{ + "begin", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "gtid", + "commit"} + insertEvents2 := []string{ + "begin", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "gtid", + "commit"} + + testcases := []testcase{ + { + input: []string{}, + output: [][]string{t1FieldEvent, {"gtid"}, t1Events, {"begin", "lastpk", "commit"}, t2FieldEvent, t2Events, {"begin", "lastpk", "commit"}}, + }, + + { + input: []string{ + "insert into t1 values (101, 1010)", + }, + output: [][]string{insertEvents1}, + }, + { + input: []string{ + "insert into t2 values (202, 2020)", + }, + output: [][]string{insertEvents2}, + }, + } + + runCases(t, filter, testcases, "vscopy", tablePKs) + log.Infof("Pos at end of test: %s", masterPosition(t)) +} + +func TestVStreamCopyWithDifferentFilters(t *testing.T) { + if testing.Short() { + t.Skip() + } + execStatements(t, []string{ + "create table t1(id1 int, id2 int, id3 int, primary key(id1))", + "create table t2a(id1 int, id2 int, primary key(id1))", + "create table t2b(id1 varchar(20), id2 int, primary key(id1))", + }) + defer execStatements(t, []string{ + "drop table t1", + "drop table t2a", + "drop table t2b", + }) + engine.se.Reload(context.Background()) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/t2.*", + }, { + Match: "t1", + Filter: "select id1, id2 from t1", + }}, + } + + execStatements(t, []string{ + "insert into t1(id1, id2, id3) values (1, 2, 3)", + "insert into t2a(id1, id2) values (1, 4)", + "insert into t2b(id1, id2) values ('b', 6)", + "insert into t2b(id1, id2) values ('a', 5)", + }) + + var expectedEvents = []string{ + "type:BEGIN ", + "type:FIELD field_event: fields: > ", + "type:GTID", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT ", + "type:BEGIN ", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT ", + "type:BEGIN ", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT ", + "type:BEGIN ", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT ", + "type:BEGIN ", + "type:FIELD field_event: fields: > ", + "type:ROW row_event: > > ", + "type:ROW row_event: > > ", + "type:LASTPK last_p_k_event: > > > ", + "type:COMMIT ", + "type:BEGIN ", + "type:LASTPK last_p_k_event: completed:true > ", + "type:COMMIT ", + } + + var allEvents []*binlogdatapb.VEvent + var wg sync.WaitGroup + wg.Add(1) + ctx2, cancel2 := context.WithDeadline(ctx, time.Now().Add(10*time.Second)) + defer cancel2() + go func() { + defer wg.Done() + engine.Stream(ctx2, "", nil, filter, func(evs []*binlogdatapb.VEvent) error { + for _, ev := range evs { + if ev.Type == binlogdatapb.VEventType_HEARTBEAT { + continue + } + allEvents = append(allEvents, ev) + } + if len(allEvents) == len(expectedEvents) { + log.Infof("Got %d events as expected", len(allEvents)) + for i, ev := range allEvents { + ev.Timestamp = 0 + got := ev.String() + want := expectedEvents[i] + if !strings.HasPrefix(got, want) { + t.Fatalf("Event %d did not match, want %s, got %s", i, want, got) + } + } + + return io.EOF + } + return nil + }) + }() + wg.Wait() +} + func TestFilteredVarBinary(t *testing.T) { if testing.Short() { t.Skip() @@ -127,7 +337,7 @@ func TestFilteredVarBinary(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } func TestFilteredInt(t *testing.T) { @@ -180,7 +390,7 @@ func TestFilteredInt(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } func TestStatements(t *testing.T) { @@ -260,7 +470,7 @@ func TestStatements(t *testing.T) { `type:DDL ddl:"truncate table stream2" `, }}, }} - runCases(t, nil, testcases, "current") + runCases(t, nil, testcases, "current", nil) // Test FilePos flavor savedEngine := engine @@ -270,7 +480,7 @@ func TestStatements(t *testing.T) { return in }) defer engine.Close() - runCases(t, nil, testcases, "current") + runCases(t, nil, testcases, "current", nil) } // TestOther tests "other" and "priv" statements. These statements can @@ -309,7 +519,7 @@ func TestOther(t *testing.T) { t.Logf("Run mode: %v", mode) ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ch := startStream(ctx, t, nil, "") + ch := startStream(ctx, t, nil, "", nil) want := [][]string{{ `gtid`, @@ -383,13 +593,15 @@ func TestRegexp(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } func TestREKeyRange(t *testing.T) { if testing.Short() { t.Skip() } + // Needed for this test to run if run standalone + engine.watcherOnce.Do(engine.setWatch) execStatements(t, []string{ "create table t1(id1 int, id2 int, val varbinary(128), primary key(id1))", @@ -411,7 +623,7 @@ func TestREKeyRange(t *testing.T) { Filter: "-80", }}, } - ch := startStream(ctx, t, filter, "") + ch := startStream(ctx, t, filter, "", nil) // 1, 2, 3 and 5 are in shard -80. // 4 and 6 are in shard 80-. @@ -501,7 +713,7 @@ func TestInKeyRangeMultiColumn(t *testing.T) { Filter: "select id, region, val, keyspace_id() from t1 where in_keyrange('-80')", }}, } - ch := startStream(ctx, t, filter, "") + ch := startStream(ctx, t, filter, "", nil) // 1, 2, 3 and 5 are in shard -80. // 4 and 6 are in shard 80-. @@ -556,7 +768,7 @@ func TestREMultiColumnVindex(t *testing.T) { Filter: "-80", }}, } - ch := startStream(ctx, t, filter, "") + ch := startStream(ctx, t, filter, "", nil) // 1, 2, 3 and 5 are in shard -80. // 4 and 6 are in shard 80-. @@ -620,7 +832,7 @@ func TestSelectFilter(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } func TestDDLAddColumn(t *testing.T) { @@ -670,7 +882,7 @@ func TestDDLAddColumn(t *testing.T) { ch := make(chan []*binlogdatapb.VEvent) go func() { defer close(ch) - if err := vstream(ctx, t, pos, filter, ch); err != nil { + if err := vstream(ctx, t, pos, nil, filter, ch); err != nil { t.Error(err) } }() @@ -729,7 +941,7 @@ func TestDDLDropColumn(t *testing.T) { } }() defer close(ch) - err := vstream(ctx, t, pos, nil, ch) + err := vstream(ctx, t, pos, nil, nil, ch) want := "cannot determine table columns" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain %s", err, want) @@ -759,7 +971,7 @@ func TestUnsentDDL(t *testing.T) { Match: "/none/", }}, } - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } func TestBuffering(t *testing.T) { @@ -859,7 +1071,7 @@ func TestBuffering(t *testing.T) { `type:DDL ddl:"alter table packet_test change val val varchar(128)" `, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestBestEffortNameInFieldEvent(t *testing.T) { @@ -909,7 +1121,7 @@ func TestBestEffortNameInFieldEvent(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, position) + runCases(t, filter, testcases, position, nil) } func TestTypes(t *testing.T) { @@ -1050,7 +1262,7 @@ func TestTypes(t *testing.T) { `commit`, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestJSON(t *testing.T) { @@ -1079,7 +1291,7 @@ func TestJSON(t *testing.T) { `commit`, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestExternalTable(t *testing.T) { @@ -1109,7 +1321,7 @@ func TestExternalTable(t *testing.T) { `commit`, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestJournal(t *testing.T) { @@ -1148,7 +1360,7 @@ func TestJournal(t *testing.T) { `commit`, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestMinimalMode(t *testing.T) { @@ -1183,7 +1395,7 @@ func TestMinimalMode(t *testing.T) { } }() defer close(ch) - err := vstream(ctx, t, pos, nil, ch) + err := vstream(ctx, t, pos, nil, nil, ch) want := "partial row image encountered" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain '%s'", err, want) @@ -1225,7 +1437,7 @@ func TestStatementMode(t *testing.T) { `commit`, }}, }} - runCases(t, nil, testcases, "") + runCases(t, nil, testcases, "", nil) } func TestHeartbeat(t *testing.T) { @@ -1236,7 +1448,7 @@ func TestHeartbeat(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ch := startStream(ctx, t, nil, "") + ch := startStream(ctx, t, nil, "", nil) evs := <-ch require.Equal(t, 1, len(evs)) assert.Equal(t, binlogdatapb.VEventType_HEARTBEAT, evs[0].Type) @@ -1275,7 +1487,7 @@ func TestNoFutureGTID(t *testing.T) { } }() defer close(ch) - err = vstream(ctx, t, future, nil, ch) + err = vstream(ctx, t, future, nil, nil, ch) want := "is ahead of current position" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain %s", err, want) @@ -1325,14 +1537,14 @@ func TestFilteredMultipleWhere(t *testing.T) { `commit`, }}, }} - runCases(t, filter, testcases, "") + runCases(t, filter, testcases, "", nil) } -func runCases(t *testing.T, filter *binlogdatapb.Filter, testcases []testcase, position string) { +func runCases(t *testing.T, filter *binlogdatapb.Filter, testcases []testcase, position string, tablePK []*binlogdatapb.TableLastPK) { t.Helper() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ch := startStream(ctx, t, filter, position) + ch := startStream(ctx, t, filter, position, tablePK) // If position is 'current', we wait for a heartbeat to be // sure the vstreamer has started. @@ -1401,6 +1613,10 @@ func expectLog(ctx context.Context, t *testing.T, input interface{}, ch <-chan [ if evs[i].Type != binlogdatapb.VEventType_GTID { t.Fatalf("%v (%d): event: %v, want gtid", input, i, evs[i]) } + case "lastpk": + if evs[i].Type != binlogdatapb.VEventType_LASTPK { + t.Fatalf("%v (%d): event: %v, want lastpk", input, i, evs[i]) + } case "commit": if evs[i].Type != binlogdatapb.VEventType_COMMIT { t.Fatalf("%v (%d): event: %v, want commit", input, i, evs[i]) @@ -1417,20 +1633,26 @@ func expectLog(ctx context.Context, t *testing.T, input interface{}, ch <-chan [ var lastPos string -func startStream(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter, position string) <-chan []*binlogdatapb.VEvent { +func startStream(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter, position string, tablePKs []*binlogdatapb.TableLastPK) <-chan []*binlogdatapb.VEvent { if position == "" { position = masterPosition(t) } + if position == "vscopy" { + position = "" + } ch := make(chan []*binlogdatapb.VEvent) go func() { defer close(ch) - _ = vstream(ctx, t, position, filter, ch) + err := vstream(ctx, t, position, tablePKs, filter, ch) + if len(tablePKs) == 0 { + require.Nil(t, err) + } }() return ch } -func vstream(ctx context.Context, t *testing.T, pos string, filter *binlogdatapb.Filter, ch chan []*binlogdatapb.VEvent) error { +func vstream(ctx context.Context, t *testing.T, pos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, ch chan []*binlogdatapb.VEvent) error { if filter == nil { filter = &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ @@ -1438,7 +1660,7 @@ func vstream(ctx context.Context, t *testing.T, pos string, filter *binlogdatapb }}, } } - return engine.Stream(ctx, pos, filter, func(evs []*binlogdatapb.VEvent) error { + return engine.Stream(ctx, pos, tablePKs, filter, func(evs []*binlogdatapb.VEvent) error { if t.Name() == "TestVersion" { // emulate tracker only for the version test for _, ev := range evs { log.Infof("Original stream: %s event found %v", ev.Type, ev) @@ -1451,6 +1673,7 @@ func vstream(ctx context.Context, t *testing.T, pos string, filter *binlogdatapb } } } + t.Logf("Received events: %v", evs) select { case ch <- evs: case <-ctx.Done(): @@ -1498,11 +1721,9 @@ func setVSchema(t *testing.T, vschema string) { t.Helper() curCount := engine.vschemaUpdates.Get() - if err := env.SetVSchema(vschema); err != nil { t.Fatal(err) } - // Wait for curCount to go up. updated := false for i := 0; i < 10; i++ { diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 6f88e03c016..27888108795 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -231,6 +231,7 @@ enum VEventType { VGTID = 15; JOURNAL = 16; VERSION = 17; + LASTPK = 18; } // RowChange represents one row change. @@ -263,6 +264,7 @@ message ShardGtid { string keyspace = 1; string shard = 2; string gtid = 3; + repeated TableLastPK table_p_ks = 4; } // A VGtid is a list of ShardGtids. @@ -341,6 +343,8 @@ message VEvent { // CurrentTime specifies the current time when the message was sent. // This can be used to compenssate for clock skew. int64 current_time = 20; + // LastPK is the last PK for a table + LastPKEvent last_p_k_event = 21; } message MinimalTable { @@ -361,6 +365,7 @@ message VStreamRequest { string position = 4; Filter filter = 5; + repeated TableLastPK table_last_p_ks = 6; } // VStreamResponse is the response from VStreamer @@ -387,6 +392,16 @@ message VStreamRowsResponse { query.Row lastpk = 5; } +message LastPKEvent { + TableLastPK table_last_p_k = 1; + bool completed = 2; +} + +message TableLastPK { + string table_name = 1; + query.QueryResult lastpk = 3; +} + // VStreamResultsRequest is the payload for VStreamResults // The ids match VStreamRows, in case we decide to merge the two. // The ids match VStreamRows, in case we decide to merge the two.