From b48a84ca640909fff5a4e004904c489a2779ecdc Mon Sep 17 00:00:00 2001 From: Alain Jobart Date: Fri, 2 Sep 2016 10:07:36 -0700 Subject: [PATCH] Adding UpdateStream API to vtgate. The vtgate API takes a starting timestamp, or a starting EventToken. It will only use the starting EventToken if it's relevant. This is mostly for tests, but could be used by real clients too to avoid the timestamp search on the servers. The only restriction in the vtgate routing implementation is that a query can only end up on one shard. The stream aggregation code inside vtgate will be added later. This change includes: - proto changes. - implementing the server side interface. - implementing the client side interface. - adding a vtctl VtTabletUpdateStream command to stream from a given tablet. This is used in end-to-end tests. - using the python vtgate_client update_stream API in all end-to-end tests. - removing the python vttablet direct stream_update API. - vtgate now better preserves remote errors through its API now, as withSuffix and withPrefix will preserve the error codes of all VtError, not just *VitessError. - Also adding callerid vtgateclienttest tests for all API calls. --- .../vtgateclienttest/goclienttest/callerid.go | 69 ++- go/cmd/vtgateclienttest/goclienttest/echo.go | 34 +- .../vtgateclienttest/goclienttest/errors.go | 26 + go/cmd/vtgateclienttest/services/callerid.go | 7 + go/cmd/vtgateclienttest/services/echo.go | 22 + go/cmd/vtgateclienttest/services/errors.go | 7 + go/cmd/vtgateclienttest/services/fallback.go | 4 + go/cmd/vtgateclienttest/services/terminal.go | 4 + go/vt/proto/vtgate/vtgate.pb.go | 253 ++++++--- go/vt/proto/vtgateservice/vtgateservice.pb.go | 124 ++++- go/vt/vitessdriver/fakeserver_test.go | 5 + go/vt/vtctl/query.go | 53 ++ go/vt/vtgate/fakerpcvtgateconn/conn.go | 5 + go/vt/vtgate/grpcvtgateconn/conn.go | 34 ++ go/vt/vtgate/grpcvtgateservice/server.go | 21 + go/vt/vtgate/resolver.go | 57 +- go/vt/vtgate/scatter_conn.go | 19 + go/vt/vtgate/vtgate.go | 43 +- go/vt/vtgate/vtgateconn/vtgateconn.go | 23 +- go/vt/vtgate/vtgateconntest/client.go | 199 +++++++ go/vt/vtgate/vtgateservice/interface.go | 11 +- .../mock_vtgateservice.go | 10 + .../Automationservice/AutomationClient.php | 4 +- .../Proto/Binlogdata/StreamEvent/Category.php | 13 - .../Proto/Binlogdata/StreamUpdateResponse.php | 74 --- .../Binlogservice/UpdateStreamClient.php | 6 - .../Vitess/Proto/Mysqlctl/MysqlCtlClient.php | 4 +- php/src/Vitess/Proto/Query/StreamEvent.php | 143 +++++ .../StreamEvent/Statement.php} | 89 +-- .../Query/StreamEvent/Statement/Category.php | 12 + .../Proto/Query/UpdateStreamRequest.php | 268 ++++++++++ .../Proto/Query/UpdateStreamResponse.php | 74 +++ .../Vitess/Proto/Queryservice/QueryClient.php | 10 +- .../Proto/Throttlerdata/Configuration.php | 505 ++++++++++++++++++ .../GetConfigurationRequest.php} | 34 +- .../GetConfigurationResponse.php | 94 ++++ .../ConfigurationsEntry.php | 122 +++++ .../ResetConfigurationRequest.php | 73 +++ .../ResetConfigurationResponse.php | 93 ++++ .../UpdateConfigurationRequest.php | 170 ++++++ .../UpdateConfigurationResponse.php | 93 ++++ .../Proto/Vtgate/UpdateStreamRequest.php | 365 +++++++++++++ .../Proto/Vtgate/UpdateStreamResponse.php | 122 +++++ .../Proto/Vtgateservice/VitessClient.php | 6 + php/src/Vitess/Proto/Vttest/Keyspace.php | 96 ++++ .../Vitess/Proto/Vttest/VTTestTopology.php | 68 +++ .../Proto/Vtworkerservice/VtworkerClient.php | 4 +- proto/vtgate.proto | 43 ++ proto/vtgateservice.proto | 4 + py/vtdb/dbexceptions.py | 8 + py/vtdb/grpc_update_stream.py | 63 --- py/vtdb/grpc_vtgate_client.py | 29 + py/vtdb/proto3_encoding.py | 65 +++ py/vtdb/update_stream.py | 120 ----- py/vtdb/vtgate_client.py | 34 ++ py/vtproto/vtgate_pb2.py | 134 ++++- py/vtproto/vtgateservice_pb2.py | 18 +- test/binlog.py | 62 +-- test/grpc_protocols_flavor.py | 4 - test/protocols_flavor.py | 7 - test/tablet.py | 7 - test/update_stream.py | 157 +++--- 62 files changed, 3697 insertions(+), 630 deletions(-) delete mode 100644 php/src/Vitess/Proto/Binlogdata/StreamEvent/Category.php delete mode 100644 php/src/Vitess/Proto/Binlogdata/StreamUpdateResponse.php create mode 100644 php/src/Vitess/Proto/Query/StreamEvent.php rename php/src/Vitess/Proto/{Binlogdata/StreamEvent.php => Query/StreamEvent/Statement.php} (73%) create mode 100644 php/src/Vitess/Proto/Query/StreamEvent/Statement/Category.php create mode 100644 php/src/Vitess/Proto/Query/UpdateStreamRequest.php create mode 100644 php/src/Vitess/Proto/Query/UpdateStreamResponse.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/Configuration.php rename php/src/Vitess/Proto/{Binlogdata/StreamUpdateRequest.php => Throttlerdata/GetConfigurationRequest.php} (56%) create mode 100644 php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse/ConfigurationsEntry.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/ResetConfigurationRequest.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/ResetConfigurationResponse.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationRequest.php create mode 100644 php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationResponse.php create mode 100644 php/src/Vitess/Proto/Vtgate/UpdateStreamRequest.php create mode 100644 php/src/Vitess/Proto/Vtgate/UpdateStreamResponse.php delete mode 100644 py/vtdb/grpc_update_stream.py delete mode 100644 py/vtdb/update_stream.py diff --git a/go/cmd/vtgateclienttest/goclienttest/callerid.go b/go/cmd/vtgateclienttest/goclienttest/callerid.go index ddc120c638..3e88d41107 100644 --- a/go/cmd/vtgateclienttest/goclienttest/callerid.go +++ b/go/cmd/vtgateclienttest/goclienttest/callerid.go @@ -14,7 +14,9 @@ import ( "github.com/youtube/vitess/go/vt/vtgate/vtgateconn" "golang.org/x/net/context" + querypb "github.com/youtube/vitess/go/vt/proto/query" topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" + vtgatepb "github.com/youtube/vitess/go/vt/proto/vtgate" ) // testCallerID adds a caller ID to a context, and makes sure the server @@ -32,12 +34,65 @@ func testCallerID(t *testing.T, conn *vtgateconn.VTGateConn) { } query := services.CallerIDPrefix + string(data) - // test Execute forwards the callerID - if _, err := conn.Execute(ctx, query, nil, topodatapb.TabletType_MASTER); err != nil { - if !strings.Contains(err.Error(), "SUCCESS: ") { - t.Errorf("failed to pass callerid: %v", err) - } - } + // test Execute calls forward the callerID + _, err = conn.Execute(ctx, query, nil, topodatapb.TabletType_MASTER) + checkCallerIDError(t, "Execute", err) - // FIXME(alainjobart) add all function calls + _, err = conn.ExecuteShards(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER) + checkCallerIDError(t, "ExecuteShards", err) + + _, err = conn.ExecuteKeyspaceIds(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER) + checkCallerIDError(t, "ExecuteKeyspaceIds", err) + + _, err = conn.ExecuteKeyRanges(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER) + checkCallerIDError(t, "ExecuteKeyRanges", err) + + _, err = conn.ExecuteEntityIds(ctx, query, "", "", nil, nil, topodatapb.TabletType_MASTER) + checkCallerIDError(t, "ExecuteEntityIds", err) + + // test ExecuteBatch calls forward the callerID + _, err = conn.ExecuteBatchShards(ctx, []*vtgatepb.BoundShardQuery{ + { + Query: &querypb.BoundQuery{ + Sql: query, + }, + }, + }, topodatapb.TabletType_MASTER, false) + checkCallerIDError(t, "ExecuteBatchShards", err) + + _, err = conn.ExecuteBatchKeyspaceIds(ctx, []*vtgatepb.BoundKeyspaceIdQuery{ + { + Query: &querypb.BoundQuery{ + Sql: query, + }, + }, + }, topodatapb.TabletType_MASTER, false) + checkCallerIDError(t, "ExecuteBatchKeyspaceIds", err) + + // test StreamExecute calls forward the callerID + err = getStreamError(conn.StreamExecute(ctx, query, nil, topodatapb.TabletType_MASTER)) + checkCallerIDError(t, "StreamExecute", err) + + err = getStreamError(conn.StreamExecuteShards(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER)) + checkCallerIDError(t, "StreamExecuteShards", err) + + err = getStreamError(conn.StreamExecuteKeyspaceIds(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER)) + checkCallerIDError(t, "StreamExecuteKeyspaceIds", err) + + err = getStreamError(conn.StreamExecuteKeyRanges(ctx, query, "", nil, nil, topodatapb.TabletType_MASTER)) + checkCallerIDError(t, "StreamExecuteKeyRanges", err) + + // test UpdateStream forwards the callerID + err = getUpdateStreamError(conn.UpdateStream(ctx, query, nil, topodatapb.TabletType_MASTER, 0, nil)) + checkCallerIDError(t, "UpdateStream", err) +} + +func checkCallerIDError(t *testing.T, name string, err error) { + if err == nil { + t.Errorf("callerid: got no error for %v", name) + return + } + if !strings.Contains(err.Error(), "SUCCESS: ") { + t.Errorf("failed to pass callerid for %v: %v", name, err) + } } diff --git a/go/cmd/vtgateclienttest/goclienttest/echo.go b/go/cmd/vtgateclienttest/goclienttest/echo.go index b80135fe72..df31c9b59f 100644 --- a/go/cmd/vtgateclienttest/goclienttest/echo.go +++ b/go/cmd/vtgateclienttest/goclienttest/echo.go @@ -37,7 +37,8 @@ var ( keyRanges = []*topodatapb.KeyRange{ {Start: []byte{1, 2, 3, 4}, End: []byte{5, 6, 7, 8}}, } - keyRangesEcho = "[start:\"\\001\\002\\003\\004\" end:\"\\005\\006\\007\\010\" ]" + keyRangeZeroEcho = "start:\"\\001\\002\\003\\004\" end:\"\\005\\006\\007\\010\" " + keyRangesEcho = "[" + keyRangeZeroEcho + "]" entityKeyspaceIDs = []*vtgatepb.ExecuteEntityIdsRequest_EntityId{ { @@ -87,6 +88,10 @@ var ( callerID = callerid.NewEffectiveCallerID("test_principal", "test_component", "test_subcomponent") callerIDEcho = "principal:\"test_principal\" component:\"test_component\" subcomponent:\"test_subcomponent\" " + + timestamp int64 = 876543 + position = "test_position" + updateStreamEcho = "map[callerId:" + callerIDEcho + " event:timestamp:876543 shard:\"-80\" position:\"" + position + "\" keyRange:" + keyRangeZeroEcho + " keyspace:conn_ks shard:echo://test query tabletType:REPLICA timestamp:0]" ) // testEcho exercises the test cases provided by the "echo" service. @@ -95,6 +100,7 @@ func testEcho(t *testing.T, conn *vtgateconn.VTGateConn) { testEchoStreamExecute(t, conn) testEchoTransactionExecute(t, conn) testEchoSplitQuery(t, conn) + testEchoUpdateStream(t, conn) } func testEchoExecute(t *testing.T, conn *vtgateconn.VTGateConn) { @@ -404,9 +410,35 @@ func testEchoSplitQuery(t *testing.T, conn *vtgateconn.VTGateConn) { } } +func testEchoUpdateStream(t *testing.T, conn *vtgateconn.VTGateConn) { + var stream vtgateconn.UpdateStreamReader + var err error + + ctx := callerid.NewContext(context.Background(), callerID, nil) + + stream, err = conn.UpdateStream(ctx, echoPrefix+query, keyRanges[0], tabletType, 0, &querypb.EventToken{ + Timestamp: timestamp, + Shard: shards[0], + Position: position, + }) + if err != nil { + t.Fatal(err) + } + se, _, err := stream.Recv() + if err != nil { + t.Fatal(err) + } + if se.EventToken.Position != updateStreamEcho { + t.Errorf("UpdateStream(0) = %v, want %v", se.EventToken.Position, updateStreamEcho) + } +} + // getEcho extracts the echoed field values from a query result. func getEcho(qr *sqltypes.Result) map[string]sqltypes.Value { values := map[string]sqltypes.Value{} + if qr == nil { + return values + } for i, field := range qr.Fields { values[field.Name] = qr.Rows[0][i] } diff --git a/go/cmd/vtgateclienttest/goclienttest/errors.go b/go/cmd/vtgateclienttest/goclienttest/errors.go index c1819be5bb..595694bc20 100644 --- a/go/cmd/vtgateclienttest/goclienttest/errors.go +++ b/go/cmd/vtgateclienttest/goclienttest/errors.go @@ -40,6 +40,7 @@ func testErrors(t *testing.T, conn *vtgateconn.VTGateConn) { testExecuteErrors(t, conn) testStreamExecuteErrors(t, conn) testTransactionExecuteErrors(t, conn) + testUpdateStreamErrors(t, conn) } func testExecuteErrors(t *testing.T, conn *vtgateconn.VTGateConn) { @@ -110,6 +111,14 @@ func testStreamExecuteErrors(t *testing.T, conn *vtgateconn.VTGateConn) { }) } +func testUpdateStreamErrors(t *testing.T, conn *vtgateconn.VTGateConn) { + ctx := context.Background() + + checkStreamExecuteErrors(t, func(query string) error { + return getUpdateStreamError(conn.UpdateStream(ctx, query, nil, tabletType, 0, nil)) + }) +} + func testTransactionExecuteErrors(t *testing.T, conn *vtgateconn.VTGateConn) { ctx := context.Background() @@ -178,6 +187,23 @@ func getStreamError(stream sqltypes.ResultStream, err error) error { } } +func getUpdateStreamError(stream vtgateconn.UpdateStreamReader, err error) error { + if err != nil { + return err + } + for { + _, _, err := stream.Recv() + switch err { + case nil: + // keep going + case io.EOF: + return nil + default: + return err + } + } +} + func checkExecuteErrors(t *testing.T, execute func(string) error) { for errStr, errCode := range executeErrors { query := errorPrefix + errStr diff --git a/go/cmd/vtgateclienttest/services/callerid.go b/go/cmd/vtgateclienttest/services/callerid.go index a52ff84338..159b93f251 100644 --- a/go/cmd/vtgateclienttest/services/callerid.go +++ b/go/cmd/vtgateclienttest/services/callerid.go @@ -176,3 +176,10 @@ func (c *callerIDClient) SplitQueryV2( numRowsPerQueryPart, algorithm) } + +func (c *callerIDClient) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + if ok, err := c.checkCallerID(ctx, shard); ok { + return err + } + return c.fallbackClient.UpdateStream(ctx, keyspace, shard, keyRange, tabletType, timestamp, event, sendReply) +} diff --git a/go/cmd/vtgateclienttest/services/echo.go b/go/cmd/vtgateclienttest/services/echo.go index 24c88bf630..763b281c85 100644 --- a/go/cmd/vtgateclienttest/services/echo.go +++ b/go/cmd/vtgateclienttest/services/echo.go @@ -325,3 +325,25 @@ func (c *echoClient) SplitQueryV2( numRowsPerQueryPart, algorithm) } + +func (c *echoClient) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + if strings.HasPrefix(shard, EchoPrefix) { + m := map[string]interface{}{ + "callerId": callerid.EffectiveCallerIDFromContext(ctx), + "keyspace": keyspace, + "shard": shard, + "keyRange": keyRange, + "timestamp": timestamp, + "tabletType": tabletType, + "event": event, + } + bytes := printSortedMap(reflect.ValueOf(m)) + sendReply(&querypb.StreamEvent{ + EventToken: &querypb.EventToken{ + Position: string(bytes), + }, + }, 0) + return nil + } + return c.fallbackClient.UpdateStream(ctx, keyspace, shard, keyRange, tabletType, timestamp, event, sendReply) +} diff --git a/go/cmd/vtgateclienttest/services/errors.go b/go/cmd/vtgateclienttest/services/errors.go index db3747a103..680584d5be 100644 --- a/go/cmd/vtgateclienttest/services/errors.go +++ b/go/cmd/vtgateclienttest/services/errors.go @@ -293,3 +293,10 @@ func (c *errorClient) GetSrvKeyspace(ctx context.Context, keyspace string) (*top } return c.fallbackClient.GetSrvKeyspace(ctx, keyspace) } + +func (c *errorClient) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + if err := requestToError(shard); err != nil { + return err + } + return c.fallbackClient.UpdateStream(ctx, keyspace, shard, keyRange, tabletType, timestamp, event, sendReply) +} diff --git a/go/cmd/vtgateclienttest/services/fallback.go b/go/cmd/vtgateclienttest/services/fallback.go index 95fec60c42..1577a7dd43 100644 --- a/go/cmd/vtgateclienttest/services/fallback.go +++ b/go/cmd/vtgateclienttest/services/fallback.go @@ -106,6 +106,10 @@ func (c fallbackClient) GetSrvKeyspace(ctx context.Context, keyspace string) (*t return c.fallback.GetSrvKeyspace(ctx, keyspace) } +func (c fallbackClient) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + return c.fallback.UpdateStream(ctx, keyspace, shard, keyRange, tabletType, timestamp, event, sendReply) +} + func (c fallbackClient) HandlePanic(err *error) { c.fallback.HandlePanic(err) } diff --git a/go/cmd/vtgateclienttest/services/terminal.go b/go/cmd/vtgateclienttest/services/terminal.go index f284e7c8a2..892b6ce214 100644 --- a/go/cmd/vtgateclienttest/services/terminal.go +++ b/go/cmd/vtgateclienttest/services/terminal.go @@ -110,6 +110,10 @@ func (c *terminalClient) GetSrvKeyspace(ctx context.Context, keyspace string) (* return nil, errTerminal } +func (c *terminalClient) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + return errTerminal +} + func (c *terminalClient) HandlePanic(err *error) { if x := recover(); x != nil { log.Errorf("Uncaught panic:\n%v\n%s", x, tb.Stack(4)) diff --git a/go/vt/proto/vtgate/vtgate.pb.go b/go/vt/proto/vtgate/vtgate.pb.go index 62e90f227a..6f911e1a29 100644 --- a/go/vt/proto/vtgate/vtgate.pb.go +++ b/go/vt/proto/vtgate/vtgate.pb.go @@ -44,6 +44,8 @@ It has these top-level messages: SplitQueryResponse GetSrvKeyspaceRequest GetSrvKeyspaceResponse + UpdateStreamRequest + UpdateStreamResponse */ package vtgate @@ -1341,6 +1343,80 @@ func (m *GetSrvKeyspaceResponse) GetSrvKeyspace() *topodata.SrvKeyspace { return nil } +// UpdateStreamRequest is the payload to UpdateStream. +type UpdateStreamRequest struct { + // caller_id identifies the caller. This is the effective caller ID, + // set by the application to further identify the caller. + CallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=caller_id,json=callerId" json:"caller_id,omitempty"` + // keyspace to target the query to. + Keyspace string `protobuf:"bytes,2,opt,name=keyspace" json:"keyspace,omitempty"` + // shard to target the query to, for unsharded keyspaces. + Shard string `protobuf:"bytes,3,opt,name=shard" json:"shard,omitempty"` + // KeyRange to target the query to, for sharded keyspaces. + KeyRange *topodata.KeyRange `protobuf:"bytes,4,opt,name=key_range,json=keyRange" json:"key_range,omitempty"` + // tablet_type is the type of tablets that this request is targeted to. + TabletType topodata.TabletType `protobuf:"varint,5,opt,name=tablet_type,json=tabletType,enum=topodata.TabletType" json:"tablet_type,omitempty"` + // timestamp is the timestamp to start the stream from. It is + // unused is event is set, and we are only streaming from the shard + // described by event.shard. + Timestamp int64 `protobuf:"varint,6,opt,name=timestamp" json:"timestamp,omitempty"` + // event is the event to start the stream from. + // Note it is only used if we are streaming from exactly the same shard + // as this event was coming from. Otherwise we can't use this event, + // and will use the timestamp as a starting point. + Event *query.EventToken `protobuf:"bytes,7,opt,name=event" json:"event,omitempty"` +} + +func (m *UpdateStreamRequest) Reset() { *m = UpdateStreamRequest{} } +func (m *UpdateStreamRequest) String() string { return proto.CompactTextString(m) } +func (*UpdateStreamRequest) ProtoMessage() {} +func (*UpdateStreamRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{35} } + +func (m *UpdateStreamRequest) GetCallerId() *vtrpc.CallerID { + if m != nil { + return m.CallerId + } + return nil +} + +func (m *UpdateStreamRequest) GetKeyRange() *topodata.KeyRange { + if m != nil { + return m.KeyRange + } + return nil +} + +func (m *UpdateStreamRequest) GetEvent() *query.EventToken { + if m != nil { + return m.Event + } + return nil +} + +// UpdateStreamResponse is streamed by UpdateStream. +type UpdateStreamResponse struct { + // event is one event from the stream. + Event *query.StreamEvent `protobuf:"bytes,1,opt,name=event" json:"event,omitempty"` + // resume_timestamp is the timestamp to resume streaming from if the + // client is interrupted. If the Update Stream only goes to one + // shard, this is equal to event.timestamp. If the Update Stream + // goes to multiple shards and aggregates, this is the minimum value + // of the current timestamp for all shards. + ResumeTimestamp int64 `protobuf:"varint,2,opt,name=resume_timestamp,json=resumeTimestamp" json:"resume_timestamp,omitempty"` +} + +func (m *UpdateStreamResponse) Reset() { *m = UpdateStreamResponse{} } +func (m *UpdateStreamResponse) String() string { return proto.CompactTextString(m) } +func (*UpdateStreamResponse) ProtoMessage() {} +func (*UpdateStreamResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{36} } + +func (m *UpdateStreamResponse) GetEvent() *query.StreamEvent { + if m != nil { + return m.Event + } + return nil +} + func init() { proto.RegisterType((*Session)(nil), "vtgate.Session") proto.RegisterType((*Session_ShardSession)(nil), "vtgate.Session.ShardSession") @@ -1382,93 +1458,102 @@ func init() { proto.RegisterType((*SplitQueryResponse_Part)(nil), "vtgate.SplitQueryResponse.Part") proto.RegisterType((*GetSrvKeyspaceRequest)(nil), "vtgate.GetSrvKeyspaceRequest") proto.RegisterType((*GetSrvKeyspaceResponse)(nil), "vtgate.GetSrvKeyspaceResponse") + proto.RegisterType((*UpdateStreamRequest)(nil), "vtgate.UpdateStreamRequest") + proto.RegisterType((*UpdateStreamResponse)(nil), "vtgate.UpdateStreamResponse") } func init() { proto.RegisterFile("vtgate.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 1324 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xd4, 0x59, 0x4f, 0x6f, 0xe3, 0x44, - 0x14, 0x97, 0xed, 0xfc, 0x69, 0x9e, 0x93, 0xb4, 0x9d, 0x6d, 0xb7, 0x21, 0x2c, 0xdb, 0xac, 0x45, - 0xb5, 0x01, 0xaa, 0x48, 0x9b, 0xe5, 0x9f, 0x10, 0x12, 0xd0, 0x50, 0xa1, 0x68, 0x61, 0x55, 0xa6, - 0x15, 0xe2, 0x00, 0xb2, 0xdc, 0x64, 0xd4, 0x9a, 0x24, 0x76, 0xd6, 0x33, 0xce, 0x12, 0x0e, 0xdc, - 0x90, 0xb8, 0xed, 0x01, 0x21, 0x21, 0xc4, 0x85, 0x8f, 0xc0, 0x37, 0x80, 0x13, 0x47, 0x0e, 0x1c, - 0xb8, 0xf1, 0x11, 0xe0, 0x0b, 0x70, 0x40, 0x99, 0x19, 0xc7, 0x8e, 0x9b, 0xa4, 0x69, 0xd2, 0x56, - 0xe9, 0xa9, 0x9e, 0x99, 0x37, 0xf3, 0xde, 0xfb, 0xfd, 0xde, 0xbc, 0x37, 0xaf, 0x81, 0x6c, 0x8f, - 0x9d, 0x58, 0x8c, 0x54, 0xba, 0x9e, 0xcb, 0x5c, 0x94, 0x12, 0xa3, 0xa2, 0xfe, 0xc4, 0x27, 0x5e, - 0x5f, 0x4c, 0x16, 0xf3, 0xcc, 0xed, 0xba, 0x4d, 0x8b, 0x59, 0x72, 0xac, 0xf7, 0x98, 0xd7, 0x6d, - 0x88, 0x81, 0xf1, 0xa7, 0x02, 0xe9, 0x43, 0x42, 0xa9, 0xed, 0x3a, 0x68, 0x07, 0xf2, 0xb6, 0x63, - 0x32, 0xcf, 0x72, 0xa8, 0xd5, 0x60, 0xb6, 0xeb, 0x14, 0x94, 0x92, 0x52, 0x5e, 0xc1, 0x39, 0xdb, - 0x39, 0x0a, 0x27, 0x51, 0x0d, 0xf2, 0xf4, 0xd4, 0xf2, 0x9a, 0x26, 0x15, 0xfb, 0x68, 0x41, 0x2d, - 0x69, 0x65, 0xbd, 0x7a, 0xa7, 0x22, 0x6d, 0x91, 0xe7, 0x55, 0x0e, 0x07, 0x52, 0x72, 0x80, 0x73, - 0x34, 0x32, 0xa2, 0xc5, 0xcf, 0x20, 0x1b, 0x5d, 0x46, 0x3b, 0x90, 0x62, 0x96, 0x77, 0x42, 0x18, - 0xd7, 0xa9, 0x57, 0x73, 0x15, 0xe1, 0xc2, 0x11, 0x9f, 0xc4, 0x72, 0x71, 0x60, 0x62, 0xc4, 0x3e, - 0xd3, 0x6e, 0x16, 0xd4, 0x92, 0x52, 0xd6, 0x70, 0x2e, 0x32, 0x5b, 0x6f, 0x1a, 0xdf, 0xa9, 0x90, - 0xdf, 0xff, 0x92, 0x34, 0x7c, 0x46, 0x30, 0x79, 0xe2, 0x13, 0xca, 0xd0, 0x2e, 0x64, 0x1a, 0x56, - 0xbb, 0x4d, 0xbc, 0xc1, 0x26, 0xa1, 0x63, 0xb5, 0x22, 0x90, 0xa8, 0xf1, 0xf9, 0xfa, 0xfb, 0x78, - 0x45, 0x48, 0xd4, 0x9b, 0xe8, 0x25, 0x48, 0x4b, 0xef, 0xb8, 0x02, 0x21, 0x1b, 0x75, 0x0e, 0x07, - 0xeb, 0xe8, 0x3e, 0x24, 0xb9, 0xa9, 0x05, 0x8d, 0x0b, 0xae, 0x4b, 0xc3, 0xf7, 0x5c, 0xdf, 0x69, - 0x7e, 0x3c, 0xf8, 0xc4, 0x62, 0x1d, 0xbd, 0x06, 0x3a, 0xb3, 0x8e, 0xdb, 0x84, 0x99, 0xac, 0xdf, - 0x25, 0x85, 0x44, 0x49, 0x29, 0xe7, 0xab, 0x1b, 0x95, 0x21, 0x3b, 0x47, 0x7c, 0xf1, 0xa8, 0xdf, - 0x25, 0x18, 0xd8, 0xf0, 0x1b, 0xed, 0x02, 0x72, 0x5c, 0x66, 0xc6, 0x98, 0x49, 0x72, 0x66, 0xd6, - 0x1c, 0x97, 0xd5, 0x47, 0xc8, 0x29, 0xc2, 0x4a, 0x8b, 0xf4, 0x69, 0xd7, 0x6a, 0x90, 0x42, 0xaa, - 0xa4, 0x94, 0x33, 0x78, 0x38, 0x36, 0x9e, 0x29, 0xb0, 0x3a, 0x44, 0x85, 0x76, 0x5d, 0x87, 0x12, - 0xb4, 0x03, 0x49, 0xe2, 0x79, 0xae, 0x17, 0x83, 0x04, 0x1f, 0xd4, 0xf6, 0x07, 0xd3, 0x58, 0xac, - 0x5e, 0x04, 0x8f, 0x97, 0x21, 0xe5, 0x11, 0xea, 0xb7, 0x99, 0x04, 0x04, 0x49, 0x40, 0x04, 0x16, - 0x7c, 0x05, 0x4b, 0x09, 0xe3, 0x17, 0x15, 0x36, 0xa4, 0x45, 0x3c, 0x1a, 0xe8, 0xf2, 0xb0, 0x15, - 0x05, 0x32, 0x31, 0x0a, 0x24, 0xba, 0x0d, 0x29, 0x1e, 0xcd, 0xb4, 0x90, 0x2c, 0x69, 0xe5, 0x0c, - 0x96, 0xa3, 0x38, 0xc3, 0xa9, 0x85, 0x18, 0x4e, 0x8f, 0x67, 0xd8, 0xf8, 0x5e, 0x81, 0xcd, 0x18, - 0x66, 0x4b, 0xc1, 0xe5, 0x6f, 0x2a, 0x3c, 0x27, 0xed, 0x7a, 0x24, 0x81, 0xaa, 0xdf, 0x14, 0x42, - 0xef, 0x41, 0x36, 0xf8, 0x36, 0x6d, 0x49, 0x6b, 0x16, 0xeb, 0xad, 0xd0, 0x8f, 0xeb, 0xe1, 0xf6, - 0x47, 0x05, 0x8a, 0xe3, 0x30, 0x5c, 0x0a, 0x82, 0xff, 0x50, 0x61, 0x2b, 0x34, 0x0e, 0x5b, 0xce, - 0x09, 0xb9, 0x21, 0xf4, 0x3e, 0x00, 0x68, 0x91, 0xbe, 0xe9, 0x71, 0x93, 0x39, 0xb9, 0x03, 0x4f, - 0x87, 0xd4, 0x05, 0xde, 0xe0, 0x4c, 0x2b, 0xf0, 0xeb, 0x7a, 0xe8, 0xfe, 0x41, 0x81, 0xc2, 0x59, - 0x44, 0x97, 0x82, 0xec, 0x6f, 0x13, 0x43, 0xb2, 0xf7, 0x1d, 0x66, 0xb3, 0xfe, 0x8d, 0xb9, 0xcb, - 0xbb, 0x80, 0x08, 0xb7, 0xd8, 0x6c, 0xb8, 0x6d, 0xbf, 0xe3, 0x98, 0x8e, 0xd5, 0x21, 0xbc, 0x5e, - 0x66, 0xf0, 0x9a, 0x58, 0xa9, 0xf1, 0x85, 0xc7, 0x56, 0x87, 0xa0, 0x4f, 0xe1, 0x96, 0x94, 0x1e, - 0x49, 0x00, 0x29, 0x1e, 0x23, 0xe5, 0xc0, 0xd2, 0x09, 0x48, 0x54, 0x82, 0x09, 0xbc, 0x2e, 0x0e, - 0x79, 0x34, 0x39, 0x61, 0xa4, 0x17, 0x8a, 0xa0, 0x95, 0xf1, 0x11, 0x54, 0x3c, 0x86, 0x95, 0xc0, - 0x06, 0xb4, 0x0d, 0x09, 0xae, 0x49, 0xe1, 0x9a, 0xf4, 0xe0, 0x01, 0x35, 0x50, 0xc0, 0x17, 0xd0, - 0x06, 0x24, 0x7b, 0x56, 0xdb, 0x27, 0x9c, 0x87, 0x2c, 0x16, 0x03, 0xb4, 0x0d, 0x7a, 0xc4, 0x75, - 0x0e, 0x7d, 0x16, 0x43, 0x98, 0xfa, 0xa2, 0x51, 0x1a, 0x01, 0x60, 0x29, 0xa2, 0xd4, 0x81, 0x55, - 0x1e, 0x1c, 0xbc, 0x10, 0x72, 0x81, 0x30, 0x86, 0x94, 0x0b, 0xc4, 0x90, 0x3a, 0xb1, 0xc0, 0x6b, - 0xd1, 0x02, 0x6f, 0x7c, 0x13, 0xd6, 0xb8, 0x3d, 0x8b, 0x35, 0x4e, 0xaf, 0xe9, 0xd1, 0xf2, 0x00, - 0xd2, 0x03, 0x9b, 0x6d, 0x22, 0xec, 0xd1, 0xab, 0x5b, 0x81, 0x68, 0xcc, 0x7b, 0x1c, 0xc8, 0xcd, - 0xfb, 0xd8, 0xdc, 0x81, 0xbc, 0x45, 0xc7, 0x3c, 0x34, 0x73, 0x16, 0x8d, 0x26, 0xae, 0x9f, 0xc2, - 0x3a, 0x35, 0x82, 0xc3, 0x95, 0x05, 0xc5, 0x2e, 0xa4, 0x05, 0xe5, 0x01, 0x02, 0xe3, 0xa2, 0x22, - 0x10, 0x31, 0xbe, 0x86, 0x0d, 0x0e, 0x4c, 0x78, 0x1d, 0x2f, 0x31, 0x36, 0xe2, 0x6f, 0x05, 0xed, - 0xcc, 0x5b, 0xc1, 0x78, 0xa6, 0xc2, 0xdd, 0x28, 0x3c, 0xd7, 0xf9, 0x1e, 0x7a, 0x3d, 0x1e, 0x2b, - 0x77, 0x46, 0x62, 0x25, 0x06, 0xc9, 0x75, 0x05, 0xcc, 0xcf, 0x0a, 0x6c, 0x4f, 0x44, 0x64, 0x49, - 0xa2, 0xe6, 0x57, 0x05, 0x36, 0x0e, 0x99, 0x47, 0xac, 0xce, 0x42, 0xad, 0xe3, 0x30, 0xc8, 0xd4, - 0x8b, 0xf5, 0x83, 0xda, 0x8c, 0x88, 0x4f, 0xa9, 0x7d, 0x46, 0x0d, 0x36, 0x63, 0x1e, 0x48, 0x6c, - 0xc3, 0xa4, 0xaa, 0x9c, 0x9b, 0x54, 0xff, 0x56, 0xa0, 0x38, 0x72, 0xca, 0x22, 0x59, 0x6e, 0x66, - 0x34, 0xa2, 0x6e, 0x69, 0x13, 0xd3, 0x71, 0x62, 0x5a, 0xbf, 0x95, 0x9c, 0x0d, 0x41, 0xa3, 0x0e, - 0xcf, 0x8f, 0xf5, 0x6f, 0x0e, 0xac, 0xfe, 0x51, 0x60, 0x7b, 0xe4, 0xac, 0x85, 0xaf, 0xfa, 0xa5, - 0x00, 0x16, 0xcf, 0x51, 0x89, 0x73, 0xfb, 0x99, 0x59, 0xb1, 0x7b, 0x0c, 0xa5, 0xc9, 0xfe, 0xce, - 0x01, 0xe0, 0x7f, 0x0a, 0xbc, 0x10, 0x3f, 0x70, 0x91, 0xd6, 0xe2, 0x52, 0xe0, 0x1b, 0xed, 0x17, - 0x12, 0x73, 0xf4, 0x0b, 0xb3, 0xc2, 0xf9, 0x21, 0xdc, 0x9d, 0xe4, 0xfd, 0x1c, 0x60, 0xbe, 0x0d, - 0xd9, 0x3d, 0x72, 0x62, 0x3b, 0x73, 0x41, 0x67, 0xbc, 0x05, 0x39, 0xb9, 0x5b, 0xaa, 0x8e, 0x64, - 0x5a, 0x65, 0x7a, 0xa6, 0x35, 0x4e, 0x21, 0x57, 0x73, 0x3b, 0x1d, 0x9b, 0x5d, 0x75, 0x7d, 0x33, - 0xd6, 0x20, 0x1f, 0x68, 0x12, 0x66, 0x1a, 0x5f, 0xc0, 0x2a, 0x76, 0xdb, 0xed, 0x63, 0xab, 0xd1, - 0xba, 0x72, 0xed, 0x08, 0xd6, 0x42, 0x5d, 0x52, 0xff, 0xbf, 0x2a, 0xac, 0x1f, 0x76, 0xdb, 0x36, - 0x93, 0x94, 0xcc, 0x63, 0xc2, 0xb4, 0x07, 0xc7, 0xcc, 0x5d, 0xd1, 0x3d, 0xc8, 0xd2, 0x81, 0x1d, - 0xb2, 0xf1, 0x91, 0xc9, 0x52, 0xe7, 0x73, 0xa2, 0xe5, 0x19, 0x3c, 0xf6, 0x03, 0x11, 0xdf, 0x61, - 0x3c, 0x4c, 0x35, 0x0c, 0x52, 0xc2, 0x77, 0x18, 0x7a, 0x15, 0xb6, 0x1c, 0xbf, 0x63, 0x7a, 0xee, - 0x53, 0x6a, 0x76, 0x89, 0x67, 0xf2, 0x93, 0xcd, 0xae, 0xe5, 0x31, 0xde, 0x03, 0x6b, 0xf8, 0x96, - 0xe3, 0x77, 0xb0, 0xfb, 0x94, 0x1e, 0x10, 0x8f, 0x2b, 0x3f, 0xb0, 0x3c, 0x86, 0xde, 0x85, 0x8c, - 0xd5, 0x3e, 0x71, 0x3d, 0x9b, 0x9d, 0x76, 0x64, 0xa7, 0x63, 0x48, 0x33, 0xcf, 0x20, 0x53, 0x79, - 0x2f, 0x90, 0xc4, 0xe1, 0x26, 0xf4, 0x0a, 0x20, 0x9f, 0x12, 0x53, 0x18, 0x27, 0x94, 0xf6, 0xaa, - 0xb2, 0xed, 0x59, 0xf5, 0x29, 0x09, 0x8f, 0xf9, 0xa4, 0x6a, 0xfc, 0xae, 0x01, 0x8a, 0x9e, 0x2b, - 0xe3, 0xf5, 0x0d, 0x48, 0xf1, 0xfd, 0xb4, 0xa0, 0xf0, 0x2b, 0xbb, 0x3d, 0xa4, 0xf1, 0x8c, 0x6c, - 0x65, 0x60, 0x36, 0x96, 0xe2, 0xc5, 0xcf, 0x21, 0x1b, 0x5c, 0x3c, 0xee, 0x4e, 0x94, 0x0d, 0x65, - 0x6a, 0x6e, 0x50, 0x67, 0xc8, 0x0d, 0xc5, 0x77, 0x20, 0xc3, 0x4b, 0xcc, 0xb9, 0x67, 0x87, 0x75, - 0x4e, 0x8d, 0xd6, 0xb9, 0xe2, 0x5f, 0x0a, 0x24, 0xf8, 0xe6, 0x99, 0x1f, 0xb0, 0x1f, 0x41, 0x7e, - 0x68, 0xa5, 0x60, 0x4f, 0x44, 0xf6, 0xfd, 0x29, 0x90, 0x44, 0x21, 0xc0, 0xd9, 0x56, 0x14, 0x90, - 0x1a, 0x80, 0xf8, 0x97, 0x3f, 0x3f, 0x4a, 0xc4, 0xe1, 0x8b, 0x53, 0x8e, 0x1a, 0xba, 0x8b, 0x33, - 0x74, 0xe8, 0x39, 0x82, 0x04, 0xb5, 0xbf, 0x12, 0x8f, 0x16, 0x0d, 0xf3, 0x6f, 0xe3, 0x21, 0x6c, - 0x7e, 0x40, 0xd8, 0xa1, 0xd7, 0x0b, 0xea, 0x48, 0x70, 0x7d, 0xa6, 0xc0, 0x64, 0x60, 0xb8, 0x1d, - 0xdf, 0x24, 0x23, 0xe0, 0x4d, 0xc8, 0x52, 0xaf, 0x67, 0x8e, 0xec, 0xd4, 0xab, 0x9b, 0x21, 0x3d, - 0xd1, 0x4d, 0x3a, 0x0d, 0x07, 0x7b, 0x45, 0x28, 0x34, 0xdc, 0x4e, 0xa5, 0xef, 0xfa, 0xcc, 0x3f, - 0x26, 0x95, 0x9e, 0xcd, 0x08, 0xa5, 0xe2, 0x37, 0x92, 0xe3, 0x14, 0xff, 0xf3, 0xf0, 0xff, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x12, 0x77, 0x8b, 0xc2, 0x6c, 0x19, 0x00, 0x00, + // 1434 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xd4, 0x59, 0x4f, 0x6f, 0x1b, 0x45, + 0x14, 0xd7, 0xae, 0xff, 0x24, 0x7e, 0xfe, 0x93, 0x74, 0xea, 0xb4, 0xc6, 0x94, 0xc6, 0x5d, 0x11, + 0xd5, 0x85, 0xc8, 0xa8, 0x2e, 0xff, 0x84, 0x90, 0x80, 0x98, 0x08, 0x45, 0x85, 0xaa, 0x4c, 0x02, + 0xe2, 0x00, 0x5a, 0x4d, 0xec, 0x51, 0xb2, 0xd8, 0xbb, 0xeb, 0xee, 0xcc, 0xba, 0x98, 0x03, 0x37, + 0x24, 0x6e, 0x3d, 0x20, 0x24, 0x84, 0xb8, 0x20, 0xf1, 0x05, 0xf8, 0x06, 0x70, 0xe2, 0xc8, 0x81, + 0x03, 0x37, 0x3e, 0x02, 0x7c, 0x01, 0x0e, 0x68, 0x67, 0x66, 0xbd, 0xeb, 0x8d, 0xed, 0xba, 0x4e, + 0x1a, 0x39, 0xa7, 0xec, 0xcc, 0xbc, 0x99, 0x79, 0xef, 0xf7, 0xfb, 0xcd, 0x7b, 0x33, 0x31, 0x14, + 0x06, 0xfc, 0x88, 0x70, 0xda, 0xe8, 0x7b, 0x2e, 0x77, 0x51, 0x56, 0xb6, 0xaa, 0xf9, 0x07, 0x3e, + 0xf5, 0x86, 0xb2, 0xb3, 0x5a, 0xe2, 0x6e, 0xdf, 0xed, 0x10, 0x4e, 0x54, 0x3b, 0x3f, 0xe0, 0x5e, + 0xbf, 0x2d, 0x1b, 0xc6, 0x9f, 0x1a, 0xac, 0xec, 0x53, 0xc6, 0x2c, 0xd7, 0x41, 0x5b, 0x50, 0xb2, + 0x1c, 0x93, 0x7b, 0xc4, 0x61, 0xa4, 0xcd, 0x2d, 0xd7, 0xa9, 0x68, 0x35, 0xad, 0xbe, 0x8a, 0x8b, + 0x96, 0x73, 0x10, 0x75, 0xa2, 0x16, 0x94, 0xd8, 0x31, 0xf1, 0x3a, 0x26, 0x93, 0xf3, 0x58, 0x45, + 0xaf, 0xa5, 0xea, 0xf9, 0xe6, 0xb5, 0x86, 0xf2, 0x45, 0xad, 0xd7, 0xd8, 0x0f, 0xac, 0x54, 0x03, + 0x17, 0x59, 0xac, 0xc5, 0xaa, 0x9f, 0x42, 0x21, 0x3e, 0x8c, 0xb6, 0x20, 0xcb, 0x89, 0x77, 0x44, + 0xb9, 0xd8, 0x33, 0xdf, 0x2c, 0x36, 0x64, 0x08, 0x07, 0xa2, 0x13, 0xab, 0xc1, 0xc0, 0xc5, 0x98, + 0x7f, 0xa6, 0xd5, 0xa9, 0xe8, 0x35, 0xad, 0x9e, 0xc2, 0xc5, 0x58, 0xef, 0x5e, 0xc7, 0xf8, 0x56, + 0x87, 0xd2, 0xee, 0x17, 0xb4, 0xed, 0x73, 0x8a, 0xe9, 0x03, 0x9f, 0x32, 0x8e, 0xb6, 0x21, 0xd7, + 0x26, 0xbd, 0x1e, 0xf5, 0x82, 0x49, 0x72, 0x8f, 0xb5, 0x86, 0x44, 0xa2, 0x25, 0xfa, 0xf7, 0xde, + 0xc5, 0xab, 0xd2, 0x62, 0xaf, 0x83, 0x6e, 0xc1, 0x8a, 0x8a, 0x4e, 0x6c, 0x20, 0x6d, 0xe3, 0xc1, + 0xe1, 0x70, 0x1c, 0xdd, 0x84, 0x8c, 0x70, 0xb5, 0x92, 0x12, 0x86, 0x97, 0x94, 0xe3, 0x3b, 0xae, + 0xef, 0x74, 0x3e, 0x0c, 0x3e, 0xb1, 0x1c, 0x47, 0xaf, 0x40, 0x9e, 0x93, 0xc3, 0x1e, 0xe5, 0x26, + 0x1f, 0xf6, 0x69, 0x25, 0x5d, 0xd3, 0xea, 0xa5, 0x66, 0xb9, 0x31, 0x62, 0xe7, 0x40, 0x0c, 0x1e, + 0x0c, 0xfb, 0x14, 0x03, 0x1f, 0x7d, 0xa3, 0x6d, 0x40, 0x8e, 0xcb, 0xcd, 0x04, 0x33, 0x19, 0xc1, + 0xcc, 0xba, 0xe3, 0xf2, 0xbd, 0x31, 0x72, 0xaa, 0xb0, 0xda, 0xa5, 0x43, 0xd6, 0x27, 0x6d, 0x5a, + 0xc9, 0xd6, 0xb4, 0x7a, 0x0e, 0x8f, 0xda, 0xc6, 0x23, 0x0d, 0xd6, 0x46, 0xa8, 0xb0, 0xbe, 0xeb, + 0x30, 0x8a, 0xb6, 0x20, 0x43, 0x3d, 0xcf, 0xf5, 0x12, 0x90, 0xe0, 0xfb, 0xad, 0xdd, 0xa0, 0x1b, + 0xcb, 0xd1, 0x27, 0xc1, 0xe3, 0x05, 0xc8, 0x7a, 0x94, 0xf9, 0x3d, 0xae, 0x00, 0x41, 0x0a, 0x10, + 0x89, 0x85, 0x18, 0xc1, 0xca, 0xc2, 0xf8, 0x45, 0x87, 0xb2, 0xf2, 0x48, 0xa8, 0x81, 0x2d, 0x0f, + 0x5b, 0x71, 0x20, 0xd3, 0xe3, 0x40, 0xa2, 0x2b, 0x90, 0x15, 0x6a, 0x66, 0x95, 0x4c, 0x2d, 0x55, + 0xcf, 0x61, 0xd5, 0x4a, 0x32, 0x9c, 0x3d, 0x15, 0xc3, 0x2b, 0x93, 0x19, 0x36, 0xbe, 0xd3, 0x60, + 0x23, 0x81, 0xd9, 0x52, 0x70, 0xf9, 0x9b, 0x0e, 0xcf, 0x28, 0xbf, 0xee, 0x2a, 0xa0, 0xf6, 0x2e, + 0x0a, 0xa1, 0x37, 0xa0, 0x10, 0x7e, 0x9b, 0x96, 0xa2, 0xb5, 0x80, 0xf3, 0xdd, 0x28, 0x8e, 0xf3, + 0xe1, 0xf6, 0x07, 0x0d, 0xaa, 0x93, 0x30, 0x5c, 0x0a, 0x82, 0xff, 0xd0, 0xe1, 0x6a, 0xe4, 0x1c, + 0x26, 0xce, 0x11, 0xbd, 0x20, 0xf4, 0xde, 0x06, 0xe8, 0xd2, 0xa1, 0xe9, 0x09, 0x97, 0x05, 0xb9, + 0x41, 0xa4, 0x23, 0xea, 0xc2, 0x68, 0x70, 0xae, 0x1b, 0xc6, 0x75, 0x3e, 0x74, 0x7f, 0xaf, 0x41, + 0xe5, 0x24, 0xa2, 0x4b, 0x41, 0xf6, 0x37, 0xe9, 0x11, 0xd9, 0xbb, 0x0e, 0xb7, 0xf8, 0xf0, 0xc2, + 0x9c, 0xe5, 0x6d, 0x40, 0x54, 0x78, 0x6c, 0xb6, 0xdd, 0x9e, 0x6f, 0x3b, 0xa6, 0x43, 0x6c, 0x2a, + 0xea, 0x65, 0x0e, 0xaf, 0xcb, 0x91, 0x96, 0x18, 0xb8, 0x47, 0x6c, 0x8a, 0x3e, 0x81, 0xcb, 0xca, + 0x7a, 0x2c, 0x01, 0x64, 0x85, 0x46, 0xea, 0xa1, 0xa7, 0x53, 0x90, 0x68, 0x84, 0x1d, 0xf8, 0x92, + 0x5c, 0xe4, 0xee, 0xf4, 0x84, 0xb1, 0x72, 0x2a, 0x05, 0xad, 0x4e, 0x56, 0x50, 0xf5, 0x10, 0x56, + 0x43, 0x1f, 0xd0, 0x26, 0xa4, 0xc5, 0x4e, 0x9a, 0xd8, 0x29, 0x1f, 0x5e, 0xa0, 0x82, 0x0d, 0xc4, + 0x00, 0x2a, 0x43, 0x66, 0x40, 0x7a, 0x3e, 0x15, 0x3c, 0x14, 0xb0, 0x6c, 0xa0, 0x4d, 0xc8, 0xc7, + 0x42, 0x17, 0xd0, 0x17, 0x30, 0x44, 0xa9, 0x2f, 0xae, 0xd2, 0x18, 0x00, 0x4b, 0xa1, 0x52, 0x07, + 0xd6, 0x84, 0x38, 0x44, 0x21, 0x14, 0x06, 0x91, 0x86, 0xb4, 0x27, 0xd0, 0x90, 0x3e, 0xb5, 0xc0, + 0xa7, 0xe2, 0x05, 0xde, 0xf8, 0x3a, 0xaa, 0x71, 0x3b, 0x84, 0xb7, 0x8f, 0xcf, 0xe9, 0xd2, 0x72, + 0x1b, 0x56, 0x02, 0x9f, 0x2d, 0x2a, 0xfd, 0xc9, 0x37, 0xaf, 0x86, 0xa6, 0x89, 0xe8, 0x71, 0x68, + 0xb7, 0xe8, 0x65, 0x73, 0x0b, 0x4a, 0x84, 0x4d, 0xb8, 0x68, 0x16, 0x09, 0x8b, 0x27, 0xae, 0x1f, + 0xa3, 0x3a, 0x35, 0x86, 0xc3, 0x53, 0x13, 0xc5, 0x36, 0xac, 0x48, 0xca, 0x43, 0x04, 0x26, 0xa9, + 0x22, 0x34, 0x31, 0xbe, 0x82, 0xb2, 0x00, 0x26, 0x3a, 0x8e, 0x67, 0xa8, 0x8d, 0xe4, 0x5d, 0x21, + 0x75, 0xe2, 0xae, 0x60, 0x3c, 0xd2, 0xe1, 0x7a, 0x1c, 0x9e, 0xf3, 0xbc, 0x0f, 0xbd, 0x9a, 0xd4, + 0xca, 0xb5, 0x31, 0xad, 0x24, 0x20, 0x39, 0x2f, 0xc1, 0xfc, 0xa4, 0xc1, 0xe6, 0x54, 0x44, 0x96, + 0x44, 0x35, 0xbf, 0x6a, 0x50, 0xde, 0xe7, 0x1e, 0x25, 0xf6, 0xa9, 0x9e, 0x8e, 0x23, 0x91, 0xe9, + 0x4f, 0xf6, 0x1e, 0x4c, 0xcd, 0x89, 0xf8, 0x8c, 0xda, 0x67, 0xb4, 0x60, 0x23, 0x11, 0x81, 0xc2, + 0x36, 0x4a, 0xaa, 0xda, 0x63, 0x93, 0xea, 0xdf, 0x1a, 0x54, 0xc7, 0x56, 0x39, 0x4d, 0x96, 0x9b, + 0x1b, 0x8d, 0x78, 0x58, 0xa9, 0xa9, 0xe9, 0x38, 0x3d, 0xeb, 0xbd, 0x95, 0x99, 0x0f, 0x41, 0x63, + 0x0f, 0x9e, 0x9d, 0x18, 0xdf, 0x02, 0x58, 0xfd, 0xa3, 0xc1, 0xe6, 0xd8, 0x5a, 0xa7, 0x3e, 0xea, + 0x67, 0x02, 0x58, 0x32, 0x47, 0xa5, 0x1f, 0xfb, 0x9e, 0x99, 0x17, 0xbb, 0x7b, 0x50, 0x9b, 0x1e, + 0xef, 0x02, 0x00, 0xfe, 0xa7, 0xc1, 0x73, 0xc9, 0x05, 0x4f, 0xf3, 0xb4, 0x38, 0x13, 0xf8, 0xc6, + 0xdf, 0x0b, 0xe9, 0x05, 0xde, 0x0b, 0xf3, 0xc2, 0xf9, 0x3e, 0x5c, 0x9f, 0x16, 0xfd, 0x02, 0x60, + 0xbe, 0x09, 0x85, 0x1d, 0x7a, 0x64, 0x39, 0x0b, 0x41, 0x67, 0xbc, 0x01, 0x45, 0x35, 0x5b, 0x6d, + 0x1d, 0xcb, 0xb4, 0xda, 0xec, 0x4c, 0x6b, 0x1c, 0x43, 0xb1, 0xe5, 0xda, 0xb6, 0xc5, 0x9f, 0x76, + 0x7d, 0x33, 0xd6, 0xa1, 0x14, 0xee, 0x24, 0xdd, 0x34, 0x3e, 0x87, 0x35, 0xec, 0xf6, 0x7a, 0x87, + 0xa4, 0xdd, 0x7d, 0xea, 0xbb, 0x23, 0x58, 0x8f, 0xf6, 0x52, 0xfb, 0xff, 0xab, 0xc3, 0xa5, 0xfd, + 0x7e, 0xcf, 0xe2, 0x8a, 0x92, 0x45, 0x5c, 0x98, 0x75, 0xe1, 0x98, 0xfb, 0x55, 0x74, 0x03, 0x0a, + 0x2c, 0xf0, 0x43, 0x3d, 0x7c, 0x54, 0xb2, 0xcc, 0x8b, 0x3e, 0xf9, 0xe4, 0x09, 0x2e, 0xfb, 0xa1, + 0x89, 0xef, 0x70, 0x21, 0xd3, 0x14, 0x06, 0x65, 0xe1, 0x3b, 0x1c, 0xbd, 0x0c, 0x57, 0x1d, 0xdf, + 0x36, 0x3d, 0xf7, 0x21, 0x33, 0xfb, 0xd4, 0x33, 0xc5, 0xca, 0x66, 0x9f, 0x78, 0x5c, 0xbc, 0x81, + 0x53, 0xf8, 0xb2, 0xe3, 0xdb, 0xd8, 0x7d, 0xc8, 0xee, 0x53, 0x4f, 0x6c, 0x7e, 0x9f, 0x78, 0x1c, + 0xbd, 0x0d, 0x39, 0xd2, 0x3b, 0x72, 0x3d, 0x8b, 0x1f, 0xdb, 0xea, 0xa5, 0x63, 0x28, 0x37, 0x4f, + 0x20, 0xd3, 0x78, 0x27, 0xb4, 0xc4, 0xd1, 0x24, 0xf4, 0x22, 0x20, 0x9f, 0x51, 0x53, 0x3a, 0x27, + 0x37, 0x1d, 0x34, 0xd5, 0xb3, 0x67, 0xcd, 0x67, 0x34, 0x5a, 0xe6, 0xe3, 0xa6, 0xf1, 0x7b, 0x0a, + 0x50, 0x7c, 0x5d, 0xa5, 0xd7, 0xd7, 0x20, 0x2b, 0xe6, 0xb3, 0x8a, 0x26, 0x8e, 0xec, 0xe6, 0x88, + 0xc6, 0x13, 0xb6, 0x8d, 0xc0, 0x6d, 0xac, 0xcc, 0xab, 0x9f, 0x41, 0x21, 0x3c, 0x78, 0x22, 0x9c, + 0x38, 0x1b, 0xda, 0xcc, 0xdc, 0xa0, 0xcf, 0x91, 0x1b, 0xaa, 0x6f, 0x41, 0x4e, 0x94, 0x98, 0xc7, + 0xae, 0x1d, 0xd5, 0x39, 0x3d, 0x5e, 0xe7, 0xaa, 0x7f, 0x69, 0x90, 0x16, 0x93, 0xe7, 0xbe, 0xc0, + 0x7e, 0x00, 0xa5, 0x91, 0x97, 0x92, 0x3d, 0xa9, 0xec, 0x9b, 0x33, 0x20, 0x89, 0x43, 0x80, 0x0b, + 0xdd, 0x38, 0x20, 0x2d, 0x00, 0xf9, 0x2f, 0x7f, 0xb1, 0x94, 0xd4, 0xe1, 0xf3, 0x33, 0x96, 0x1a, + 0x85, 0x8b, 0x73, 0x6c, 0x14, 0x39, 0x82, 0x34, 0xb3, 0xbe, 0x94, 0x97, 0x96, 0x14, 0x16, 0xdf, + 0xc6, 0x1d, 0xd8, 0x78, 0x8f, 0xf2, 0x7d, 0x6f, 0x10, 0xd6, 0x91, 0xf0, 0xf8, 0xcc, 0x80, 0xc9, + 0xc0, 0x70, 0x25, 0x39, 0x49, 0x29, 0xe0, 0x75, 0x28, 0x30, 0x6f, 0x60, 0x8e, 0xcd, 0xcc, 0x37, + 0x37, 0x22, 0x7a, 0xe2, 0x93, 0xf2, 0x2c, 0x6a, 0x18, 0x3f, 0xeb, 0x70, 0xf9, 0xa3, 0x7e, 0x87, + 0x70, 0x2a, 0xf3, 0xf1, 0xd9, 0x1f, 0xe3, 0x32, 0x64, 0x04, 0x16, 0xaa, 0xda, 0xc8, 0x06, 0x7a, + 0x09, 0x72, 0x23, 0xa2, 0x04, 0x32, 0x93, 0xd5, 0xb4, 0x1a, 0xd2, 0xb1, 0x60, 0xa1, 0x41, 0xd7, + 0x20, 0xc7, 0x2d, 0x9b, 0x32, 0x4e, 0xec, 0xbe, 0x3a, 0xc9, 0x51, 0x47, 0xa0, 0x2b, 0x3a, 0xa0, + 0x0e, 0x17, 0x67, 0x37, 0xd2, 0xd5, 0x6e, 0xd0, 0x77, 0xe0, 0x76, 0xa9, 0x83, 0xe5, 0xb8, 0xd1, + 0x85, 0xf2, 0x38, 0x4a, 0x0a, 0xf8, 0x7a, 0xb8, 0xc0, 0x78, 0x91, 0x52, 0xb5, 0x2d, 0x18, 0x51, + 0x2b, 0xa0, 0x5b, 0xb0, 0x1e, 0x54, 0x2b, 0x9b, 0x9a, 0x91, 0x3f, 0xf2, 0x37, 0x9c, 0x35, 0xd9, + 0x7f, 0x10, 0x76, 0xef, 0x54, 0xa1, 0xd2, 0x76, 0xed, 0xc6, 0xd0, 0xf5, 0xb9, 0x7f, 0x48, 0x1b, + 0x03, 0x8b, 0x53, 0xc6, 0xe4, 0xef, 0x56, 0x87, 0x59, 0xf1, 0xe7, 0xce, 0xff, 0x01, 0x00, 0x00, + 0xff, 0xff, 0x78, 0xd5, 0x81, 0xef, 0x00, 0x1b, 0x00, 0x00, } diff --git a/go/vt/proto/vtgateservice/vtgateservice.pb.go b/go/vt/proto/vtgateservice/vtgateservice.pb.go index e8b76faa02..efb6155754 100644 --- a/go/vt/proto/vtgateservice/vtgateservice.pb.go +++ b/go/vt/proto/vtgateservice/vtgateservice.pb.go @@ -106,6 +106,9 @@ type VitessClient interface { // using custom sharding. // API group: Topology GetSrvKeyspace(ctx context.Context, in *vtgate.GetSrvKeyspaceRequest, opts ...grpc.CallOption) (*vtgate.GetSrvKeyspaceResponse, error) + // UpdateStream asks the server for a stream of StreamEvent objects. + // API group: Update Stream + UpdateStream(ctx context.Context, in *vtgate.UpdateStreamRequest, opts ...grpc.CallOption) (Vitess_UpdateStreamClient, error) } type vitessClient struct { @@ -352,6 +355,38 @@ func (c *vitessClient) GetSrvKeyspace(ctx context.Context, in *vtgate.GetSrvKeys return out, nil } +func (c *vitessClient) UpdateStream(ctx context.Context, in *vtgate.UpdateStreamRequest, opts ...grpc.CallOption) (Vitess_UpdateStreamClient, error) { + stream, err := grpc.NewClientStream(ctx, &_Vitess_serviceDesc.Streams[4], c.cc, "/vtgateservice.Vitess/UpdateStream", opts...) + if err != nil { + return nil, err + } + x := &vitessUpdateStreamClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Vitess_UpdateStreamClient interface { + Recv() (*vtgate.UpdateStreamResponse, error) + grpc.ClientStream +} + +type vitessUpdateStreamClient struct { + grpc.ClientStream +} + +func (x *vitessUpdateStreamClient) Recv() (*vtgate.UpdateStreamResponse, error) { + m := new(vtgate.UpdateStreamResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // Server API for Vitess service type VitessServer interface { @@ -417,6 +452,9 @@ type VitessServer interface { // using custom sharding. // API group: Topology GetSrvKeyspace(context.Context, *vtgate.GetSrvKeyspaceRequest) (*vtgate.GetSrvKeyspaceResponse, error) + // UpdateStream asks the server for a stream of StreamEvent objects. + // API group: Update Stream + UpdateStream(*vtgate.UpdateStreamRequest, Vitess_UpdateStreamServer) error } func RegisterVitessServer(s *grpc.Server, srv VitessServer) { @@ -723,6 +761,27 @@ func _Vitess_GetSrvKeyspace_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } +func _Vitess_UpdateStream_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(vtgate.UpdateStreamRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(VitessServer).UpdateStream(m, &vitessUpdateStreamServer{stream}) +} + +type Vitess_UpdateStreamServer interface { + Send(*vtgate.UpdateStreamResponse) error + grpc.ServerStream +} + +type vitessUpdateStreamServer struct { + grpc.ServerStream +} + +func (x *vitessUpdateStreamServer) Send(m *vtgate.UpdateStreamResponse) error { + return x.ServerStream.SendMsg(m) +} + var _Vitess_serviceDesc = grpc.ServiceDesc{ ServiceName: "vtgateservice.Vitess", HandlerType: (*VitessServer)(nil), @@ -797,6 +856,11 @@ var _Vitess_serviceDesc = grpc.ServiceDesc{ Handler: _Vitess_StreamExecuteKeyRanges_Handler, ServerStreams: true, }, + { + StreamName: "UpdateStream", + Handler: _Vitess_UpdateStream_Handler, + ServerStreams: true, + }, }, Metadata: fileDescriptor0, } @@ -804,33 +868,35 @@ var _Vitess_serviceDesc = grpc.ServiceDesc{ func init() { proto.RegisterFile("vtgateservice.proto", fileDescriptor0) } var fileDescriptor0 = []byte{ - // 444 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x84, 0x94, 0x5f, 0x6b, 0xdb, 0x30, - 0x14, 0xc5, 0xb7, 0x87, 0x65, 0xe3, 0xb2, 0x8c, 0xa1, 0x6c, 0x49, 0x16, 0x96, 0x65, 0xcb, 0x58, - 0xb2, 0x27, 0x33, 0x36, 0x18, 0x0c, 0x06, 0x83, 0x8c, 0x50, 0x4a, 0xa1, 0x34, 0x31, 0xb4, 0x4f, - 0x7d, 0xb0, 0x9d, 0x8b, 0x63, 0xe2, 0x7f, 0xb1, 0x64, 0x53, 0x7f, 0xd3, 0x7e, 0x9c, 0xd2, 0xd8, - 0x52, 0x25, 0x59, 0x4e, 0xde, 0xaa, 0x73, 0xce, 0xfd, 0xa9, 0x39, 0xbe, 0x08, 0x7a, 0x05, 0xf3, - 0x1d, 0x86, 0x14, 0xb3, 0x22, 0xf0, 0xd0, 0x4a, 0xb3, 0x84, 0x25, 0xa4, 0xab, 0x88, 0xa3, 0xd7, - 0xd5, 0xb1, 0x32, 0x7f, 0xde, 0x03, 0x74, 0xae, 0x03, 0x86, 0x94, 0x92, 0xbf, 0xf0, 0x72, 0x79, - 0x87, 0x5e, 0xce, 0x90, 0xf4, 0xad, 0x3a, 0x54, 0x0b, 0x6b, 0xdc, 0xe7, 0x48, 0xd9, 0x68, 0xd0, - 0xd0, 0x69, 0x9a, 0xc4, 0x14, 0xa7, 0xcf, 0xc8, 0x25, 0x74, 0x6b, 0xd1, 0xde, 0x3a, 0xd9, 0x86, - 0x92, 0x8f, 0x5a, 0xb6, 0x92, 0x39, 0x69, 0xdc, 0xe2, 0x0a, 0xde, 0x2d, 0x90, 0xda, 0xba, 0xc0, - 0x92, 0xa6, 0x8e, 0x87, 0xe7, 0x1b, 0x4a, 0xbe, 0x68, 0x63, 0x92, 0xc7, 0xc9, 0xd3, 0x63, 0x11, - 0x81, 0xbf, 0x81, 0xb7, 0x4f, 0xfe, 0xda, 0x89, 0x7d, 0xa4, 0x64, 0xd2, 0x9c, 0xac, 0x1c, 0x8e, - 0xfe, 0xdc, 0x1e, 0x30, 0x80, 0x97, 0x31, 0x0b, 0x58, 0xf9, 0xf8, 0x5f, 0xeb, 0x60, 0xe1, 0xb4, - 0x81, 0xa5, 0x80, 0xa1, 0x90, 0x85, 0xc3, 0xbc, 0x6d, 0xdd, 0xb2, 0x5e, 0x88, 0xe4, 0xb5, 0x15, - 0xa2, 0x44, 0x04, 0x3e, 0x84, 0x81, 0xec, 0xcb, 0xa5, 0xcf, 0x4c, 0x00, 0x43, 0xf3, 0xf3, 0x93, - 0x39, 0x71, 0xdb, 0x15, 0x74, 0x6d, 0x96, 0xa1, 0x13, 0xf1, 0x8d, 0x13, 0xdb, 0xa2, 0xc8, 0x8d, - 0x6d, 0xd1, 0x5c, 0xce, 0xfb, 0xf1, 0x9c, 0xb8, 0xd0, 0x53, 0xcc, 0xba, 0x9f, 0xa9, 0x71, 0x52, - 0x2d, 0xe8, 0xeb, 0xd1, 0x8c, 0x74, 0xc7, 0x1e, 0x86, 0x4a, 0x44, 0x2e, 0x69, 0x6e, 0x84, 0x18, - 0x5a, 0xfa, 0x7e, 0x3a, 0x28, 0x5d, 0xb9, 0x83, 0xbe, 0x9e, 0xab, 0xb7, 0xf5, 0x5b, 0x1b, 0x47, - 0xdd, 0xd9, 0xd9, 0xa9, 0x98, 0x74, 0xd9, 0x6f, 0x78, 0xb1, 0x40, 0x3f, 0x88, 0xc9, 0x3b, 0x3e, - 0x74, 0x38, 0x72, 0xd4, 0x7b, 0x4d, 0x15, 0x5f, 0xf3, 0x0f, 0x74, 0xfe, 0x27, 0x51, 0x14, 0x30, - 0x22, 0x22, 0xd5, 0x99, 0x4f, 0xf6, 0x75, 0x59, 0x8c, 0xfe, 0x83, 0x57, 0xeb, 0x24, 0x0c, 0x5d, - 0xc7, 0xdb, 0x11, 0xf1, 0xba, 0x70, 0x85, 0x8f, 0x0f, 0x9b, 0x86, 0x00, 0x2c, 0x01, 0xec, 0x34, - 0x0c, 0xd8, 0x2a, 0xc7, 0xac, 0x24, 0x1f, 0xc4, 0xaf, 0x15, 0x1a, 0x87, 0x8c, 0x4c, 0x96, 0xc0, - 0xac, 0xe0, 0xcd, 0x19, 0x32, 0x3b, 0x2b, 0xf8, 0x87, 0x20, 0x62, 0xe7, 0x54, 0x9d, 0xe3, 0x3e, - 0xb5, 0xd9, 0x1c, 0xb9, 0x98, 0xc0, 0xd8, 0x4b, 0x22, 0xab, 0x4c, 0x72, 0x96, 0xbb, 0x68, 0x15, - 0x87, 0x57, 0xb6, 0x7a, 0x76, 0x2d, 0x3f, 0x4b, 0x3d, 0xb7, 0x73, 0xf8, 0xfb, 0xd7, 0x43, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x36, 0x12, 0x3e, 0x98, 0xb6, 0x05, 0x00, 0x00, + // 465 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x84, 0x94, 0x5f, 0x8b, 0xd3, 0x40, + 0x14, 0xc5, 0xf5, 0xc1, 0x2a, 0xd7, 0xad, 0xc8, 0xac, 0x76, 0xd7, 0xba, 0xeb, 0x6a, 0xc5, 0x5d, + 0x9f, 0x82, 0x28, 0x08, 0x82, 0x20, 0x54, 0x8a, 0x88, 0x28, 0x6e, 0x83, 0xfa, 0xe4, 0xc3, 0x24, + 0xbd, 0x64, 0xc3, 0xe6, 0x5f, 0x33, 0x93, 0x60, 0x3e, 0x91, 0x5f, 0x53, 0x6c, 0x32, 0xb7, 0x33, + 0x93, 0x49, 0xfb, 0xb6, 0x39, 0xe7, 0xdc, 0xdf, 0x6c, 0xcf, 0x5c, 0x06, 0x0e, 0x6b, 0x19, 0x71, + 0x89, 0x02, 0xcb, 0x3a, 0x0e, 0xd1, 0x2b, 0xca, 0x5c, 0xe6, 0x6c, 0x6c, 0x88, 0xd3, 0x83, 0xf6, + 0xb3, 0x35, 0x5f, 0xff, 0xbd, 0x0b, 0xa3, 0x9f, 0xb1, 0x44, 0x21, 0xd8, 0x7b, 0xb8, 0xbd, 0xf8, + 0x83, 0x61, 0x25, 0x91, 0x4d, 0xbc, 0x2e, 0xd4, 0x09, 0x4b, 0x5c, 0x57, 0x28, 0xe4, 0xf4, 0xa8, + 0xa7, 0x8b, 0x22, 0xcf, 0x04, 0xce, 0x6e, 0xb0, 0x6f, 0x30, 0xee, 0x44, 0xff, 0x8a, 0x97, 0x2b, + 0xc1, 0x4e, 0xac, 0x6c, 0x2b, 0x2b, 0xd2, 0xe9, 0x80, 0x4b, 0xbc, 0xdf, 0xc0, 0x3a, 0xeb, 0x0b, + 0x36, 0xa2, 0xe0, 0x21, 0x7e, 0x5e, 0x09, 0xf6, 0xcc, 0x1a, 0xd3, 0x3c, 0x45, 0x9e, 0xed, 0x8a, + 0x10, 0xfe, 0x17, 0xdc, 0xdf, 0xfa, 0x4b, 0x9e, 0x45, 0x28, 0xd8, 0x59, 0x7f, 0xb2, 0x75, 0x14, + 0xfa, 0xe9, 0x70, 0xc0, 0x01, 0x5e, 0x64, 0x32, 0x96, 0xcd, 0xff, 0xff, 0xda, 0x06, 0x93, 0x33, + 0x04, 0xd6, 0x02, 0x8e, 0x42, 0xe6, 0x5c, 0x86, 0x57, 0x5d, 0xcb, 0x76, 0x21, 0x9a, 0x37, 0x54, + 0x88, 0x11, 0x21, 0x7c, 0x02, 0x47, 0xba, 0xaf, 0x97, 0x7e, 0xee, 0x02, 0x38, 0x9a, 0xbf, 0xd8, + 0x9b, 0xa3, 0xd3, 0xbe, 0xc3, 0xd8, 0x97, 0x25, 0xf2, 0x54, 0x6d, 0x1c, 0x6d, 0x8b, 0x21, 0xf7, + 0xb6, 0xc5, 0x72, 0x15, 0xef, 0xd5, 0x4d, 0x16, 0xc0, 0xa1, 0x61, 0x76, 0xfd, 0xcc, 0x9c, 0x93, + 0x66, 0x41, 0xcf, 0x77, 0x66, 0xb4, 0x33, 0xd6, 0x70, 0x6c, 0x44, 0xf4, 0x92, 0x2e, 0x9c, 0x10, + 0x47, 0x4b, 0x2f, 0xf7, 0x07, 0xb5, 0x23, 0xaf, 0x61, 0x62, 0xe7, 0xba, 0x6d, 0x7d, 0x31, 0xc4, + 0x31, 0x77, 0xf6, 0x7c, 0x5f, 0x4c, 0x3b, 0xec, 0x2d, 0xdc, 0x9a, 0x63, 0x14, 0x67, 0xec, 0x81, + 0x1a, 0xda, 0x7c, 0x2a, 0xd4, 0x43, 0x4b, 0xa5, 0xdb, 0x7c, 0x07, 0xa3, 0x8f, 0x79, 0x9a, 0xc6, + 0x92, 0x51, 0xa4, 0xfd, 0x56, 0x93, 0x13, 0x5b, 0xa6, 0xd1, 0x0f, 0x70, 0x67, 0x99, 0x27, 0x49, + 0xc0, 0xc3, 0x6b, 0x46, 0xaf, 0x8b, 0x52, 0xd4, 0xf8, 0x71, 0xdf, 0x20, 0xc0, 0x02, 0xc0, 0x2f, + 0x92, 0x58, 0x5e, 0x56, 0x58, 0x36, 0xec, 0x11, 0xfd, 0x5a, 0xd2, 0x14, 0x64, 0xea, 0xb2, 0x08, + 0x73, 0x09, 0xf7, 0x3e, 0xa1, 0xf4, 0xcb, 0x5a, 0x5d, 0x04, 0xa3, 0x9d, 0x33, 0x75, 0x85, 0x7b, + 0x32, 0x64, 0x13, 0xf2, 0x2b, 0x1c, 0xfc, 0x28, 0x56, 0x5c, 0x62, 0xdb, 0x3c, 0x7b, 0xac, 0x26, + 0x74, 0x55, 0xe1, 0x4e, 0xdc, 0xe6, 0xf6, 0x72, 0xe6, 0x67, 0x70, 0x1a, 0xe6, 0xa9, 0xd7, 0xe4, + 0x95, 0xac, 0x02, 0xf4, 0xea, 0xcd, 0xa3, 0xdd, 0xbe, 0xe2, 0x5e, 0x54, 0x16, 0x61, 0x30, 0xda, + 0xfc, 0xfd, 0xe6, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x83, 0x0a, 0x30, 0xea, 0x05, 0x06, 0x00, + 0x00, } diff --git a/go/vt/vitessdriver/fakeserver_test.go b/go/vt/vitessdriver/fakeserver_test.go index 19481ab9bd..282950992a 100644 --- a/go/vt/vitessdriver/fakeserver_test.go +++ b/go/vt/vitessdriver/fakeserver_test.go @@ -237,6 +237,11 @@ func (f *fakeVTGateService) GetSrvKeyspace(ctx context.Context, keyspace string) return &topodatapb.SrvKeyspace{}, nil } +// UpdateStream is part of the VTGateService interface +func (f *fakeVTGateService) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + return nil +} + // HandlePanic is part of the VTGateService interface func (f *fakeVTGateService) HandlePanic(err *error) { if x := recover(); x != nil { diff --git a/go/vt/vtctl/query.go b/go/vt/vtctl/query.go index 09b55a245f..f27cf0cb88 100644 --- a/go/vt/vtctl/query.go +++ b/go/vt/vtctl/query.go @@ -92,6 +92,11 @@ func init() { commandVtTabletStreamHealth, "[-count ] [-connect_timeout ] ", "Executes the StreamHealth streaming query to a vttablet process. Will stop after getting answers."}) + addCommand(queriesGroupName, command{ + "VtTabletUpdateStream", + commandVtTabletUpdateStream, + "[-count ] [-connect_timeout ] [-position ] [-timestamp ] ", + "Executes the UpdateStream streaming query to a vttablet process. Will stop after getting answers."}) }) } @@ -472,6 +477,54 @@ func commandVtTabletStreamHealth(ctx context.Context, wr *wrangler.Wrangler, sub return nil } +func commandVtTabletUpdateStream(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { + count := subFlags.Int("count", 1, "number of responses to wait for") + timestamp := subFlags.Int("timestamp", 0, "timestamp to start the stream from") + position := subFlags.String("position", "", "position to start the stream from") + connectTimeout := subFlags.Duration("connect_timeout", 30*time.Second, "Connection timeout for vttablet client") + if err := subFlags.Parse(args); err != nil { + return err + } + if subFlags.NArg() != 1 { + return fmt.Errorf("The argument is required for the VtTabletUpdateStream command.") + } + tabletAlias, err := topoproto.ParseTabletAlias(subFlags.Arg(0)) + if err != nil { + return err + } + tabletInfo, err := wr.TopoServer().GetTablet(ctx, tabletAlias) + if err != nil { + return err + } + + conn, err := tabletconn.GetDialer()(tabletInfo.Tablet, *connectTimeout) + if err != nil { + return fmt.Errorf("cannot connect to tablet %v: %v", tabletAlias, err) + } + + stream, err := conn.UpdateStream(ctx, &querypb.Target{ + Keyspace: tabletInfo.Tablet.Keyspace, + Shard: tabletInfo.Tablet.Shard, + TabletType: tabletInfo.Tablet.Type, + }, *position, int64(*timestamp)) + if err != nil { + return err + } + for i := 0; i < *count; i++ { + se, err := stream.Recv() + if err != nil { + return fmt.Errorf("stream ended early: %v", err) + } + data, err := json.Marshal(se) + if err != nil { + wr.Logger().Errorf("cannot json-marshal structure: %v", err) + } else { + wr.Logger().Printf("%v\n", string(data)) + } + } + return nil +} + // loggerWriter turns a Logger into a Writer by decorating it with a Write() // method that sends everything to Logger.Printf(). type loggerWriter struct { diff --git a/go/vt/vtgate/fakerpcvtgateconn/conn.go b/go/vt/vtgate/fakerpcvtgateconn/conn.go index 5a223b1720..96002a2019 100644 --- a/go/vt/vtgate/fakerpcvtgateconn/conn.go +++ b/go/vt/vtgate/fakerpcvtgateconn/conn.go @@ -426,6 +426,11 @@ func (conn *FakeVTGateConn) GetSrvKeyspace(ctx context.Context, keyspace string) return nil, fmt.Errorf("NYI") } +// UpdateStream please see vtgateconn.Impl.UpdateStream +func (conn *FakeVTGateConn) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken) (vtgateconn.UpdateStreamReader, error) { + return nil, fmt.Errorf("NYI") +} + // Close please see vtgateconn.Impl.Close func (conn *FakeVTGateConn) Close() { } diff --git a/go/vt/vtgate/grpcvtgateconn/conn.go b/go/vt/vtgate/grpcvtgateconn/conn.go index 8b7ee0db4d..a4db633108 100644 --- a/go/vt/vtgate/grpcvtgateconn/conn.go +++ b/go/vt/vtgate/grpcvtgateconn/conn.go @@ -457,6 +457,40 @@ func (conn *vtgateConn) GetSrvKeyspace(ctx context.Context, keyspace string) (*t return response.SrvKeyspace, nil } +type updateStreamAdapter struct { + stream vtgateservicepb.Vitess_UpdateStreamClient +} + +func (a *updateStreamAdapter) Recv() (*querypb.StreamEvent, int64, error) { + r, err := a.stream.Recv() + if err != nil { + if err != io.EOF { + err = vterrors.FromGRPCError(err) + } + return nil, 0, err + } + return r.Event, r.ResumeTimestamp, nil +} + +func (conn *vtgateConn) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken) (vtgateconn.UpdateStreamReader, error) { + req := &vtgatepb.UpdateStreamRequest{ + CallerId: callerid.EffectiveCallerIDFromContext(ctx), + Keyspace: keyspace, + Shard: shard, + KeyRange: keyRange, + TabletType: tabletType, + Timestamp: timestamp, + Event: event, + } + stream, err := conn.c.UpdateStream(ctx, req) + if err != nil { + return nil, vterrors.FromGRPCError(err) + } + return &updateStreamAdapter{ + stream: stream, + }, nil +} + func (conn *vtgateConn) Close() { conn.cc.Close() } diff --git a/go/vt/vtgate/grpcvtgateservice/server.go b/go/vt/vtgate/grpcvtgateservice/server.go index 8870ba76bc..a3903d3e6e 100644 --- a/go/vt/vtgate/grpcvtgateservice/server.go +++ b/go/vt/vtgate/grpcvtgateservice/server.go @@ -22,6 +22,7 @@ import ( "github.com/youtube/vitess/go/vt/vtgate/vtgateservice" "golang.org/x/net/context" + querypb "github.com/youtube/vitess/go/vt/proto/query" vtgatepb "github.com/youtube/vitess/go/vt/proto/vtgate" vtgateservicepb "github.com/youtube/vitess/go/vt/proto/vtgateservice" vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" @@ -390,6 +391,26 @@ func (vtg *VTGate) GetSrvKeyspace(ctx context.Context, request *vtgatepb.GetSrvK }, nil } +// UpdateStream is the RPC version of vtgateservice.VTGateService method +func (vtg *VTGate) UpdateStream(request *vtgatepb.UpdateStreamRequest, stream vtgateservicepb.Vitess_UpdateStreamServer) (err error) { + defer vtg.server.HandlePanic(&err) + ctx := withCallerIDContext(stream.Context(), request.CallerId) + vtgErr := vtg.server.UpdateStream(ctx, + request.Keyspace, + request.Shard, + request.KeyRange, + request.TabletType, + request.Timestamp, + request.Event, + func(event *querypb.StreamEvent, resumeTimestamp int64) error { + return stream.Send(&vtgatepb.UpdateStreamResponse{ + Event: event, + ResumeTimestamp: resumeTimestamp, + }) + }) + return vterrors.ToGRPCError(vtgErr) +} + func init() { vtgate.RegisterVTGates = append(vtgate.RegisterVTGates, func(vtGate vtgateservice.VTGateService) { if servenv.GRPCCheckServiceMap("vtgateservice") { diff --git a/go/vt/vtgate/resolver.go b/go/vt/vtgate/resolver.go index 064c76b552..aaf68c3f6a 100644 --- a/go/vt/vtgate/resolver.go +++ b/go/vt/vtgate/resolver.go @@ -21,6 +21,7 @@ import ( "github.com/youtube/vitess/go/vt/vtgate/gateway" "golang.org/x/net/context" + querypb "github.com/youtube/vitess/go/vt/proto/query" topodatapb "github.com/youtube/vitess/go/vt/proto/topodata" vtgatepb "github.com/youtube/vitess/go/vt/proto/vtgate" vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc" @@ -302,7 +303,7 @@ func (res *Resolver) StreamExecuteKeyspaceIds(ctx context.Context, sql string, b tabletType, keyspaceIds) } - return res.StreamExecute(ctx, sql, bindVariables, keyspace, tabletType, mapToShards, sendReply) + return res.streamExecute(ctx, sql, bindVariables, keyspace, tabletType, mapToShards, sendReply) } // StreamExecuteKeyRanges executes a streaming query on the specified KeyRanges. @@ -321,14 +322,14 @@ func (res *Resolver) StreamExecuteKeyRanges(ctx context.Context, sql string, bin tabletType, keyRanges) } - return res.StreamExecute(ctx, sql, bindVariables, keyspace, tabletType, mapToShards, sendReply) + return res.streamExecute(ctx, sql, bindVariables, keyspace, tabletType, mapToShards, sendReply) } -// StreamExecute executes a streaming query on shards resolved by given func. +// streamExecute executes a streaming query on shards resolved by given func. // This function currently temporarily enforces the restriction of executing on // one shard since it cannot merge-sort the results to guarantee ordering of // response which is needed for checkpointing. -func (res *Resolver) StreamExecute( +func (res *Resolver) streamExecute( ctx context.Context, sql string, bindVars map[string]interface{}, @@ -362,6 +363,54 @@ func (res *Resolver) Rollback(ctx context.Context, inSession *vtgatepb.Session) return res.scatterConn.Rollback(ctx, NewSafeSession(inSession)) } +// UpdateStream streams the events. +// TODO(alainjobart): Implement the multi-shards merge code. +func (res *Resolver) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + if shard != "" { + // If we pass in a shard, resolve the keyspace following redirects. + var err error + keyspace, _, _, err = getKeyspaceShards(ctx, res.toposerv, res.cell, keyspace, tabletType) + if err != nil { + return err + } + } else { + // If we pass in a KeyRange, resolve it to one shard only for now. + var shards []string + var err error + keyspace, shards, err = mapExactShards( + ctx, + res.toposerv, + res.cell, + keyspace, + tabletType, + keyRange) + if err != nil { + return err + } + if len(shards) != 1 { + return fmt.Errorf("UpdateStream only supports exactly one shard per keyrange at the moment, but provided keyrange %v maps to %v.", keyRange, shards) + } + shard = shards[0] + } + + // Just send it to ScatterConn. With just one connection, the + // timestamp to resume from is the one we get. + // Also use the incoming event if the shard matches. + position := "" + if event != nil && event.Shard == shard { + position = event.Position + timestamp = 0 + } + return res.scatterConn.UpdateStream(ctx, keyspace, shard, tabletType, timestamp, position, func(se *querypb.StreamEvent) error { + var timestamp int64 + if se.EventToken != nil { + timestamp = se.EventToken.Timestamp + se.EventToken.Shard = shard + } + return sendReply(se, timestamp) + }) +} + // GetGatewayCacheStatus returns a displayable version of the Gateway cache. func (res *Resolver) GetGatewayCacheStatus() gateway.TabletCacheStatusList { return res.scatterConn.GetGatewayCacheStatus() diff --git a/go/vt/vtgate/scatter_conn.go b/go/vt/vtgate/scatter_conn.go index db895d7e6f..fa91bb17f5 100644 --- a/go/vt/vtgate/scatter_conn.go +++ b/go/vt/vtgate/scatter_conn.go @@ -492,6 +492,25 @@ func (stc *ScatterConn) Rollback(ctx context.Context, session *SafeSession) (err return nil } +// UpdateStream just sends the query to the gateway, +// and sends the results back. +func (stc *ScatterConn) UpdateStream(ctx context.Context, keyspace string, shard string, tabletType topodatapb.TabletType, timestamp int64, position string, sendReply func(*querypb.StreamEvent) error) error { + ser, err := stc.gateway.UpdateStream(ctx, keyspace, shard, tabletType, position, timestamp) + if err != nil { + return err + } + + for { + se, err := ser.Recv() + if err != nil { + return err + } + if err = sendReply(se); err != nil { + return err + } + } +} + // SplitQueryKeyRange scatters a SplitQuery request to all shards. For a set of // splits received from a shard, it construct a KeyRange queries by // appending that shard's keyrange to the splits. Aggregates all splits across diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 282ff33d49..ea21a291e5 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -80,6 +80,7 @@ type VTGate struct { logStreamExecuteKeyspaceIds *logutil.ThrottledLogger logStreamExecuteKeyRanges *logutil.ThrottledLogger logStreamExecuteShards *logutil.ThrottledLogger + logUpdateStream *logutil.ThrottledLogger } // RegisterVTGate defines the type of registration mechanism. @@ -111,6 +112,7 @@ func Init(ctx context.Context, hc discovery.HealthCheck, topoServer topo.Server, logStreamExecuteKeyspaceIds: logutil.NewThrottledLogger("StreamExecuteKeyspaceIds", 5*time.Second), logStreamExecuteKeyRanges: logutil.NewThrottledLogger("StreamExecuteKeyRanges", 5*time.Second), logStreamExecuteShards: logutil.NewThrottledLogger("StreamExecuteShards", 5*time.Second), + logUpdateStream: logutil.NewThrottledLogger("UpdateStream", 5*time.Second), } // Resuse resolver's scatterConn. rpcVTGate.router = NewRouter(ctx, serv, cell, "VTGateRouter", rpcVTGate.resolver.scatterConn) @@ -382,7 +384,6 @@ func (vtg *VTGate) StreamExecute(ctx context.Context, sql string, bindVariables statsKey := []string{"StreamExecute", "Any", ltt} defer vtg.timings.Record(statsKey, startTime) - var rowCount int64 err := vtg.router.StreamExecute( ctx, sql, @@ -390,7 +391,6 @@ func (vtg *VTGate) StreamExecute(ctx context.Context, sql string, bindVariables keyspace, tabletType, func(reply *sqltypes.Result) error { - rowCount += int64(len(reply.Rows)) vtg.rowsReturned.Add(statsKey, int64(len(reply.Rows))) return sendReply(reply) }) @@ -420,7 +420,6 @@ func (vtg *VTGate) StreamExecuteKeyspaceIds(ctx context.Context, sql string, bin statsKey := []string{"StreamExecuteKeyspaceIds", keyspace, ltt} defer vtg.timings.Record(statsKey, startTime) - var rowCount int64 err := vtg.resolver.StreamExecuteKeyspaceIds( ctx, sql, @@ -429,7 +428,6 @@ func (vtg *VTGate) StreamExecuteKeyspaceIds(ctx context.Context, sql string, bin keyspaceIds, tabletType, func(reply *sqltypes.Result) error { - rowCount += int64(len(reply.Rows)) vtg.rowsReturned.Add(statsKey, int64(len(reply.Rows))) return sendReply(reply) }) @@ -460,7 +458,6 @@ func (vtg *VTGate) StreamExecuteKeyRanges(ctx context.Context, sql string, bindV statsKey := []string{"StreamExecuteKeyRanges", keyspace, ltt} defer vtg.timings.Record(statsKey, startTime) - var rowCount int64 err := vtg.resolver.StreamExecuteKeyRanges( ctx, sql, @@ -469,7 +466,6 @@ func (vtg *VTGate) StreamExecuteKeyRanges(ctx context.Context, sql string, bindV keyRanges, tabletType, func(reply *sqltypes.Result) error { - rowCount += int64(len(reply.Rows)) vtg.rowsReturned.Add(statsKey, int64(len(reply.Rows))) return sendReply(reply) }) @@ -495,8 +491,7 @@ func (vtg *VTGate) StreamExecuteShards(ctx context.Context, sql string, bindVari statsKey := []string{"StreamExecuteShards", keyspace, ltt} defer vtg.timings.Record(statsKey, startTime) - var rowCount int64 - err := vtg.resolver.StreamExecute( + err := vtg.resolver.streamExecute( ctx, sql, bindVariables, @@ -506,7 +501,6 @@ func (vtg *VTGate) StreamExecuteShards(ctx context.Context, sql string, bindVari return keyspace, shards, nil }, func(reply *sqltypes.Result) error { - rowCount += int64(len(reply.Rows)) vtg.rowsReturned.Add(statsKey, int64(len(reply.Rows))) return sendReply(reply) }) @@ -722,6 +716,37 @@ func (vtg *VTGate) GetSrvKeyspace(ctx context.Context, keyspace string) (*topoda return vtg.resolver.toposerv.GetSrvKeyspace(ctx, vtg.resolver.cell, keyspace) } +// UpdateStream is part of the vtgate service API. +func (vtg *VTGate) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + startTime := time.Now() + ltt := topoproto.TabletTypeLString(tabletType) + statsKey := []string{"UpdateStream", keyspace, ltt} + defer vtg.timings.Record(statsKey, startTime) + + err := vtg.resolver.UpdateStream( + ctx, + keyspace, + shard, + keyRange, + tabletType, + timestamp, + event, + sendReply, + ) + if err != nil { + normalErrors.Add(statsKey, 1) + query := map[string]interface{}{ + "Keyspace": keyspace, + "Shard": shard, + "KeyRange": keyRange, + "TabletType": ltt, + "Timestamp": timestamp, + } + logError(err, query, vtg.logUpdateStream) + } + return formatError(err) +} + // GetGatewayCacheStatus returns a displayable version of the Gateway cache. func (vtg *VTGate) GetGatewayCacheStatus() gateway.TabletCacheStatusList { return vtg.resolver.GetGatewayCacheStatus() diff --git a/go/vt/vtgate/vtgateconn/vtgateconn.go b/go/vt/vtgate/vtgateconn/vtgateconn.go index d16323a765..316e158102 100644 --- a/go/vt/vtgate/vtgateconn/vtgateconn.go +++ b/go/vt/vtgate/vtgateconn/vtgateconn.go @@ -160,6 +160,21 @@ func (conn *VTGateConn) GetSrvKeyspace(ctx context.Context, keyspace string) (*t return conn.impl.GetSrvKeyspace(ctx, keyspace) } +// UpdateStreamReader is returned by UpdateStream. +type UpdateStreamReader interface { + // Recv returns the next result on the stream. + // It will return io.EOF if the stream ended. + Recv() (*querypb.StreamEvent, int64, error) +} + +// UpdateStream executes a streaming query on vtgate. It returns an +// UpdateStreamReader and an error. First check the error. Then you +// can pull values from the UpdateStreamReader until io.EOF, or +// another error. +func (conn *VTGateConn) UpdateStream(ctx context.Context, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken) (UpdateStreamReader, error) { + return conn.impl.UpdateStream(ctx, conn.keyspace, shard, keyRange, tabletType, timestamp, event) +} + // VTGateTx defines an ongoing transaction. // It should not be concurrently used across goroutines. type VTGateTx struct { @@ -325,13 +340,15 @@ type Impl interface { // GetSrvKeyspace returns a topo.SrvKeyspace. GetSrvKeyspace(ctx context.Context, keyspace string) (*topodatapb.SrvKeyspace, error) + // UpdateStream asks for a stream of StreamEvent. + UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken) (UpdateStreamReader, error) + // Close must be called for releasing resources. Close() } -// DialerFunc represents a function that will return a VTGateConn -// object that can communicate with a VTGate. Keyspace is only used -// for Execute and StreamExecute calls. +// DialerFunc represents a function that will return an Impl +// object that can communicate with a VTGate. type DialerFunc func(ctx context.Context, address string, timeout time.Duration) (Impl, error) var dialers = make(map[string]DialerFunc) diff --git a/go/vt/vtgate/vtgateconntest/client.go b/go/vt/vtgate/vtgateconntest/client.go index 41db630e36..0d5e42deed 100644 --- a/go/vt/vtgate/vtgateconntest/client.go +++ b/go/vt/vtgate/vtgateconntest/client.go @@ -684,6 +684,74 @@ func (f *fakeVTGateService) GetSrvKeyspace(ctx context.Context, keyspace string) return getSrvKeyspaceResult, nil } +// queryUpdateStream contains all the fields we use to test UpdateStream +type queryUpdateStream struct { + Keyspace string + Shard string + KeyRange *topodatapb.KeyRange + TabletType topodatapb.TabletType + Timestamp int64 + Event *querypb.EventToken +} + +// UpdateStream is part of the VTGateService interface +func (f *fakeVTGateService) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error { + if f.panics { + panic(fmt.Errorf("test forced panic")) + } + execCase, ok := execMap[shard] + if !ok { + return fmt.Errorf("no match for: %s", shard) + } + f.checkCallerID(ctx, "UpdateStream") + query := &queryUpdateStream{ + Keyspace: keyspace, + Shard: shard, + KeyRange: keyRange, + TabletType: tabletType, + Timestamp: timestamp, + Event: event, + } + if !reflect.DeepEqual(query, execCase.updateStreamQuery) { + f.t.Errorf("UpdateStream: %+v, want %+v", query, execCase.updateStreamQuery) + return nil + } + if execCase.result != nil { + // The first result only has statement with fields. + result := &querypb.StreamEvent{ + Statements: []*querypb.StreamEvent_Statement{ + { + PrimaryKeyFields: execCase.result.Fields, + }, + }, + } + if err := sendReply(result, int64(execCase.result.RowsAffected)); err != nil { + return err + } + if f.hasError { + // wait until the client has the response, since all streaming implementation may not + // send previous messages if an error has been triggered. + <-f.errorWait + f.errorWait = make(chan struct{}) // for next test + return errTestVtGateError + } + for _, row := range execCase.result.Rows { + + result := &querypb.StreamEvent{ + Statements: []*querypb.StreamEvent_Statement{ + { + PrimaryKeyValues: sqltypes.RowsToProto3([][]sqltypes.Value{row}), + }, + }, + } + if err := sendReply(result, int64(execCase.result.RowsAffected)); err != nil { + return err + } + } + } + return nil +} + // CreateFakeServer returns the fake server for the tests func CreateFakeServer(t *testing.T) vtgateservice.VTGateService { return &fakeVTGateService{ @@ -737,6 +805,7 @@ func TestSuite(t *testing.T, impl vtgateconn.Impl, fakeServer vtgateservice.VTGa testSplitQuery(t, conn) testSplitQueryV2(t, conn) testGetSrvKeyspace(t, conn) + testUpdateStream(t, conn) // force a panic at every call, then test that works fs.panics = true @@ -757,6 +826,7 @@ func TestSuite(t *testing.T, impl vtgateconn.Impl, fakeServer vtgateservice.VTGa testSplitQueryPanic(t, conn) testSplitQueryV2Panic(t, conn) testGetSrvKeyspacePanic(t, conn) + testUpdateStreamPanic(t, conn) fs.panics = false } @@ -788,6 +858,7 @@ func TestErrorSuite(t *testing.T, fakeServer vtgateservice.VTGateService) { testSplitQueryError(t, conn) testSplitQueryV2Error(t, conn) testGetSrvKeyspaceError(t, conn) + testUpdateStreamError(t, conn, fs) fs.hasError = false } @@ -1739,6 +1810,88 @@ func testGetSrvKeyspacePanic(t *testing.T, conn *vtgateconn.VTGateConn) { expectPanic(t, err) } +func testUpdateStream(t *testing.T, conn *vtgateconn.VTGateConn) { + ctx := newContext() + execCase := execMap["request1"] + stream, err := conn.UpdateStream(ctx, execCase.updateStreamQuery.Shard, execCase.updateStreamQuery.KeyRange, execCase.execQuery.TabletType, execCase.updateStreamQuery.Timestamp, execCase.updateStreamQuery.Event) + if err != nil { + t.Fatal(err) + } + var qr querypb.QueryResult + for { + packet, resumeTimestamp, err := stream.Recv() + if err != nil { + if err != io.EOF { + t.Error(err) + } + break + } + qr.RowsAffected = uint64(resumeTimestamp) + if len(packet.Statements[0].PrimaryKeyFields) != 0 { + qr.Fields = packet.Statements[0].PrimaryKeyFields + } + if len(packet.Statements[0].PrimaryKeyValues) != 0 { + qr.Rows = append(qr.Rows, packet.Statements[0].PrimaryKeyValues...) + } + } + + sqr := sqltypes.Proto3ToResult(&qr) + wantResult := *execCase.result + wantResult.InsertID = 0 + if !reflect.DeepEqual(sqr, &wantResult) { + t.Errorf("Unexpected result from Execute: got %+v want %+v", sqr, wantResult) + } + + stream, err = conn.UpdateStream(ctx, "none", nil, topodatapb.TabletType_RDONLY, 0, nil) + if err != nil { + t.Fatal(err) + } + _, _, err = stream.Recv() + want := "no match for: none" + if err == nil || !strings.Contains(err.Error(), want) { + t.Errorf("none request: %v, want %v", err, want) + } +} + +func testUpdateStreamError(t *testing.T, conn *vtgateconn.VTGateConn, fake *fakeVTGateService) { + ctx := newContext() + execCase := execMap["request1"] + stream, err := conn.UpdateStream(ctx, execCase.updateStreamQuery.Shard, execCase.updateStreamQuery.KeyRange, execCase.execQuery.TabletType, execCase.updateStreamQuery.Timestamp, execCase.updateStreamQuery.Event) + if err != nil { + t.Fatalf("UpdateStream failed: %v", err) + } + qr, _, err := stream.Recv() + if err != nil { + t.Fatalf("UpdateStream failed: cannot read result1: %v", err) + } + + if !reflect.DeepEqual(qr.Statements[0].PrimaryKeyFields, execCase.result.Fields) { + t.Errorf("Unexpected result from UpdateStream: got %#v want %#v", qr.Statements[0].PrimaryKeyFields, execCase.result.Fields) + } + // signal to the server that the first result has been received + close(fake.errorWait) + // After 1 result, we expect to get an error (no more results). + qr, _, err = stream.Recv() + if err == nil { + t.Fatalf("UpdateStream channel wasn't closed") + } + verifyError(t, err, "UpdateStream") +} + +func testUpdateStreamPanic(t *testing.T, conn *vtgateconn.VTGateConn) { + ctx := newContext() + execCase := execMap["request1"] + stream, err := conn.UpdateStream(ctx, execCase.updateStreamQuery.Shard, execCase.updateStreamQuery.KeyRange, execCase.execQuery.TabletType, execCase.updateStreamQuery.Timestamp, execCase.updateStreamQuery.Event) + if err != nil { + t.Fatal(err) + } + _, _, err = stream.Recv() + if err == nil { + t.Fatalf("Received packets instead of panic?") + } + expectPanic(t, err) +} + var testCallerID = &vtrpcpb.CallerID{ Principal: "test_principal", Component: "test_component", @@ -1753,6 +1906,7 @@ var execMap = map[string]struct { entityIdsQuery *queryExecuteEntityIds batchQueryShard *queryExecuteBatchShards keyspaceIDBatchQuery *queryExecuteBatchKeyspaceIds + updateStreamQuery *queryUpdateStream result *sqltypes.Result outSession *vtgatepb.Session err error @@ -1863,6 +2017,21 @@ var execMap = map[string]struct { AsTransaction: true, Session: nil, }, + updateStreamQuery: &queryUpdateStream{ + Keyspace: "connection_ks", + Shard: "request1", + KeyRange: &topodatapb.KeyRange{ + Start: []byte{0x72}, + End: []byte{0x90}, + }, + TabletType: topodatapb.TabletType_RDONLY, + Timestamp: 123789, + Event: &querypb.EventToken{ + Timestamp: 1234567, + Shard: "request1", + Position: "streaming_position", + }, + }, result: &result1, outSession: nil, }, @@ -1972,6 +2141,21 @@ var execMap = map[string]struct { AsTransaction: false, Session: nil, }, + updateStreamQuery: &queryUpdateStream{ + Keyspace: "connection_ks", + Shard: "errorRequst", + KeyRange: &topodatapb.KeyRange{ + Start: []byte{0x72}, + End: []byte{0x90}, + }, + TabletType: topodatapb.TabletType_RDONLY, + Timestamp: 123789, + Event: &querypb.EventToken{ + Timestamp: 1234567, + Shard: "request1", + Position: "streaming_position", + }, + }, result: nil, outSession: nil, }, @@ -2079,6 +2263,21 @@ var execMap = map[string]struct { TabletType: topodatapb.TabletType_RDONLY, Session: session1, }, + updateStreamQuery: &queryUpdateStream{ + Keyspace: "connection_ks", + Shard: "txRequest", + KeyRange: &topodatapb.KeyRange{ + Start: []byte{0x72}, + End: []byte{0x90}, + }, + TabletType: topodatapb.TabletType_RDONLY, + Timestamp: 123789, + Event: &querypb.EventToken{ + Timestamp: 1234567, + Shard: "request1", + Position: "streaming_position", + }, + }, result: nil, outSession: session2, }, diff --git a/go/vt/vtgate/vtgateservice/interface.go b/go/vt/vtgate/vtgateservice/interface.go index 5e8f7d8661..7f02014f90 100644 --- a/go/vt/vtgate/vtgateservice/interface.go +++ b/go/vt/vtgate/vtgateservice/interface.go @@ -20,6 +20,7 @@ import ( type VTGateService interface { // Regular query execution. // All these methods can change the provided session. + Execute(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, tabletType topodatapb.TabletType, session *vtgatepb.Session, notInTransaction bool) (*sqltypes.Result, error) ExecuteShards(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, shards []string, tabletType topodatapb.TabletType, session *vtgatepb.Session, notInTransaction bool) (*sqltypes.Result, error) ExecuteKeyspaceIds(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, keyspaceIds [][]byte, tabletType topodatapb.TabletType, session *vtgatepb.Session, notInTransaction bool) (*sqltypes.Result, error) @@ -29,20 +30,21 @@ type VTGateService interface { ExecuteBatchKeyspaceIds(ctx context.Context, queries []*vtgatepb.BoundKeyspaceIdQuery, tabletType topodatapb.TabletType, asTransaction bool, session *vtgatepb.Session) ([]sqltypes.Result, error) // Streaming queries + StreamExecute(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, tabletType topodatapb.TabletType, sendReply func(*sqltypes.Result) error) error StreamExecuteShards(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, shards []string, tabletType topodatapb.TabletType, sendReply func(*sqltypes.Result) error) error StreamExecuteKeyspaceIds(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, keyspaceIds [][]byte, tabletType topodatapb.TabletType, sendReply func(*sqltypes.Result) error) error StreamExecuteKeyRanges(ctx context.Context, sql string, bindVariables map[string]interface{}, keyspace string, keyRanges []*topodatapb.KeyRange, tabletType topodatapb.TabletType, sendReply func(*sqltypes.Result) error) error // Transaction management + Begin(ctx context.Context) (*vtgatepb.Session, error) Commit(ctx context.Context, session *vtgatepb.Session) error Rollback(ctx context.Context, session *vtgatepb.Session) error // Map Reduce support - SplitQuery(ctx context.Context, keyspace string, sql string, bindVariables map[string]interface{}, splitColumn string, splitCount int64) ([]*vtgatepb.SplitQueryResponse_Part, error) - // MapReduce support + SplitQuery(ctx context.Context, keyspace string, sql string, bindVariables map[string]interface{}, splitColumn string, splitCount int64) ([]*vtgatepb.SplitQueryResponse_Part, error) // TODO(erez): Rename to SplitQuery after migration to SplitQuery V2. SplitQueryV2( ctx context.Context, @@ -55,8 +57,13 @@ type VTGateService interface { algorithm querypb.SplitQueryRequest_Algorithm) ([]*vtgatepb.SplitQueryResponse_Part, error) // Topology support + GetSrvKeyspace(ctx context.Context, keyspace string) (*topodatapb.SrvKeyspace, error) + // Update Stream methods + + UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodatapb.KeyRange, tabletType topodatapb.TabletType, timestamp int64, event *querypb.EventToken, sendReply func(*querypb.StreamEvent, int64) error) error + // HandlePanic should be called with defer at the beginning of each // RPC implementation method, before calling any of the previous methods HandlePanic(err *error) diff --git a/go/vt/vtgate/vtgateservice/vtgateservice_testing/mock_vtgateservice.go b/go/vt/vtgate/vtgateservice/vtgateservice_testing/mock_vtgateservice.go index 1d39415592..60e41f5e85 100644 --- a/go/vt/vtgate/vtgateservice/vtgateservice_testing/mock_vtgateservice.go +++ b/go/vt/vtgate/vtgateservice/vtgateservice_testing/mock_vtgateservice.go @@ -214,6 +214,16 @@ func (_mr *_MockVTGateServiceRecorder) GetSrvKeyspace(arg0, arg1 interface{}) *g return _mr.mock.ctrl.RecordCall(_mr.mock, "GetSrvKeyspace", arg0, arg1) } +func (_m *MockVTGateService) UpdateStream(ctx context.Context, keyspace string, shard string, keyRange *topodata.KeyRange, tabletType topodata.TabletType, timestamp int64, event *query.EventToken, sendReply func(*query.StreamEvent, int64) error) error { + ret := _m.ctrl.Call(_m, "UpdateStream", ctx, keyspace, shard, keyRange, tabletType, timestamp, event, sendReply) + ret0, _ := ret[0].(error) + return ret0 +} + +func (_mr *_MockVTGateServiceRecorder) UpdateStream(arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7 interface{}) *gomock.Call { + return _mr.mock.ctrl.RecordCall(_mr.mock, "UpdateStream", arg0, arg1, arg2, arg3, arg4, arg5, arg6, arg7) +} + func (_m *MockVTGateService) HandlePanic(err *error) { _m.ctrl.Call(_m, "HandlePanic", err) } diff --git a/php/src/Vitess/Proto/Automationservice/AutomationClient.php b/php/src/Vitess/Proto/Automationservice/AutomationClient.php index 949aba654e..b5f9f829a3 100644 --- a/php/src/Vitess/Proto/Automationservice/AutomationClient.php +++ b/php/src/Vitess/Proto/Automationservice/AutomationClient.php @@ -6,8 +6,8 @@ namespace Vitess\Proto\Automationservice { class AutomationClient extends \Grpc\BaseStub { - public function __construct($hostname, $opts, $channel = null) { - parent::__construct($hostname, $opts, $channel); + public function __construct($hostname, $opts) { + parent::__construct($hostname, $opts); } /** * @param Vitess\Proto\Automation\EnqueueClusterOperationRequest $input diff --git a/php/src/Vitess/Proto/Binlogdata/StreamEvent/Category.php b/php/src/Vitess/Proto/Binlogdata/StreamEvent/Category.php deleted file mode 100644 index b40a3c9955..0000000000 --- a/php/src/Vitess/Proto/Binlogdata/StreamEvent/Category.php +++ /dev/null @@ -1,13 +0,0 @@ -number = 1; - $f->name = "stream_event"; - $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; - $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; - $f->reference = '\Vitess\Proto\Binlogdata\StreamEvent'; - $descriptor->addField($f); - - foreach (self::$__extensions as $cb) { - $descriptor->addField($cb(), true); - } - - return $descriptor; - } - - /** - * Check if has a value - * - * @return boolean - */ - public function hasStreamEvent(){ - return $this->_has(1); - } - - /** - * Clear value - * - * @return \Vitess\Proto\Binlogdata\StreamUpdateResponse - */ - public function clearStreamEvent(){ - return $this->_clear(1); - } - - /** - * Get value - * - * @return \Vitess\Proto\Binlogdata\StreamEvent - */ - public function getStreamEvent(){ - return $this->_get(1); - } - - /** - * Set value - * - * @param \Vitess\Proto\Binlogdata\StreamEvent $value - * @return \Vitess\Proto\Binlogdata\StreamUpdateResponse - */ - public function setStreamEvent(\Vitess\Proto\Binlogdata\StreamEvent $value){ - return $this->_set(1, $value); - } - } -} - diff --git a/php/src/Vitess/Proto/Binlogservice/UpdateStreamClient.php b/php/src/Vitess/Proto/Binlogservice/UpdateStreamClient.php index 74d861e797..f4a0137591 100644 --- a/php/src/Vitess/Proto/Binlogservice/UpdateStreamClient.php +++ b/php/src/Vitess/Proto/Binlogservice/UpdateStreamClient.php @@ -9,12 +9,6 @@ namespace Vitess\Proto\Binlogservice { public function __construct($hostname, $opts) { parent::__construct($hostname, $opts); } - /** - * @param Vitess\Proto\Binlogdata\StreamUpdateRequest $input - */ - public function StreamUpdate($argument, $metadata = array(), $options = array()) { - return $this->_serverStreamRequest('/binlogservice.UpdateStream/StreamUpdate', $argument, '\Vitess\Proto\Binlogdata\StreamUpdateResponse::deserialize', $metadata, $options); - } /** * @param Vitess\Proto\Binlogdata\StreamKeyRangeRequest $input */ diff --git a/php/src/Vitess/Proto/Mysqlctl/MysqlCtlClient.php b/php/src/Vitess/Proto/Mysqlctl/MysqlCtlClient.php index c29011a071..47c8a08239 100644 --- a/php/src/Vitess/Proto/Mysqlctl/MysqlCtlClient.php +++ b/php/src/Vitess/Proto/Mysqlctl/MysqlCtlClient.php @@ -6,8 +6,8 @@ namespace Vitess\Proto\Mysqlctl { class MysqlCtlClient extends \Grpc\BaseStub { - public function __construct($hostname, $opts, $channel = null) { - parent::__construct($hostname, $opts, $channel); + public function __construct($hostname, $opts) { + parent::__construct($hostname, $opts); } /** * @param Vitess\Proto\Mysqlctl\StartRequest $input diff --git a/php/src/Vitess/Proto/Query/StreamEvent.php b/php/src/Vitess/Proto/Query/StreamEvent.php new file mode 100644 index 0000000000..cff84eabad --- /dev/null +++ b/php/src/Vitess/Proto/Query/StreamEvent.php @@ -0,0 +1,143 @@ +number = 1; + $f->name = "statements"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_REPEATED; + $f->reference = '\Vitess\Proto\Query\StreamEvent\Statement'; + $descriptor->addField($f); + + // OPTIONAL MESSAGE event_token = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "event_token"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\EventToken'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasStatements(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\StreamEvent + */ + public function clearStatements(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Query\StreamEvent\Statement + */ + public function getStatements($idx = NULL){ + return $this->_get(1, $idx); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\StreamEvent\Statement $value + * @return \Vitess\Proto\Query\StreamEvent + */ + public function setStatements(\Vitess\Proto\Query\StreamEvent\Statement $value, $idx = NULL){ + return $this->_set(1, $value, $idx); + } + + /** + * Get all elements of + * + * @return \Vitess\Proto\Query\StreamEvent\Statement[] + */ + public function getStatementsList(){ + return $this->_get(1); + } + + /** + * Add a new element to + * + * @param \Vitess\Proto\Query\StreamEvent\Statement $value + * @return \Vitess\Proto\Query\StreamEvent + */ + public function addStatements(\Vitess\Proto\Query\StreamEvent\Statement $value){ + return $this->_add(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEventToken(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\StreamEvent + */ + public function clearEventToken(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\EventToken + */ + public function getEventToken(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\EventToken $value + * @return \Vitess\Proto\Query\StreamEvent + */ + public function setEventToken(\Vitess\Proto\Query\EventToken $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Binlogdata/StreamEvent.php b/php/src/Vitess/Proto/Query/StreamEvent/Statement.php similarity index 73% rename from php/src/Vitess/Proto/Binlogdata/StreamEvent.php rename to php/src/Vitess/Proto/Query/StreamEvent/Statement.php index 4c2f207d6e..32ebee461e 100644 --- a/php/src/Vitess/Proto/Binlogdata/StreamEvent.php +++ b/php/src/Vitess/Proto/Query/StreamEvent/Statement.php @@ -1,12 +1,12 @@ name = "category"; $f->type = \DrSlump\Protobuf::TYPE_ENUM; $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; - $f->reference = '\Vitess\Proto\Binlogdata\StreamEvent\Category'; + $f->reference = '\Vitess\Proto\Query\StreamEvent\Statement\Category'; $descriptor->addField($f); // OPTIONAL STRING table_name = 2 @@ -75,15 +72,6 @@ namespace Vitess\Proto\Binlogdata { $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; $descriptor->addField($f); - // OPTIONAL MESSAGE event_token = 8 - $f = new \DrSlump\Protobuf\Field(); - $f->number = 8; - $f->name = "event_token"; - $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; - $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; - $f->reference = '\Vitess\Proto\Query\EventToken'; - $descriptor->addField($f); - foreach (self::$__extensions as $cb) { $descriptor->addField($cb(), true); } @@ -103,7 +91,7 @@ namespace Vitess\Proto\Binlogdata { /** * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function clearCategory(){ return $this->_clear(1); @@ -112,7 +100,7 @@ namespace Vitess\Proto\Binlogdata { /** * Get value * - * @return int - \Vitess\Proto\Binlogdata\StreamEvent\Category + * @return int - \Vitess\Proto\Query\StreamEvent\Statement\Category */ public function getCategory(){ return $this->_get(1); @@ -121,8 +109,8 @@ namespace Vitess\Proto\Binlogdata { /** * Set value * - * @param int - \Vitess\Proto\Binlogdata\StreamEvent\Category $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @param int - \Vitess\Proto\Query\StreamEvent\Statement\Category $value + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function setCategory( $value){ return $this->_set(1, $value); @@ -140,7 +128,7 @@ namespace Vitess\Proto\Binlogdata { /** * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function clearTableName(){ return $this->_clear(2); @@ -159,7 +147,7 @@ namespace Vitess\Proto\Binlogdata { * Set value * * @param string $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function setTableName( $value){ return $this->_set(2, $value); @@ -177,7 +165,7 @@ namespace Vitess\Proto\Binlogdata { /** * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function clearPrimaryKeyFields(){ return $this->_clear(3); @@ -197,7 +185,7 @@ namespace Vitess\Proto\Binlogdata { * Set value * * @param \Vitess\Proto\Query\Field $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function setPrimaryKeyFields(\Vitess\Proto\Query\Field $value, $idx = NULL){ return $this->_set(3, $value, $idx); @@ -216,7 +204,7 @@ namespace Vitess\Proto\Binlogdata { * Add a new element to * * @param \Vitess\Proto\Query\Field $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function addPrimaryKeyFields(\Vitess\Proto\Query\Field $value){ return $this->_add(3, $value); @@ -234,7 +222,7 @@ namespace Vitess\Proto\Binlogdata { /** * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function clearPrimaryKeyValues(){ return $this->_clear(4); @@ -254,7 +242,7 @@ namespace Vitess\Proto\Binlogdata { * Set value * * @param \Vitess\Proto\Query\Row $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function setPrimaryKeyValues(\Vitess\Proto\Query\Row $value, $idx = NULL){ return $this->_set(4, $value, $idx); @@ -273,7 +261,7 @@ namespace Vitess\Proto\Binlogdata { * Add a new element to * * @param \Vitess\Proto\Query\Row $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function addPrimaryKeyValues(\Vitess\Proto\Query\Row $value){ return $this->_add(4, $value); @@ -291,7 +279,7 @@ namespace Vitess\Proto\Binlogdata { /** * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function clearSql(){ return $this->_clear(5); @@ -310,48 +298,11 @@ namespace Vitess\Proto\Binlogdata { * Set value * * @param string $value - * @return \Vitess\Proto\Binlogdata\StreamEvent + * @return \Vitess\Proto\Query\StreamEvent\Statement */ public function setSql( $value){ return $this->_set(5, $value); } - - /** - * Check if has a value - * - * @return boolean - */ - public function hasEventToken(){ - return $this->_has(8); - } - - /** - * Clear value - * - * @return \Vitess\Proto\Binlogdata\StreamEvent - */ - public function clearEventToken(){ - return $this->_clear(8); - } - - /** - * Get value - * - * @return \Vitess\Proto\Query\EventToken - */ - public function getEventToken(){ - return $this->_get(8); - } - - /** - * Set value - * - * @param \Vitess\Proto\Query\EventToken $value - * @return \Vitess\Proto\Binlogdata\StreamEvent - */ - public function setEventToken(\Vitess\Proto\Query\EventToken $value){ - return $this->_set(8, $value); - } } } diff --git a/php/src/Vitess/Proto/Query/StreamEvent/Statement/Category.php b/php/src/Vitess/Proto/Query/StreamEvent/Statement/Category.php new file mode 100644 index 0000000000..3c65e6bc1c --- /dev/null +++ b/php/src/Vitess/Proto/Query/StreamEvent/Statement/Category.php @@ -0,0 +1,12 @@ +number = 1; + $f->name = "effective_caller_id"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Vtrpc\CallerID'; + $descriptor->addField($f); + + // OPTIONAL MESSAGE immediate_caller_id = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "immediate_caller_id"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\VTGateCallerID'; + $descriptor->addField($f); + + // OPTIONAL MESSAGE target = 3 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 3; + $f->name = "target"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\Target'; + $descriptor->addField($f); + + // OPTIONAL STRING position = 4 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 4; + $f->name = "position"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 timestamp = 5 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 5; + $f->name = "timestamp"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEffectiveCallerId(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function clearEffectiveCallerId(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return \Vitess\Proto\Vtrpc\CallerID + */ + public function getEffectiveCallerId(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param \Vitess\Proto\Vtrpc\CallerID $value + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function setEffectiveCallerId(\Vitess\Proto\Vtrpc\CallerID $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasImmediateCallerId(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function clearImmediateCallerId(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\VTGateCallerID + */ + public function getImmediateCallerId(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\VTGateCallerID $value + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function setImmediateCallerId(\Vitess\Proto\Query\VTGateCallerID $value){ + return $this->_set(2, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasTarget(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function clearTarget(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\Target + */ + public function getTarget(){ + return $this->_get(3); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\Target $value + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function setTarget(\Vitess\Proto\Query\Target $value){ + return $this->_set(3, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasPosition(){ + return $this->_has(4); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function clearPosition(){ + return $this->_clear(4); + } + + /** + * Get value + * + * @return string + */ + public function getPosition(){ + return $this->_get(4); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function setPosition( $value){ + return $this->_set(4, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasTimestamp(){ + return $this->_has(5); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function clearTimestamp(){ + return $this->_clear(5); + } + + /** + * Get value + * + * @return int + */ + public function getTimestamp(){ + return $this->_get(5); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Query\UpdateStreamRequest + */ + public function setTimestamp( $value){ + return $this->_set(5, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Query/UpdateStreamResponse.php b/php/src/Vitess/Proto/Query/UpdateStreamResponse.php new file mode 100644 index 0000000000..46c823f992 --- /dev/null +++ b/php/src/Vitess/Proto/Query/UpdateStreamResponse.php @@ -0,0 +1,74 @@ +number = 1; + $f->name = "event"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\StreamEvent'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEvent(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Query\UpdateStreamResponse + */ + public function clearEvent(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\StreamEvent + */ + public function getEvent(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\StreamEvent $value + * @return \Vitess\Proto\Query\UpdateStreamResponse + */ + public function setEvent(\Vitess\Proto\Query\StreamEvent $value){ + return $this->_set(1, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Queryservice/QueryClient.php b/php/src/Vitess/Proto/Queryservice/QueryClient.php index d66ee77931..1ef2168b64 100644 --- a/php/src/Vitess/Proto/Queryservice/QueryClient.php +++ b/php/src/Vitess/Proto/Queryservice/QueryClient.php @@ -6,8 +6,8 @@ namespace Vitess\Proto\Queryservice { class QueryClient extends \Grpc\BaseStub { - public function __construct($hostname, $opts, $channel = null) { - parent::__construct($hostname, $opts, $channel); + public function __construct($hostname, $opts) { + parent::__construct($hostname, $opts); } /** * @param Vitess\Proto\Query\ExecuteRequest $input @@ -69,5 +69,11 @@ namespace Vitess\Proto\Queryservice { public function StreamHealth($argument, $metadata = array(), $options = array()) { return $this->_serverStreamRequest('/queryservice.Query/StreamHealth', $argument, '\Vitess\Proto\Query\StreamHealthResponse::deserialize', $metadata, $options); } + /** + * @param Vitess\Proto\Query\UpdateStreamRequest $input + */ + public function UpdateStream($argument, $metadata = array(), $options = array()) { + return $this->_serverStreamRequest('/queryservice.Query/UpdateStream', $argument, '\Vitess\Proto\Query\UpdateStreamResponse::deserialize', $metadata, $options); + } } } diff --git a/php/src/Vitess/Proto/Throttlerdata/Configuration.php b/php/src/Vitess/Proto/Throttlerdata/Configuration.php new file mode 100644 index 0000000000..8279f45553 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/Configuration.php @@ -0,0 +1,505 @@ +number = 1; + $f->name = "target_replication_lag_sec"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 max_replication_lag_sec = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "max_replication_lag_sec"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 initial_rate = 3 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 3; + $f->name = "initial_rate"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL DOUBLE max_increase = 4 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 4; + $f->name = "max_increase"; + $f->type = \DrSlump\Protobuf::TYPE_DOUBLE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL DOUBLE emergency_decrease = 5 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 5; + $f->name = "emergency_decrease"; + $f->type = \DrSlump\Protobuf::TYPE_DOUBLE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 min_duration_between_changes_sec = 6 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 6; + $f->name = "min_duration_between_changes_sec"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 max_duration_between_increases_sec = 7 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 7; + $f->name = "max_duration_between_increases_sec"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT32 ignore_n_slowest_replicas = 8 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 8; + $f->name = "ignore_n_slowest_replicas"; + $f->type = \DrSlump\Protobuf::TYPE_INT32; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT64 age_bad_rate_after_sec = 9 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 9; + $f->name = "age_bad_rate_after_sec"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL DOUBLE bad_rate_increase = 10 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 10; + $f->name = "bad_rate_increase"; + $f->type = \DrSlump\Protobuf::TYPE_DOUBLE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasTargetReplicationLagSec(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearTargetReplicationLagSec(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return int + */ + public function getTargetReplicationLagSec(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setTargetReplicationLagSec( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasMaxReplicationLagSec(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearMaxReplicationLagSec(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return int + */ + public function getMaxReplicationLagSec(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setMaxReplicationLagSec( $value){ + return $this->_set(2, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasInitialRate(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearInitialRate(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return int + */ + public function getInitialRate(){ + return $this->_get(3); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setInitialRate( $value){ + return $this->_set(3, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasMaxIncrease(){ + return $this->_has(4); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearMaxIncrease(){ + return $this->_clear(4); + } + + /** + * Get value + * + * @return float + */ + public function getMaxIncrease(){ + return $this->_get(4); + } + + /** + * Set value + * + * @param float $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setMaxIncrease( $value){ + return $this->_set(4, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEmergencyDecrease(){ + return $this->_has(5); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearEmergencyDecrease(){ + return $this->_clear(5); + } + + /** + * Get value + * + * @return float + */ + public function getEmergencyDecrease(){ + return $this->_get(5); + } + + /** + * Set value + * + * @param float $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setEmergencyDecrease( $value){ + return $this->_set(5, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasMinDurationBetweenChangesSec(){ + return $this->_has(6); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearMinDurationBetweenChangesSec(){ + return $this->_clear(6); + } + + /** + * Get value + * + * @return int + */ + public function getMinDurationBetweenChangesSec(){ + return $this->_get(6); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setMinDurationBetweenChangesSec( $value){ + return $this->_set(6, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasMaxDurationBetweenIncreasesSec(){ + return $this->_has(7); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearMaxDurationBetweenIncreasesSec(){ + return $this->_clear(7); + } + + /** + * Get value + * + * @return int + */ + public function getMaxDurationBetweenIncreasesSec(){ + return $this->_get(7); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setMaxDurationBetweenIncreasesSec( $value){ + return $this->_set(7, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasIgnoreNSlowestReplicas(){ + return $this->_has(8); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearIgnoreNSlowestReplicas(){ + return $this->_clear(8); + } + + /** + * Get value + * + * @return int + */ + public function getIgnoreNSlowestReplicas(){ + return $this->_get(8); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setIgnoreNSlowestReplicas( $value){ + return $this->_set(8, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasAgeBadRateAfterSec(){ + return $this->_has(9); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearAgeBadRateAfterSec(){ + return $this->_clear(9); + } + + /** + * Get value + * + * @return int + */ + public function getAgeBadRateAfterSec(){ + return $this->_get(9); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setAgeBadRateAfterSec( $value){ + return $this->_set(9, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasBadRateIncrease(){ + return $this->_has(10); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function clearBadRateIncrease(){ + return $this->_clear(10); + } + + /** + * Get value + * + * @return float + */ + public function getBadRateIncrease(){ + return $this->_get(10); + } + + /** + * Set value + * + * @param float $value + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function setBadRateIncrease( $value){ + return $this->_set(10, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Binlogdata/StreamUpdateRequest.php b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationRequest.php similarity index 56% rename from php/src/Vitess/Proto/Binlogdata/StreamUpdateRequest.php rename to php/src/Vitess/Proto/Throttlerdata/GetConfigurationRequest.php index 8d9578baf2..bc5feec168 100644 --- a/php/src/Vitess/Proto/Binlogdata/StreamUpdateRequest.php +++ b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationRequest.php @@ -1,13 +1,13 @@ number = 1; - $f->name = "position"; + $f->name = "throttler_name"; $f->type = \DrSlump\Protobuf::TYPE_STRING; $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; $descriptor->addField($f); @@ -33,39 +33,39 @@ namespace Vitess\Proto\Binlogdata { } /** - * Check if has a value + * Check if has a value * * @return boolean */ - public function hasPosition(){ + public function hasThrottlerName(){ return $this->_has(1); } /** - * Clear value + * Clear value * - * @return \Vitess\Proto\Binlogdata\StreamUpdateRequest + * @return \Vitess\Proto\Throttlerdata\GetConfigurationRequest */ - public function clearPosition(){ + public function clearThrottlerName(){ return $this->_clear(1); } /** - * Get value + * Get value * * @return string */ - public function getPosition(){ + public function getThrottlerName(){ return $this->_get(1); } /** - * Set value + * Set value * * @param string $value - * @return \Vitess\Proto\Binlogdata\StreamUpdateRequest + * @return \Vitess\Proto\Throttlerdata\GetConfigurationRequest */ - public function setPosition( $value){ + public function setThrottlerName( $value){ return $this->_set(1, $value); } } diff --git a/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse.php b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse.php new file mode 100644 index 0000000000..78268146c1 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse.php @@ -0,0 +1,94 @@ +number = 1; + $f->name = "configurations"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_REPEATED; + $f->reference = '\Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasConfigurations(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse + */ + public function clearConfigurations(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry + */ + public function getConfigurations($idx = NULL){ + return $this->_get(1, $idx); + } + + /** + * Set value + * + * @param \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry $value + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse + */ + public function setConfigurations(\Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry $value, $idx = NULL){ + return $this->_set(1, $value, $idx); + } + + /** + * Get all elements of + * + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry[] + */ + public function getConfigurationsList(){ + return $this->_get(1); + } + + /** + * Add a new element to + * + * @param \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry $value + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse + */ + public function addConfigurations(\Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry $value){ + return $this->_add(1, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse/ConfigurationsEntry.php b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse/ConfigurationsEntry.php new file mode 100644 index 0000000000..e3277745e4 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/GetConfigurationResponse/ConfigurationsEntry.php @@ -0,0 +1,122 @@ +number = 1; + $f->name = "key"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE value = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "value"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Throttlerdata\Configuration'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasKey(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry + */ + public function clearKey(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getKey(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry + */ + public function setKey( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasValue(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry + */ + public function clearValue(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function getValue(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param \Vitess\Proto\Throttlerdata\Configuration $value + * @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry + */ + public function setValue(\Vitess\Proto\Throttlerdata\Configuration $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationRequest.php b/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationRequest.php new file mode 100644 index 0000000000..214ecfb158 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationRequest.php @@ -0,0 +1,73 @@ +number = 1; + $f->name = "throttler_name"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasThrottlerName(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\ResetConfigurationRequest + */ + public function clearThrottlerName(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getThrottlerName(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\ResetConfigurationRequest + */ + public function setThrottlerName( $value){ + return $this->_set(1, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationResponse.php b/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationResponse.php new file mode 100644 index 0000000000..32c5be4dda --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/ResetConfigurationResponse.php @@ -0,0 +1,93 @@ +number = 1; + $f->name = "names"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_REPEATED; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasNames(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\ResetConfigurationResponse + */ + public function clearNames(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @param int $idx + * @return string + */ + public function getNames($idx = NULL){ + return $this->_get(1, $idx); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\ResetConfigurationResponse + */ + public function setNames( $value, $idx = NULL){ + return $this->_set(1, $value, $idx); + } + + /** + * Get all elements of + * + * @return string[] + */ + public function getNamesList(){ + return $this->_get(1); + } + + /** + * Add a new element to + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\ResetConfigurationResponse + */ + public function addNames( $value){ + return $this->_add(1, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationRequest.php b/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationRequest.php new file mode 100644 index 0000000000..46f8edfce3 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationRequest.php @@ -0,0 +1,170 @@ +number = 1; + $f->name = "throttler_name"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE configuration = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "configuration"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Throttlerdata\Configuration'; + $descriptor->addField($f); + + // OPTIONAL BOOL copy_zero_values = 3 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 3; + $f->name = "copy_zero_values"; + $f->type = \DrSlump\Protobuf::TYPE_BOOL; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasThrottlerName(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function clearThrottlerName(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getThrottlerName(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function setThrottlerName( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasConfiguration(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function clearConfiguration(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Throttlerdata\Configuration + */ + public function getConfiguration(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param \Vitess\Proto\Throttlerdata\Configuration $value + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function setConfiguration(\Vitess\Proto\Throttlerdata\Configuration $value){ + return $this->_set(2, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasCopyZeroValues(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function clearCopyZeroValues(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return boolean + */ + public function getCopyZeroValues(){ + return $this->_get(3); + } + + /** + * Set value + * + * @param boolean $value + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest + */ + public function setCopyZeroValues( $value){ + return $this->_set(3, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationResponse.php b/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationResponse.php new file mode 100644 index 0000000000..31b36bd8c5 --- /dev/null +++ b/php/src/Vitess/Proto/Throttlerdata/UpdateConfigurationResponse.php @@ -0,0 +1,93 @@ +number = 1; + $f->name = "names"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_REPEATED; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasNames(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationResponse + */ + public function clearNames(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @param int $idx + * @return string + */ + public function getNames($idx = NULL){ + return $this->_get(1, $idx); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationResponse + */ + public function setNames( $value, $idx = NULL){ + return $this->_set(1, $value, $idx); + } + + /** + * Get all elements of + * + * @return string[] + */ + public function getNamesList(){ + return $this->_get(1); + } + + /** + * Add a new element to + * + * @param string $value + * @return \Vitess\Proto\Throttlerdata\UpdateConfigurationResponse + */ + public function addNames( $value){ + return $this->_add(1, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vtgate/UpdateStreamRequest.php b/php/src/Vitess/Proto/Vtgate/UpdateStreamRequest.php new file mode 100644 index 0000000000..40951c7909 --- /dev/null +++ b/php/src/Vitess/Proto/Vtgate/UpdateStreamRequest.php @@ -0,0 +1,365 @@ +number = 1; + $f->name = "caller_id"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Vtrpc\CallerID'; + $descriptor->addField($f); + + // OPTIONAL STRING keyspace = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "keyspace"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL STRING shard = 3 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 3; + $f->name = "shard"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE key_range = 4 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 4; + $f->name = "key_range"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Topodata\KeyRange'; + $descriptor->addField($f); + + // OPTIONAL ENUM tablet_type = 5 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 5; + $f->name = "tablet_type"; + $f->type = \DrSlump\Protobuf::TYPE_ENUM; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Topodata\TabletType'; + $descriptor->addField($f); + + // OPTIONAL INT64 timestamp = 6 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 6; + $f->name = "timestamp"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE event = 7 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 7; + $f->name = "event"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\EventToken'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasCallerId(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearCallerId(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return \Vitess\Proto\Vtrpc\CallerID + */ + public function getCallerId(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param \Vitess\Proto\Vtrpc\CallerID $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setCallerId(\Vitess\Proto\Vtrpc\CallerID $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasKeyspace(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearKeyspace(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return string + */ + public function getKeyspace(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setKeyspace( $value){ + return $this->_set(2, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasShard(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearShard(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return string + */ + public function getShard(){ + return $this->_get(3); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setShard( $value){ + return $this->_set(3, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasKeyRange(){ + return $this->_has(4); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearKeyRange(){ + return $this->_clear(4); + } + + /** + * Get value + * + * @return \Vitess\Proto\Topodata\KeyRange + */ + public function getKeyRange(){ + return $this->_get(4); + } + + /** + * Set value + * + * @param \Vitess\Proto\Topodata\KeyRange $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setKeyRange(\Vitess\Proto\Topodata\KeyRange $value){ + return $this->_set(4, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasTabletType(){ + return $this->_has(5); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearTabletType(){ + return $this->_clear(5); + } + + /** + * Get value + * + * @return int - \Vitess\Proto\Topodata\TabletType + */ + public function getTabletType(){ + return $this->_get(5); + } + + /** + * Set value + * + * @param int - \Vitess\Proto\Topodata\TabletType $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setTabletType( $value){ + return $this->_set(5, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasTimestamp(){ + return $this->_has(6); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearTimestamp(){ + return $this->_clear(6); + } + + /** + * Get value + * + * @return int + */ + public function getTimestamp(){ + return $this->_get(6); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setTimestamp( $value){ + return $this->_set(6, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEvent(){ + return $this->_has(7); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function clearEvent(){ + return $this->_clear(7); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\EventToken + */ + public function getEvent(){ + return $this->_get(7); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\EventToken $value + * @return \Vitess\Proto\Vtgate\UpdateStreamRequest + */ + public function setEvent(\Vitess\Proto\Query\EventToken $value){ + return $this->_set(7, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vtgate/UpdateStreamResponse.php b/php/src/Vitess/Proto/Vtgate/UpdateStreamResponse.php new file mode 100644 index 0000000000..79f436afec --- /dev/null +++ b/php/src/Vitess/Proto/Vtgate/UpdateStreamResponse.php @@ -0,0 +1,122 @@ +number = 1; + $f->name = "event"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Query\StreamEvent'; + $descriptor->addField($f); + + // OPTIONAL INT64 resume_timestamp = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "resume_timestamp"; + $f->type = \DrSlump\Protobuf::TYPE_INT64; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasEvent(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamResponse + */ + public function clearEvent(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return \Vitess\Proto\Query\StreamEvent + */ + public function getEvent(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param \Vitess\Proto\Query\StreamEvent $value + * @return \Vitess\Proto\Vtgate\UpdateStreamResponse + */ + public function setEvent(\Vitess\Proto\Query\StreamEvent $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasResumeTimestamp(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vtgate\UpdateStreamResponse + */ + public function clearResumeTimestamp(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return int + */ + public function getResumeTimestamp(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Vtgate\UpdateStreamResponse + */ + public function setResumeTimestamp( $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vtgateservice/VitessClient.php b/php/src/Vitess/Proto/Vtgateservice/VitessClient.php index 564e52edc1..0d10df832b 100644 --- a/php/src/Vitess/Proto/Vtgateservice/VitessClient.php +++ b/php/src/Vitess/Proto/Vtgateservice/VitessClient.php @@ -105,5 +105,11 @@ namespace Vitess\Proto\Vtgateservice { public function GetSrvKeyspace(\Vitess\Proto\Vtgate\GetSrvKeyspaceRequest $argument, $metadata = array(), $options = array()) { return $this->_simpleRequest('/vtgateservice.Vitess/GetSrvKeyspace', $argument, '\Vitess\Proto\Vtgate\GetSrvKeyspaceResponse::deserialize', $metadata, $options); } + /** + * @param Vitess\Proto\Vtgate\UpdateStreamRequest $input + */ + public function UpdateStream($argument, $metadata = array(), $options = array()) { + return $this->_serverStreamRequest('/vtgateservice.Vitess/UpdateStream', $argument, '\Vitess\Proto\Vtgate\UpdateStreamResponse::deserialize', $metadata, $options); + } } } diff --git a/php/src/Vitess/Proto/Vttest/Keyspace.php b/php/src/Vitess/Proto/Vttest/Keyspace.php index cb01242de7..d6402497dd 100644 --- a/php/src/Vitess/Proto/Vttest/Keyspace.php +++ b/php/src/Vitess/Proto/Vttest/Keyspace.php @@ -21,6 +21,12 @@ namespace Vitess\Proto\Vttest { /** @var string */ public $served_from = null; + /** @var int */ + public $replica_count = null; + + /** @var int */ + public $rdonly_count = null; + /** @var \Closure[] */ protected static $__extensions = array(); @@ -70,6 +76,22 @@ namespace Vitess\Proto\Vttest { $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; $descriptor->addField($f); + // OPTIONAL INT32 replica_count = 6 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 6; + $f->name = "replica_count"; + $f->type = \DrSlump\Protobuf::TYPE_INT32; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL INT32 rdonly_count = 7 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 7; + $f->name = "rdonly_count"; + $f->type = \DrSlump\Protobuf::TYPE_INT32; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + foreach (self::$__extensions as $cb) { $descriptor->addField($cb(), true); } @@ -281,6 +303,80 @@ namespace Vitess\Proto\Vttest { public function setServedFrom( $value){ return $this->_set(5, $value); } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasReplicaCount(){ + return $this->_has(6); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vttest\Keyspace + */ + public function clearReplicaCount(){ + return $this->_clear(6); + } + + /** + * Get value + * + * @return int + */ + public function getReplicaCount(){ + return $this->_get(6); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Vttest\Keyspace + */ + public function setReplicaCount( $value){ + return $this->_set(6, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasRdonlyCount(){ + return $this->_has(7); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vttest\Keyspace + */ + public function clearRdonlyCount(){ + return $this->_clear(7); + } + + /** + * Get value + * + * @return int + */ + public function getRdonlyCount(){ + return $this->_get(7); + } + + /** + * Set value + * + * @param int $value + * @return \Vitess\Proto\Vttest\Keyspace + */ + public function setRdonlyCount( $value){ + return $this->_set(7, $value); + } } } diff --git a/php/src/Vitess/Proto/Vttest/VTTestTopology.php b/php/src/Vitess/Proto/Vttest/VTTestTopology.php index 7a1ef6a972..266038af4c 100644 --- a/php/src/Vitess/Proto/Vttest/VTTestTopology.php +++ b/php/src/Vitess/Proto/Vttest/VTTestTopology.php @@ -9,6 +9,9 @@ namespace Vitess\Proto\Vttest { /** @var \Vitess\Proto\Vttest\Keyspace[] */ public $keyspaces = array(); + /** @var string[] */ + public $cells = array(); + /** @var \Closure[] */ protected static $__extensions = array(); @@ -26,6 +29,14 @@ namespace Vitess\Proto\Vttest { $f->reference = '\Vitess\Proto\Vttest\Keyspace'; $descriptor->addField($f); + // REPEATED STRING cells = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "cells"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_REPEATED; + $descriptor->addField($f); + foreach (self::$__extensions as $cb) { $descriptor->addField($cb(), true); } @@ -89,6 +100,63 @@ namespace Vitess\Proto\Vttest { public function addKeyspaces(\Vitess\Proto\Vttest\Keyspace $value){ return $this->_add(1, $value); } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasCells(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vttest\VTTestTopology + */ + public function clearCells(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @param int $idx + * @return string + */ + public function getCells($idx = NULL){ + return $this->_get(2, $idx); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vttest\VTTestTopology + */ + public function setCells( $value, $idx = NULL){ + return $this->_set(2, $value, $idx); + } + + /** + * Get all elements of + * + * @return string[] + */ + public function getCellsList(){ + return $this->_get(2); + } + + /** + * Add a new element to + * + * @param string $value + * @return \Vitess\Proto\Vttest\VTTestTopology + */ + public function addCells( $value){ + return $this->_add(2, $value); + } } } diff --git a/php/src/Vitess/Proto/Vtworkerservice/VtworkerClient.php b/php/src/Vitess/Proto/Vtworkerservice/VtworkerClient.php index 073cf8ef96..4933426ba6 100644 --- a/php/src/Vitess/Proto/Vtworkerservice/VtworkerClient.php +++ b/php/src/Vitess/Proto/Vtworkerservice/VtworkerClient.php @@ -6,8 +6,8 @@ namespace Vitess\Proto\Vtworkerservice { class VtworkerClient extends \Grpc\BaseStub { - public function __construct($hostname, $opts, $channel = null) { - parent::__construct($hostname, $opts, $channel); + public function __construct($hostname, $opts) { + parent::__construct($hostname, $opts); } /** * @param Vitess\Proto\Vtworkerdata\ExecuteVtworkerCommandRequest $input diff --git a/proto/vtgate.proto b/proto/vtgate.proto index 17c4a18c21..c4a26f0d5e 100644 --- a/proto/vtgate.proto +++ b/proto/vtgate.proto @@ -632,3 +632,46 @@ message GetSrvKeyspaceResponse { // srv_keyspace is the topology object for the SrvKeyspace. topodata.SrvKeyspace srv_keyspace = 1; } + +// UpdateStreamRequest is the payload to UpdateStream. +message UpdateStreamRequest { + // caller_id identifies the caller. This is the effective caller ID, + // set by the application to further identify the caller. + vtrpc.CallerID caller_id = 1; + + // keyspace to target the query to. + string keyspace = 2; + + // shard to target the query to, for unsharded keyspaces. + string shard = 3; + + // KeyRange to target the query to, for sharded keyspaces. + topodata.KeyRange key_range = 4; + + // tablet_type is the type of tablets that this request is targeted to. + topodata.TabletType tablet_type = 5; + + // timestamp is the timestamp to start the stream from. It is + // unused is event is set, and we are only streaming from the shard + // described by event.shard. + int64 timestamp = 6; + + // event is the event to start the stream from. + // Note it is only used if we are streaming from exactly the same shard + // as this event was coming from. Otherwise we can't use this event, + // and will use the timestamp as a starting point. + query.EventToken event = 7; +} + +// UpdateStreamResponse is streamed by UpdateStream. +message UpdateStreamResponse { + // event is one event from the stream. + query.StreamEvent event = 1; + + // resume_timestamp is the timestamp to resume streaming from if the + // client is interrupted. If the Update Stream only goes to one + // shard, this is equal to event.timestamp. If the Update Stream + // goes to multiple shards and aggregates, this is the minimum value + // of the current timestamp for all shards. + int64 resume_timestamp = 2; +} diff --git a/proto/vtgateservice.proto b/proto/vtgateservice.proto index ed66c69cd2..7e1afa6c15 100644 --- a/proto/vtgateservice.proto +++ b/proto/vtgateservice.proto @@ -89,4 +89,8 @@ service Vitess { // using custom sharding. // API group: Topology rpc GetSrvKeyspace(vtgate.GetSrvKeyspaceRequest) returns (vtgate.GetSrvKeyspaceResponse) {}; + + // UpdateStream asks the server for a stream of StreamEvent objects. + // API group: Update Stream + rpc UpdateStream(vtgate.UpdateStreamRequest) returns (stream vtgate.UpdateStreamResponse) {}; } diff --git a/py/vtdb/dbexceptions.py b/py/vtdb/dbexceptions.py index b44003bf74..e2572d2531 100755 --- a/py/vtdb/dbexceptions.py +++ b/py/vtdb/dbexceptions.py @@ -72,6 +72,7 @@ class TimeoutError(OperationalError): class TxPoolFull(DatabaseError): pass + # TransientError is raised for an error that is expected to go away soon. These # errors should be retried. Examples: when a client exceedes allocated quota on # a server, or when there's a backlog of requests and new ones are temporarily @@ -84,3 +85,10 @@ class TransientError(DatabaseError): # ThrottledError is raised when client exceeds allocated quota on the server class ThrottledError(DatabaseError): pass + + +# QueryNotServed is raised when a pre-condition has failed. For instance, +# an update stream query cannot be served because there aren't enough +# binlogs on the server. +class QueryNotServed(DatabaseError): + pass diff --git a/py/vtdb/grpc_update_stream.py b/py/vtdb/grpc_update_stream.py deleted file mode 100644 index 9ca5d3346c..0000000000 --- a/py/vtdb/grpc_update_stream.py +++ /dev/null @@ -1,63 +0,0 @@ -"""gRPC update_stream.UpdateStreamConnection implementation. -""" -from urlparse import urlparse - -# Import main protobuf library first -# to work around import order issues. -import google.protobuf # pylint: disable=unused-import - -from grpc.beta import implementations -from grpc.framework.interfaces.face import face - -from vtproto import query_pb2 -from vtproto import queryservice_pb2 - -from vtdb import dbexceptions -from vtdb import update_stream - - -class GRPCUpdateStreamConnection(update_stream.UpdateStreamConnection): - """The gRPC implementation of UpdateStreamConnection. - - It is registered as 'grpc' protocol. - """ - - def __init__(self, addr, timeout): - self.addr = addr - self.timeout = timeout - self.stub = None - - def __str__(self): - return '' % self.addr - - def dial(self): - p = urlparse('http://' + self.addr) - channel = implementations.insecure_channel(p.hostname, p.port) - self.stub = queryservice_pb2.beta_create_Query_stub(channel) - - def close(self): - self.stub = None - - def is_closed(self): - return self.stub is None - - def stream_update(self, keyspace, shard, tablet_type, - position='', timestamp=0, - timeout=3600.0): - try: - target = query_pb2.Target(keyspace=keyspace, - shard=shard, - tablet_type=tablet_type) - req = query_pb2.UpdateStreamRequest(target=target, - position=position, - timestamp=timestamp) - - for response in self.stub.UpdateStream(req, timeout): - yield response.event - except face.AbortionError as e: - # FIXME(alainjobart) These exceptions don't print well, so raise - # one that will. The real fix is to define a set of exceptions - # for this library and raise that, but it's more work. - raise dbexceptions.OperationalError(e.details, e) - -update_stream.register_conn_class('grpc', GRPCUpdateStreamConnection) diff --git a/py/vtdb/grpc_vtgate_client.py b/py/vtdb/grpc_vtgate_client.py index 24c664fde8..dc2d8a6254 100644 --- a/py/vtdb/grpc_vtgate_client.py +++ b/py/vtdb/grpc_vtgate_client.py @@ -220,6 +220,33 @@ class GRPCVTGateConnection(vtgate_client.VTGateClient, except (face.AbortionError, vtgate_utils.VitessError) as e: raise _convert_exception(e, keyspace=name) + @vtgate_utils.exponential_backoff_retry((dbexceptions.TransientError)) + def update_stream( + self, keyspace_name, tablet_type, + timestamp=None, event=None, + shard=None, key_range=None, + effective_caller_id=None, + **kwargs): + + try: + request = self.update_stream_request( + keyspace_name, shard, key_range, tablet_type, + timestamp, event, effective_caller_id) + it = self.stub.UpdateStream(request, self.timeout) + except (face.AbortionError, vtgate_utils.VitessError) as e: + raise _convert_exception( + e, 'UpdateStream', + keyspace=keyspace_name, tablet_type=tablet_type) + + def row_generator(): + try: + for response in it: + yield (response.event, response.resume_timestamp) + except Exception as e: + raise _convert_exception(e) + + return row_generator() + def _convert_exception(exc, *args, **kwargs): """This parses the protocol exceptions to the api interface exceptions. @@ -259,6 +286,8 @@ def _convert_exception(exc, *args, **kwargs): return dbexceptions.TransientError(new_args) elif exc.code == interfaces.StatusCode.ALREADY_EXISTS: new_exc = _prune_integrity_error(msg, new_args) + elif exc.code == interfaces.StatusCode.FAILED_PRECONDITION: + return dbexceptions.QueryNotServed(msg, new_args) else: # Unhandled RPC application error new_exc = dbexceptions.DatabaseError(new_args + (msg,)) diff --git a/py/vtdb/proto3_encoding.py b/py/vtdb/proto3_encoding.py index abf352af76..bf7f248cef 100644 --- a/py/vtdb/proto3_encoding.py +++ b/py/vtdb/proto3_encoding.py @@ -142,6 +142,31 @@ def convert_bind_vars(bind_variables, request_bind_variables): convert_value(val, request_bind_variables[key], allow_lists=True) +def convert_stream_event_statement(statement): + """Converts encoded rows inside a StreamEvent.Statement to native types. + + Args: + statement: the StreamEvent.Statement object. + + Returns: + fields: array of names for the primary key columns. + rows: array of tuples for each primary key value. + """ + fields = [] + rows = [] + if statement.primary_key_fields: + convs = [] + for field in statement.primary_key_fields: + fields.append(field.name) + convs.append(conversions.get(field.type)) + + for r in statement.primary_key_values: + row = tuple(make_row(r, convs)) + rows.append(row) + + return fields, rows + + class Proto3Connection(object): """A base class for proto3-based python connectors. @@ -459,6 +484,46 @@ class Proto3Connection(object): rowsets.append(rowset) return rowsets + def update_stream_request(self, + keyspace_name, + shard, + key_range, + tablet_type, + timestamp, + event, + effective_caller_id): + """Builds the right vtgate_pb2 UpdateStreamRequest. + + Args: + keyspace_name: keyspace to apply the query to. + shard: shard to ask for. + key_range: keyrange.KeyRange object. + tablet_type: string tablet type. + timestamp: when to start the stream from. + event: alternate way to describe where to start the stream from. + effective_caller_id: optional vtgate_client.CallerID. + + Returns: + A vtgate_pb2.UpdateStreamRequest object. + """ + request = vtgate_pb2.UpdateStreamRequest(keyspace=keyspace_name, + tablet_type=tablet_type, + shard=shard) + if timestamp: + request.timestamp = timestamp + if event: + if event.timestamp: + request.event.timestamp = event.timestamp + if event.shard: + request.event.shard = event.shard + if event.position: + request.event.position = event.position + if key_range: + request.key_range.start = key_range.Start + request.key_range.end = key_range.End + self._add_caller_id(request, effective_caller_id) + return request + def stream_execute_request_and_name(self, sql, bind_variables, tablet_type, keyspace_name, shards, diff --git a/py/vtdb/update_stream.py b/py/vtdb/update_stream.py deleted file mode 100644 index bb1073846f..0000000000 --- a/py/vtdb/update_stream.py +++ /dev/null @@ -1,120 +0,0 @@ -# Copyright 2015, Google Inc. All rights reserved. -# Use of this source code is governed by a BSD-style license that can -# be found in the LICENSE file. -"""This module defines the update stream client interface. -""" - -from vtdb import proto3_encoding - -# mapping from protocol to python class. The protocol matches the string -# used by vttablet as a -binlog_player_protocol parameter. -update_stream_conn_classes = dict() - - -def register_conn_class(protocol, c): - """Used by implementations to register themselves. - - Args: - protocol: short string to document the protocol. - c: class to register. - """ - update_stream_conn_classes[protocol] = c - - -def connect(protocol, *pargs, **kargs): - """Return a dialed UpdateStreamConnection to an update stream server. - - Args: - protocol: The registered protocol to use. - *pargs: Passed to the registered protocol __init__ method. - **kargs: Passed to the registered protocol __init__ method. - - Returns: - A dialed UpdateStreamConnection. - - Raises: - ValueError: On bad protocol. - """ - if protocol not in update_stream_conn_classes: - raise ValueError('Unknown update stream protocol', protocol) - conn = update_stream_conn_classes[protocol](*pargs, **kargs) - conn.dial() - return conn - - -class UpdateStreamConnection(object): - """The interface for the update stream client implementations. - - All implementations must implement all these methods. If something - goes wrong with the connection, this object will be thrown out. - """ - - def __init__(self, addr, timeout): - """Initialize an update stream connection. - - Args: - addr: server address. Can be protocol dependent. - timeout: connection timeout (float, in seconds). - """ - pass - - def dial(self): - """Dial to the server. If successful, call close() to close the connection. - """ - pass - - def close(self): - """Close the connection. This object may be re-used again by calling dial(). - """ - pass - - def is_closed(self): - """Checks the connection status. - - Returns: - True if this connection is closed. - """ - pass - - def stream_update(self, keyspace, shard, tablet_type, - position='', timestamp=0, - timeout=3600.0): - """Generator method to stream the updates from a given replication point. - - Args: - keyspace: keyspace to target. - shard: shard to target. - tablet_type: tablet_type to target. - position: Starting position to stream from. - timestamp: Starting timestamp to stream from. - timeout: Should stop streaming after we reach this timeout. - - Returns: - This is a generator method that yields query_pb2.StreamEvent objects. - """ - pass - - -def convert_statement(statement): - """Converts encoded rows inside a StreamEvent.Statement to native types. - - Args: - statement: the StreamEvent.Statement object. - - Returns: - fields: array of names for the primary key columns. - rows: array of tuples for each primary key value. - """ - fields = [] - rows = [] - if statement.primary_key_fields: - conversions = [] - for field in statement.primary_key_fields: - fields.append(field.name) - conversions.append(proto3_encoding.conversions.get(field.type)) - - for r in statement.primary_key_values: - row = tuple(proto3_encoding.make_row(r, conversions)) - rows.append(row) - - return fields, rows diff --git a/py/vtdb/vtgate_client.py b/py/vtdb/vtgate_client.py index 19b7e1c76e..bd81e031e5 100644 --- a/py/vtdb/vtgate_client.py +++ b/py/vtdb/vtgate_client.py @@ -345,3 +345,37 @@ class VTGateClient(object): TBD """ raise NotImplementedError('Child class needs to implement this') + + def update_stream(self, + keyspace_name, tablet_type, + timestamp=None, event=None, + shard=None, key_range=None, + effective_caller_id=None, + **kwargs): + """Asks for an update stream. + + Args: + keyspace_name: the keyspace to get updates from. + tablet_type: the (proto3) version of the tablet type. + timestamp: when to start the stream from. Unused if event is set, + and event.shard matches the only shard we stream from. + event: query_pb2.EventToken to start streaming from. Used only if its + shard field matches the single shard we're going to stream from. + shard: the shard name to listen for. + Incompatible with key_range. + key_range: the key range to listen for. + Incompatible with shard. + effective_caller_id: CallerID object. + **kwargs: implementation specific parameters. + + Returns: + A row generator that returns tuples (event, resume timestamp). + + Raises: + dbexceptions.TimeoutError: for connection timeout. + dbexceptions.TransientError: the server is overloaded, and this query + is asked to back off. + dbexceptions.DatabaseError: generic database error. + dbexceptions.FatalError: this query should not be retried. + """ + raise NotImplementedError('Child class needs to implement this') diff --git a/py/vtproto/vtgate_pb2.py b/py/vtproto/vtgate_pb2.py index 93073b11bf..0feacb1ad6 100644 --- a/py/vtproto/vtgate_pb2.py +++ b/py/vtproto/vtgate_pb2.py @@ -22,7 +22,7 @@ DESCRIPTOR = _descriptor.FileDescriptor( name='vtgate.proto', package='vtgate', syntax='proto3', - serialized_pb=_b('\n\x0cvtgate.proto\x12\x06vtgate\x1a\x0bquery.proto\x1a\x0etopodata.proto\x1a\x0bvtrpc.proto\"\x9e\x01\n\x07Session\x12\x16\n\x0ein_transaction\x18\x01 \x01(\x08\x12\x34\n\x0eshard_sessions\x18\x02 \x03(\x0b\x32\x1c.vtgate.Session.ShardSession\x1a\x45\n\x0cShardSession\x12\x1d\n\x06target\x18\x01 \x01(\x0b\x32\r.query.Target\x12\x16\n\x0etransaction_id\x18\x02 \x01(\x03\"\xd1\x01\n\x0e\x45xecuteRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x05 \x01(\x08\x12\x10\n\x08keyspace\x18\x06 \x01(\t\"w\n\x0f\x45xecuteResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\xe7\x01\n\x14\x45xecuteShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x0e\n\x06shards\x18\x05 \x03(\t\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"}\n\x15\x45xecuteShardsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\xf2\x01\n\x19\x45xecuteKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x05 \x03(\x0c\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"\x82\x01\n\x1a\x45xecuteKeyspaceIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\x82\x02\n\x17\x45xecuteKeyRangesRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12&\n\nkey_ranges\x18\x05 \x03(\x0b\x32\x12.topodata.KeyRange\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"\x80\x01\n\x18\x45xecuteKeyRangesResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\x88\x03\n\x17\x45xecuteEntityIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x1a\n\x12\x65ntity_column_name\x18\x05 \x01(\t\x12\x45\n\x13\x65ntity_keyspace_ids\x18\x06 \x03(\x0b\x32(.vtgate.ExecuteEntityIdsRequest.EntityId\x12)\n\x0btablet_type\x18\x07 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x08 \x01(\x08\x1aI\n\x08\x45ntityId\x12\x19\n\x04type\x18\x01 \x01(\x0e\x32\x0b.query.Type\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x13\n\x0bkeyspace_id\x18\x03 \x01(\x0c\"\x80\x01\n\x18\x45xecuteEntityIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"U\n\x0f\x42oundShardQuery\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\x0e\n\x06shards\x18\x03 \x03(\t\"\xce\x01\n\x19\x45xecuteBatchShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12(\n\x07queries\x18\x03 \x03(\x0b\x32\x17.vtgate.BoundShardQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x16\n\x0e\x61s_transaction\x18\x05 \x01(\x08\"\x83\x01\n\x1a\x45xecuteBatchShardsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12#\n\x07results\x18\x03 \x03(\x0b\x32\x12.query.QueryResult\"`\n\x14\x42oundKeyspaceIdQuery\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x03 \x03(\x0c\"\xd8\x01\n\x1e\x45xecuteBatchKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12-\n\x07queries\x18\x03 \x03(\x0b\x32\x1c.vtgate.BoundKeyspaceIdQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x16\n\x0e\x61s_transaction\x18\x05 \x01(\x08\"\x88\x01\n\x1f\x45xecuteBatchKeyspaceIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12#\n\x07results\x18\x03 \x03(\x0b\x32\x12.query.QueryResult\"\x99\x01\n\x14StreamExecuteRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12\x10\n\x08keyspace\x18\x04 \x01(\t\";\n\x15StreamExecuteResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xaf\x01\n\x1aStreamExecuteShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12\x0e\n\x06shards\x18\x04 \x03(\t\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"A\n\x1bStreamExecuteShardsResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xba\x01\n\x1fStreamExecuteKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x04 \x03(\x0c\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"F\n StreamExecuteKeyspaceIdsResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xca\x01\n\x1dStreamExecuteKeyRangesRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12&\n\nkey_ranges\x18\x04 \x03(\x0b\x32\x12.topodata.KeyRange\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"D\n\x1eStreamExecuteKeyRangesResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"2\n\x0c\x42\x65ginRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\"1\n\rBeginResponse\x12 \n\x07session\x18\x01 \x01(\x0b\x32\x0f.vtgate.Session\"U\n\rCommitRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\"\x10\n\x0e\x43ommitResponse\"W\n\x0fRollbackRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\"\x12\n\x10RollbackResponse\"\x8a\x02\n\x11SplitQueryRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x14\n\x0csplit_column\x18\x04 \x03(\t\x12\x13\n\x0bsplit_count\x18\x05 \x01(\x03\x12\x1f\n\x17num_rows_per_query_part\x18\x06 \x01(\x03\x12\x35\n\talgorithm\x18\x07 \x01(\x0e\x32\".query.SplitQueryRequest.Algorithm\x12\x1a\n\x12use_split_query_v2\x18\x08 \x01(\x08\"\xf2\x02\n\x12SplitQueryResponse\x12/\n\x06splits\x18\x01 \x03(\x0b\x32\x1f.vtgate.SplitQueryResponse.Part\x1aH\n\x0cKeyRangePart\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12&\n\nkey_ranges\x18\x02 \x03(\x0b\x32\x12.topodata.KeyRange\x1a-\n\tShardPart\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\x0e\n\x06shards\x18\x02 \x03(\t\x1a\xb1\x01\n\x04Part\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12?\n\x0ekey_range_part\x18\x02 \x01(\x0b\x32\'.vtgate.SplitQueryResponse.KeyRangePart\x12\x38\n\nshard_part\x18\x03 \x01(\x0b\x32$.vtgate.SplitQueryResponse.ShardPart\x12\x0c\n\x04size\x18\x04 \x01(\x03\")\n\x15GetSrvKeyspaceRequest\x12\x10\n\x08keyspace\x18\x01 \x01(\t\"E\n\x16GetSrvKeyspaceResponse\x12+\n\x0csrv_keyspace\x18\x01 \x01(\x0b\x32\x15.topodata.SrvKeyspaceB\x1a\n\x18\x63om.youtube.vitess.protob\x06proto3') + serialized_pb=_b('\n\x0cvtgate.proto\x12\x06vtgate\x1a\x0bquery.proto\x1a\x0etopodata.proto\x1a\x0bvtrpc.proto\"\x9e\x01\n\x07Session\x12\x16\n\x0ein_transaction\x18\x01 \x01(\x08\x12\x34\n\x0eshard_sessions\x18\x02 \x03(\x0b\x32\x1c.vtgate.Session.ShardSession\x1a\x45\n\x0cShardSession\x12\x1d\n\x06target\x18\x01 \x01(\x0b\x32\r.query.Target\x12\x16\n\x0etransaction_id\x18\x02 \x01(\x03\"\xd1\x01\n\x0e\x45xecuteRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x05 \x01(\x08\x12\x10\n\x08keyspace\x18\x06 \x01(\t\"w\n\x0f\x45xecuteResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\xe7\x01\n\x14\x45xecuteShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x0e\n\x06shards\x18\x05 \x03(\t\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"}\n\x15\x45xecuteShardsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\xf2\x01\n\x19\x45xecuteKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x05 \x03(\x0c\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"\x82\x01\n\x1a\x45xecuteKeyspaceIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\x82\x02\n\x17\x45xecuteKeyRangesRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12&\n\nkey_ranges\x18\x05 \x03(\x0b\x32\x12.topodata.KeyRange\x12)\n\x0btablet_type\x18\x06 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x07 \x01(\x08\"\x80\x01\n\x18\x45xecuteKeyRangesResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"\x88\x03\n\x17\x45xecuteEntityIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x04 \x01(\t\x12\x1a\n\x12\x65ntity_column_name\x18\x05 \x01(\t\x12\x45\n\x13\x65ntity_keyspace_ids\x18\x06 \x03(\x0b\x32(.vtgate.ExecuteEntityIdsRequest.EntityId\x12)\n\x0btablet_type\x18\x07 \x01(\x0e\x32\x14.topodata.TabletType\x12\x1a\n\x12not_in_transaction\x18\x08 \x01(\x08\x1aI\n\x08\x45ntityId\x12\x19\n\x04type\x18\x01 \x01(\x0e\x32\x0b.query.Type\x12\r\n\x05value\x18\x02 \x01(\x0c\x12\x13\n\x0bkeyspace_id\x18\x03 \x01(\x0c\"\x80\x01\n\x18\x45xecuteEntityIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12\"\n\x06result\x18\x03 \x01(\x0b\x32\x12.query.QueryResult\"U\n\x0f\x42oundShardQuery\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\x0e\n\x06shards\x18\x03 \x03(\t\"\xce\x01\n\x19\x45xecuteBatchShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12(\n\x07queries\x18\x03 \x03(\x0b\x32\x17.vtgate.BoundShardQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x16\n\x0e\x61s_transaction\x18\x05 \x01(\x08\"\x83\x01\n\x1a\x45xecuteBatchShardsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12#\n\x07results\x18\x03 \x03(\x0b\x32\x12.query.QueryResult\"`\n\x14\x42oundKeyspaceIdQuery\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x03 \x03(\x0c\"\xd8\x01\n\x1e\x45xecuteBatchKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12-\n\x07queries\x18\x03 \x03(\x0b\x32\x1c.vtgate.BoundKeyspaceIdQuery\x12)\n\x0btablet_type\x18\x04 \x01(\x0e\x32\x14.topodata.TabletType\x12\x16\n\x0e\x61s_transaction\x18\x05 \x01(\x08\"\x88\x01\n\x1f\x45xecuteBatchKeyspaceIdsResponse\x12\x1e\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x0f.vtrpc.RPCError\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\x12#\n\x07results\x18\x03 \x03(\x0b\x32\x12.query.QueryResult\"\x99\x01\n\x14StreamExecuteRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12\x10\n\x08keyspace\x18\x04 \x01(\t\";\n\x15StreamExecuteResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xaf\x01\n\x1aStreamExecuteShardsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12\x0e\n\x06shards\x18\x04 \x03(\t\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"A\n\x1bStreamExecuteShardsResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xba\x01\n\x1fStreamExecuteKeyspaceIdsRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12\x14\n\x0ckeyspace_ids\x18\x04 \x03(\x0c\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"F\n StreamExecuteKeyspaceIdsResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"\xca\x01\n\x1dStreamExecuteKeyRangesRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x05query\x18\x02 \x01(\x0b\x32\x11.query.BoundQuery\x12\x10\n\x08keyspace\x18\x03 \x01(\t\x12&\n\nkey_ranges\x18\x04 \x03(\x0b\x32\x12.topodata.KeyRange\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\"D\n\x1eStreamExecuteKeyRangesResponse\x12\"\n\x06result\x18\x01 \x01(\x0b\x32\x12.query.QueryResult\"2\n\x0c\x42\x65ginRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\"1\n\rBeginResponse\x12 \n\x07session\x18\x01 \x01(\x0b\x32\x0f.vtgate.Session\"U\n\rCommitRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\"\x10\n\x0e\x43ommitResponse\"W\n\x0fRollbackRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12 \n\x07session\x18\x02 \x01(\x0b\x32\x0f.vtgate.Session\"\x12\n\x10RollbackResponse\"\x8a\x02\n\x11SplitQueryRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12 \n\x05query\x18\x03 \x01(\x0b\x32\x11.query.BoundQuery\x12\x14\n\x0csplit_column\x18\x04 \x03(\t\x12\x13\n\x0bsplit_count\x18\x05 \x01(\x03\x12\x1f\n\x17num_rows_per_query_part\x18\x06 \x01(\x03\x12\x35\n\talgorithm\x18\x07 \x01(\x0e\x32\".query.SplitQueryRequest.Algorithm\x12\x1a\n\x12use_split_query_v2\x18\x08 \x01(\x08\"\xf2\x02\n\x12SplitQueryResponse\x12/\n\x06splits\x18\x01 \x03(\x0b\x32\x1f.vtgate.SplitQueryResponse.Part\x1aH\n\x0cKeyRangePart\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12&\n\nkey_ranges\x18\x02 \x03(\x0b\x32\x12.topodata.KeyRange\x1a-\n\tShardPart\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\x0e\n\x06shards\x18\x02 \x03(\t\x1a\xb1\x01\n\x04Part\x12 \n\x05query\x18\x01 \x01(\x0b\x32\x11.query.BoundQuery\x12?\n\x0ekey_range_part\x18\x02 \x01(\x0b\x32\'.vtgate.SplitQueryResponse.KeyRangePart\x12\x38\n\nshard_part\x18\x03 \x01(\x0b\x32$.vtgate.SplitQueryResponse.ShardPart\x12\x0c\n\x04size\x18\x04 \x01(\x03\")\n\x15GetSrvKeyspaceRequest\x12\x10\n\x08keyspace\x18\x01 \x01(\t\"E\n\x16GetSrvKeyspaceResponse\x12+\n\x0csrv_keyspace\x18\x01 \x01(\x0b\x32\x15.topodata.SrvKeyspace\"\xe1\x01\n\x13UpdateStreamRequest\x12\"\n\tcaller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x10\n\x08keyspace\x18\x02 \x01(\t\x12\r\n\x05shard\x18\x03 \x01(\t\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12)\n\x0btablet_type\x18\x05 \x01(\x0e\x32\x14.topodata.TabletType\x12\x11\n\ttimestamp\x18\x06 \x01(\x03\x12 \n\x05\x65vent\x18\x07 \x01(\x0b\x32\x11.query.EventToken\"S\n\x14UpdateStreamResponse\x12!\n\x05\x65vent\x18\x01 \x01(\x0b\x32\x12.query.StreamEvent\x12\x18\n\x10resume_timestamp\x18\x02 \x01(\x03\x42\x1a\n\x18\x63om.youtube.vitess.protob\x06proto3') , dependencies=[query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,vtrpc__pb2.DESCRIPTOR,]) _sym_db.RegisterFileDescriptor(DESCRIPTOR) @@ -1873,6 +1873,117 @@ _GETSRVKEYSPACERESPONSE = _descriptor.Descriptor( serialized_end=5172, ) + +_UPDATESTREAMREQUEST = _descriptor.Descriptor( + name='UpdateStreamRequest', + full_name='vtgate.UpdateStreamRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='caller_id', full_name='vtgate.UpdateStreamRequest.caller_id', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='keyspace', full_name='vtgate.UpdateStreamRequest.keyspace', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='shard', full_name='vtgate.UpdateStreamRequest.shard', index=2, + number=3, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='key_range', full_name='vtgate.UpdateStreamRequest.key_range', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='tablet_type', full_name='vtgate.UpdateStreamRequest.tablet_type', index=4, + number=5, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='timestamp', full_name='vtgate.UpdateStreamRequest.timestamp', index=5, + number=6, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='event', full_name='vtgate.UpdateStreamRequest.event', index=6, + number=7, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5175, + serialized_end=5400, +) + + +_UPDATESTREAMRESPONSE = _descriptor.Descriptor( + name='UpdateStreamResponse', + full_name='vtgate.UpdateStreamResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='event', full_name='vtgate.UpdateStreamResponse.event', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='resume_timestamp', full_name='vtgate.UpdateStreamResponse.resume_timestamp', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=5402, + serialized_end=5485, +) + _SESSION_SHARDSESSION.fields_by_name['target'].message_type = query__pb2._TARGET _SESSION_SHARDSESSION.containing_type = _SESSION _SESSION.fields_by_name['shard_sessions'].message_type = _SESSION_SHARDSESSION @@ -1966,6 +2077,11 @@ _SPLITQUERYRESPONSE_PART.fields_by_name['shard_part'].message_type = _SPLITQUERY _SPLITQUERYRESPONSE_PART.containing_type = _SPLITQUERYRESPONSE _SPLITQUERYRESPONSE.fields_by_name['splits'].message_type = _SPLITQUERYRESPONSE_PART _GETSRVKEYSPACERESPONSE.fields_by_name['srv_keyspace'].message_type = topodata__pb2._SRVKEYSPACE +_UPDATESTREAMREQUEST.fields_by_name['caller_id'].message_type = vtrpc__pb2._CALLERID +_UPDATESTREAMREQUEST.fields_by_name['key_range'].message_type = topodata__pb2._KEYRANGE +_UPDATESTREAMREQUEST.fields_by_name['tablet_type'].enum_type = topodata__pb2._TABLETTYPE +_UPDATESTREAMREQUEST.fields_by_name['event'].message_type = query__pb2._EVENTTOKEN +_UPDATESTREAMRESPONSE.fields_by_name['event'].message_type = query__pb2._STREAMEVENT DESCRIPTOR.message_types_by_name['Session'] = _SESSION DESCRIPTOR.message_types_by_name['ExecuteRequest'] = _EXECUTEREQUEST DESCRIPTOR.message_types_by_name['ExecuteResponse'] = _EXECUTERESPONSE @@ -2001,6 +2117,8 @@ DESCRIPTOR.message_types_by_name['SplitQueryRequest'] = _SPLITQUERYREQUEST DESCRIPTOR.message_types_by_name['SplitQueryResponse'] = _SPLITQUERYRESPONSE DESCRIPTOR.message_types_by_name['GetSrvKeyspaceRequest'] = _GETSRVKEYSPACEREQUEST DESCRIPTOR.message_types_by_name['GetSrvKeyspaceResponse'] = _GETSRVKEYSPACERESPONSE +DESCRIPTOR.message_types_by_name['UpdateStreamRequest'] = _UPDATESTREAMREQUEST +DESCRIPTOR.message_types_by_name['UpdateStreamResponse'] = _UPDATESTREAMRESPONSE Session = _reflection.GeneratedProtocolMessageType('Session', (_message.Message,), dict( @@ -2287,6 +2405,20 @@ GetSrvKeyspaceResponse = _reflection.GeneratedProtocolMessageType('GetSrvKeyspac )) _sym_db.RegisterMessage(GetSrvKeyspaceResponse) +UpdateStreamRequest = _reflection.GeneratedProtocolMessageType('UpdateStreamRequest', (_message.Message,), dict( + DESCRIPTOR = _UPDATESTREAMREQUEST, + __module__ = 'vtgate_pb2' + # @@protoc_insertion_point(class_scope:vtgate.UpdateStreamRequest) + )) +_sym_db.RegisterMessage(UpdateStreamRequest) + +UpdateStreamResponse = _reflection.GeneratedProtocolMessageType('UpdateStreamResponse', (_message.Message,), dict( + DESCRIPTOR = _UPDATESTREAMRESPONSE, + __module__ = 'vtgate_pb2' + # @@protoc_insertion_point(class_scope:vtgate.UpdateStreamResponse) + )) +_sym_db.RegisterMessage(UpdateStreamResponse) + DESCRIPTOR.has_options = True DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('\n\030com.youtube.vitess.proto')) diff --git a/py/vtproto/vtgateservice_pb2.py b/py/vtproto/vtgateservice_pb2.py index 9f8a4a780c..7c67af2758 100644 --- a/py/vtproto/vtgateservice_pb2.py +++ b/py/vtproto/vtgateservice_pb2.py @@ -20,7 +20,7 @@ DESCRIPTOR = _descriptor.FileDescriptor( name='vtgateservice.proto', package='vtgateservice', syntax='proto3', - serialized_pb=_b('\n\x13vtgateservice.proto\x12\rvtgateservice\x1a\x0cvtgate.proto2\xd8\n\n\x06Vitess\x12<\n\x07\x45xecute\x12\x16.vtgate.ExecuteRequest\x1a\x17.vtgate.ExecuteResponse\"\x00\x12N\n\rExecuteShards\x12\x1c.vtgate.ExecuteShardsRequest\x1a\x1d.vtgate.ExecuteShardsResponse\"\x00\x12]\n\x12\x45xecuteKeyspaceIds\x12!.vtgate.ExecuteKeyspaceIdsRequest\x1a\".vtgate.ExecuteKeyspaceIdsResponse\"\x00\x12W\n\x10\x45xecuteKeyRanges\x12\x1f.vtgate.ExecuteKeyRangesRequest\x1a .vtgate.ExecuteKeyRangesResponse\"\x00\x12W\n\x10\x45xecuteEntityIds\x12\x1f.vtgate.ExecuteEntityIdsRequest\x1a .vtgate.ExecuteEntityIdsResponse\"\x00\x12]\n\x12\x45xecuteBatchShards\x12!.vtgate.ExecuteBatchShardsRequest\x1a\".vtgate.ExecuteBatchShardsResponse\"\x00\x12l\n\x17\x45xecuteBatchKeyspaceIds\x12&.vtgate.ExecuteBatchKeyspaceIdsRequest\x1a\'.vtgate.ExecuteBatchKeyspaceIdsResponse\"\x00\x12P\n\rStreamExecute\x12\x1c.vtgate.StreamExecuteRequest\x1a\x1d.vtgate.StreamExecuteResponse\"\x00\x30\x01\x12\x62\n\x13StreamExecuteShards\x12\".vtgate.StreamExecuteShardsRequest\x1a#.vtgate.StreamExecuteShardsResponse\"\x00\x30\x01\x12q\n\x18StreamExecuteKeyspaceIds\x12\'.vtgate.StreamExecuteKeyspaceIdsRequest\x1a(.vtgate.StreamExecuteKeyspaceIdsResponse\"\x00\x30\x01\x12k\n\x16StreamExecuteKeyRanges\x12%.vtgate.StreamExecuteKeyRangesRequest\x1a&.vtgate.StreamExecuteKeyRangesResponse\"\x00\x30\x01\x12\x36\n\x05\x42\x65gin\x12\x14.vtgate.BeginRequest\x1a\x15.vtgate.BeginResponse\"\x00\x12\x39\n\x06\x43ommit\x12\x15.vtgate.CommitRequest\x1a\x16.vtgate.CommitResponse\"\x00\x12?\n\x08Rollback\x12\x17.vtgate.RollbackRequest\x1a\x18.vtgate.RollbackResponse\"\x00\x12\x45\n\nSplitQuery\x12\x19.vtgate.SplitQueryRequest\x1a\x1a.vtgate.SplitQueryResponse\"\x00\x12Q\n\x0eGetSrvKeyspace\x12\x1d.vtgate.GetSrvKeyspaceRequest\x1a\x1e.vtgate.GetSrvKeyspaceResponse\"\x00\x42\x1f\n\x1d\x63om.youtube.vitess.proto.grpcb\x06proto3') + serialized_pb=_b('\n\x13vtgateservice.proto\x12\rvtgateservice\x1a\x0cvtgate.proto2\xa7\x0b\n\x06Vitess\x12<\n\x07\x45xecute\x12\x16.vtgate.ExecuteRequest\x1a\x17.vtgate.ExecuteResponse\"\x00\x12N\n\rExecuteShards\x12\x1c.vtgate.ExecuteShardsRequest\x1a\x1d.vtgate.ExecuteShardsResponse\"\x00\x12]\n\x12\x45xecuteKeyspaceIds\x12!.vtgate.ExecuteKeyspaceIdsRequest\x1a\".vtgate.ExecuteKeyspaceIdsResponse\"\x00\x12W\n\x10\x45xecuteKeyRanges\x12\x1f.vtgate.ExecuteKeyRangesRequest\x1a .vtgate.ExecuteKeyRangesResponse\"\x00\x12W\n\x10\x45xecuteEntityIds\x12\x1f.vtgate.ExecuteEntityIdsRequest\x1a .vtgate.ExecuteEntityIdsResponse\"\x00\x12]\n\x12\x45xecuteBatchShards\x12!.vtgate.ExecuteBatchShardsRequest\x1a\".vtgate.ExecuteBatchShardsResponse\"\x00\x12l\n\x17\x45xecuteBatchKeyspaceIds\x12&.vtgate.ExecuteBatchKeyspaceIdsRequest\x1a\'.vtgate.ExecuteBatchKeyspaceIdsResponse\"\x00\x12P\n\rStreamExecute\x12\x1c.vtgate.StreamExecuteRequest\x1a\x1d.vtgate.StreamExecuteResponse\"\x00\x30\x01\x12\x62\n\x13StreamExecuteShards\x12\".vtgate.StreamExecuteShardsRequest\x1a#.vtgate.StreamExecuteShardsResponse\"\x00\x30\x01\x12q\n\x18StreamExecuteKeyspaceIds\x12\'.vtgate.StreamExecuteKeyspaceIdsRequest\x1a(.vtgate.StreamExecuteKeyspaceIdsResponse\"\x00\x30\x01\x12k\n\x16StreamExecuteKeyRanges\x12%.vtgate.StreamExecuteKeyRangesRequest\x1a&.vtgate.StreamExecuteKeyRangesResponse\"\x00\x30\x01\x12\x36\n\x05\x42\x65gin\x12\x14.vtgate.BeginRequest\x1a\x15.vtgate.BeginResponse\"\x00\x12\x39\n\x06\x43ommit\x12\x15.vtgate.CommitRequest\x1a\x16.vtgate.CommitResponse\"\x00\x12?\n\x08Rollback\x12\x17.vtgate.RollbackRequest\x1a\x18.vtgate.RollbackResponse\"\x00\x12\x45\n\nSplitQuery\x12\x19.vtgate.SplitQueryRequest\x1a\x1a.vtgate.SplitQueryResponse\"\x00\x12Q\n\x0eGetSrvKeyspace\x12\x1d.vtgate.GetSrvKeyspaceRequest\x1a\x1e.vtgate.GetSrvKeyspaceResponse\"\x00\x12M\n\x0cUpdateStream\x12\x1b.vtgate.UpdateStreamRequest\x1a\x1c.vtgate.UpdateStreamResponse\"\x00\x30\x01\x42\x1f\n\x1d\x63om.youtube.vitess.proto.grpcb\x06proto3') , dependencies=[vtgate__pb2.DESCRIPTOR,]) _sym_db.RegisterFileDescriptor(DESCRIPTOR) @@ -87,6 +87,9 @@ class BetaVitessServicer(object): @abc.abstractmethod def GetSrvKeyspace(self, request, context): raise NotImplementedError() + @abc.abstractmethod + def UpdateStream(self, request, context): + raise NotImplementedError() class BetaVitessStub(object): """The interface to which stubs will conform.""" @@ -151,6 +154,9 @@ class BetaVitessStub(object): def GetSrvKeyspace(self, request, timeout): raise NotImplementedError() GetSrvKeyspace.future = None + @abc.abstractmethod + def UpdateStream(self, request, timeout): + raise NotImplementedError() def beta_create_Vitess_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None): import vtgate_pb2 @@ -185,6 +191,8 @@ def beta_create_Vitess_server(servicer, pool=None, pool_size=None, default_timeo import vtgate_pb2 import vtgate_pb2 import vtgate_pb2 + import vtgate_pb2 + import vtgate_pb2 request_deserializers = { ('vtgateservice.Vitess', 'Begin'): vtgate_pb2.BeginRequest.FromString, ('vtgateservice.Vitess', 'Commit'): vtgate_pb2.CommitRequest.FromString, @@ -202,6 +210,7 @@ def beta_create_Vitess_server(servicer, pool=None, pool_size=None, default_timeo ('vtgateservice.Vitess', 'StreamExecuteKeyRanges'): vtgate_pb2.StreamExecuteKeyRangesRequest.FromString, ('vtgateservice.Vitess', 'StreamExecuteKeyspaceIds'): vtgate_pb2.StreamExecuteKeyspaceIdsRequest.FromString, ('vtgateservice.Vitess', 'StreamExecuteShards'): vtgate_pb2.StreamExecuteShardsRequest.FromString, + ('vtgateservice.Vitess', 'UpdateStream'): vtgate_pb2.UpdateStreamRequest.FromString, } response_serializers = { ('vtgateservice.Vitess', 'Begin'): vtgate_pb2.BeginResponse.SerializeToString, @@ -220,6 +229,7 @@ def beta_create_Vitess_server(servicer, pool=None, pool_size=None, default_timeo ('vtgateservice.Vitess', 'StreamExecuteKeyRanges'): vtgate_pb2.StreamExecuteKeyRangesResponse.SerializeToString, ('vtgateservice.Vitess', 'StreamExecuteKeyspaceIds'): vtgate_pb2.StreamExecuteKeyspaceIdsResponse.SerializeToString, ('vtgateservice.Vitess', 'StreamExecuteShards'): vtgate_pb2.StreamExecuteShardsResponse.SerializeToString, + ('vtgateservice.Vitess', 'UpdateStream'): vtgate_pb2.UpdateStreamResponse.SerializeToString, } method_implementations = { ('vtgateservice.Vitess', 'Begin'): face_utilities.unary_unary_inline(servicer.Begin), @@ -238,6 +248,7 @@ def beta_create_Vitess_server(servicer, pool=None, pool_size=None, default_timeo ('vtgateservice.Vitess', 'StreamExecuteKeyRanges'): face_utilities.unary_stream_inline(servicer.StreamExecuteKeyRanges), ('vtgateservice.Vitess', 'StreamExecuteKeyspaceIds'): face_utilities.unary_stream_inline(servicer.StreamExecuteKeyspaceIds), ('vtgateservice.Vitess', 'StreamExecuteShards'): face_utilities.unary_stream_inline(servicer.StreamExecuteShards), + ('vtgateservice.Vitess', 'UpdateStream'): face_utilities.unary_stream_inline(servicer.UpdateStream), } server_options = beta_implementations.server_options(request_deserializers=request_deserializers, response_serializers=response_serializers, thread_pool=pool, thread_pool_size=pool_size, default_timeout=default_timeout, maximum_timeout=maximum_timeout) return beta_implementations.server(method_implementations, options=server_options) @@ -275,6 +286,8 @@ def beta_create_Vitess_stub(channel, host=None, metadata_transformer=None, pool= import vtgate_pb2 import vtgate_pb2 import vtgate_pb2 + import vtgate_pb2 + import vtgate_pb2 request_serializers = { ('vtgateservice.Vitess', 'Begin'): vtgate_pb2.BeginRequest.SerializeToString, ('vtgateservice.Vitess', 'Commit'): vtgate_pb2.CommitRequest.SerializeToString, @@ -292,6 +305,7 @@ def beta_create_Vitess_stub(channel, host=None, metadata_transformer=None, pool= ('vtgateservice.Vitess', 'StreamExecuteKeyRanges'): vtgate_pb2.StreamExecuteKeyRangesRequest.SerializeToString, ('vtgateservice.Vitess', 'StreamExecuteKeyspaceIds'): vtgate_pb2.StreamExecuteKeyspaceIdsRequest.SerializeToString, ('vtgateservice.Vitess', 'StreamExecuteShards'): vtgate_pb2.StreamExecuteShardsRequest.SerializeToString, + ('vtgateservice.Vitess', 'UpdateStream'): vtgate_pb2.UpdateStreamRequest.SerializeToString, } response_deserializers = { ('vtgateservice.Vitess', 'Begin'): vtgate_pb2.BeginResponse.FromString, @@ -310,6 +324,7 @@ def beta_create_Vitess_stub(channel, host=None, metadata_transformer=None, pool= ('vtgateservice.Vitess', 'StreamExecuteKeyRanges'): vtgate_pb2.StreamExecuteKeyRangesResponse.FromString, ('vtgateservice.Vitess', 'StreamExecuteKeyspaceIds'): vtgate_pb2.StreamExecuteKeyspaceIdsResponse.FromString, ('vtgateservice.Vitess', 'StreamExecuteShards'): vtgate_pb2.StreamExecuteShardsResponse.FromString, + ('vtgateservice.Vitess', 'UpdateStream'): vtgate_pb2.UpdateStreamResponse.FromString, } cardinalities = { 'Begin': cardinality.Cardinality.UNARY_UNARY, @@ -328,6 +343,7 @@ def beta_create_Vitess_stub(channel, host=None, metadata_transformer=None, pool= 'StreamExecuteKeyRanges': cardinality.Cardinality.UNARY_STREAM, 'StreamExecuteKeyspaceIds': cardinality.Cardinality.UNARY_STREAM, 'StreamExecuteShards': cardinality.Cardinality.UNARY_STREAM, + 'UpdateStream': cardinality.Cardinality.UNARY_STREAM, } stub_options = beta_implementations.stub_options(host=host, metadata_transformer=metadata_transformer, request_serializers=request_serializers, response_deserializers=response_deserializers, thread_pool=pool, thread_pool_size=pool_size) return beta_implementations.dynamic_stub(channel, 'vtgateservice.Vitess', cardinalities, options=stub_options) diff --git a/test/binlog.py b/test/binlog.py index a26ae7544a..19aba919e4 100755 --- a/test/binlog.py +++ b/test/binlog.py @@ -9,12 +9,11 @@ # It sets up filtered replication between two shards and checks how data flows # through binlog streamer. +import base64 import logging import unittest from vtdb import keyrange_constants -from vtdb import update_stream -from vtproto import topodata_pb2 import environment import tablet @@ -134,13 +133,28 @@ def tearDownModule(): t.remove_tree() -def _get_update_stream(tblt): - protocol, endpoint = tblt.update_stream_python_endpoint() - return update_stream.connect(protocol, endpoint, 30) - - class TestBinlog(unittest.TestCase): + def _wait_for_replica_event(self, position, sql): + """Wait for a replica event with the given SQL string.""" + while True: + event = utils.run_vtctl_json(['VtTabletUpdateStream', + '-position', position, + '-count', '1', + dst_replica.tablet_alias]) + if 'statements' not in event: + logging.debug('skipping event with no statements: %s', event) + for statement in event['statements']: + if 'sql' not in statement: + logging.debug('skipping statement with no sql: %s', statement) + continue + base64sql = statement['sql'] + s = base64.standard_b64decode(base64sql) + logging.debug('found sql: %s', s) + if s == sql: + return + position = event['event_token']['position'] + def test_charset(self): start_position = mysql_flavor().master_position(dst_replica) logging.debug('test_charset: starting @ %s', start_position) @@ -158,12 +172,12 @@ class TestBinlog(unittest.TestCase): conn_params={'charset': 'latin1'}, write=True) # Wait for it to replicate. - stream = _get_update_stream(dst_replica) - for event in stream.stream_update('test_keyspace', '-', - topodata_pb2.REPLICA, start_position): - if event.event_token.position: - break - stream.close() + event = utils.run_vtctl_json(['VtTabletUpdateStream', + '-position', start_position, + '-count', '1', + dst_replica.tablet_alias]) + self.assertIn('event_token', event) + self.assertIn('timestamp', event['event_token']) # Check the value. data = dst_master.mquery( @@ -194,16 +208,7 @@ class TestBinlog(unittest.TestCase): # Look for it using update stream to see if binlog streamer can talk to # dst_replica, which now has binlog_checksum enabled. - stream = _get_update_stream(dst_replica) - found = False - for event in stream.stream_update('test_keyspace', '-', - topodata_pb2.REPLICA, start_position): - for statement in event.statements: - if statement.sql == sql: - found = True - break - stream.close() - self.assertEqual(found, True, 'expected query not found in update stream') + self._wait_for_replica_event(start_position, sql) def test_checksum_disabled(self): # Disable binlog_checksum to make sure we can also talk to a server without @@ -224,16 +229,7 @@ class TestBinlog(unittest.TestCase): # Look for it using update stream to see if binlog streamer can talk to # dst_replica, which now has binlog_checksum disabled. - stream = _get_update_stream(dst_replica) - found = False - for event in stream.stream_update('test_keyspace', '-', - topodata_pb2.REPLICA, start_position): - for statement in event.statements: - if statement.sql == sql: - found = True - break - stream.close() - self.assertEqual(found, True, 'expected query not found in update stream') + self._wait_for_replica_event(start_position, sql) if __name__ == '__main__': diff --git a/test/grpc_protocols_flavor.py b/test/grpc_protocols_flavor.py index faa11fb217..e6a1edf2a4 100644 --- a/test/grpc_protocols_flavor.py +++ b/test/grpc_protocols_flavor.py @@ -8,7 +8,6 @@ import protocols_flavor # Now imports all the implementations we need. # We will change this to explicit registration soon. from vtctl import grpc_vtctl_client # pylint: disable=unused-import -from vtdb import grpc_update_stream # pylint: disable=unused-import from vtdb import grpc_vtgate_client # pylint: disable=unused-import @@ -19,9 +18,6 @@ class GRpcProtocolsFlavor(protocols_flavor.ProtocolsFlavor): def binlog_player_protocol(self): return 'grpc' - def binlog_player_python_protocol(self): - return 'grpc' - def vtctl_client_protocol(self): return 'grpc' diff --git a/test/protocols_flavor.py b/test/protocols_flavor.py index e0d2e8ead8..83c258a3f7 100644 --- a/test/protocols_flavor.py +++ b/test/protocols_flavor.py @@ -13,13 +13,6 @@ class ProtocolsFlavor(object): """The binlog player protocol between vttablets, in go.""" raise NotImplementedError('Not implemented in the base class') - def binlog_player_python_protocol(self): - """The binlog player protocol in for python connections. - - This is for python connections to update_stream service. - """ - raise NotImplementedError('Not implemented in the base class') - def vtctl_client_protocol(self): """The protocol to use for vtctl connections. diff --git a/test/tablet.py b/test/tablet.py index 49bdf48421..6191eadde9 100644 --- a/test/tablet.py +++ b/test/tablet.py @@ -118,13 +118,6 @@ class Tablet(object): return 'tablet: uid: %d web: http://localhost:%d/ rpc port: %d' % ( self.tablet_uid, self.port, self.grpc_port) - def update_stream_python_endpoint(self): - protocol = protocols_flavor().binlog_player_python_protocol() - port = self.port - if protocol == 'grpc': - port = self.grpc_port - return (protocol, 'localhost:%d' % port) - def mysqlctl(self, cmd, extra_my_cnf=None, with_ports=False, verbose=False): """Runs a mysqlctl command. diff --git a/test/update_stream.py b/test/update_stream.py index b5bc4b1e39..0c7b2414be 100755 --- a/test/update_stream.py +++ b/test/update_stream.py @@ -8,8 +8,9 @@ import environment import tablet import utils from vtdb import dbexceptions -from vtdb import update_stream +from vtdb import proto3_encoding from vtdb import vtgate_client +from vtproto import query_pb2 from vtproto import topodata_pb2 from mysql_flavor import mysql_flavor from protocols_flavor import protocols_flavor @@ -18,6 +19,9 @@ master_tablet = tablet.Tablet() replica_tablet = tablet.Tablet() master_host = 'localhost:%d' % master_tablet.port +# master_start_position has the replication position before we start +# doing anything to the master database. It is used by test_ddl to +# make sure we see DDLs. master_start_position = None _create_vt_insert_test = '''create table if not exists vt_insert_test ( @@ -159,13 +163,9 @@ class TestUpdateStream(unittest.TestCase): "insert into vt_b (eid, name, foo) values (%d, 'name %s', 'foo %s')" % (x, x, x) for x in xrange(count)] - def _get_master_stream_conn(self): - protocol, endpoint = master_tablet.update_stream_python_endpoint() - return update_stream.connect(protocol, endpoint, 30) - - def _get_replica_stream_conn(self): - protocol, endpoint = replica_tablet.update_stream_python_endpoint() - return update_stream.connect(protocol, endpoint, 30) + def _get_vtgate_stream_conn(self): + protocol, addr = utils.vtgate.rpc_endpoint(python=True) + return vtgate_client.connect(protocol, addr, 30.0) def _exec_vt_txn(self, query_list): protocol, addr = utils.vtgate.rpc_endpoint(python=True) @@ -185,44 +185,46 @@ class TestUpdateStream(unittest.TestCase): Also tests transactions are retrieved properly. """ - global master_start_position - timeout = 30 while True: - master_start_position = _get_master_current_position() - replica_start_position = _get_repl_current_position() - if master_start_position == replica_start_position: + master_position = _get_master_current_position() + replica_position = _get_repl_current_position() + if master_position == replica_position: break timeout = utils.wait_step( - '%s == %s' % (master_start_position, replica_start_position), + '%s == %s' % (master_position, replica_position), timeout ) logging.debug('run_test_stream_parity starting @ %s', - master_start_position) + master_position) self._exec_vt_txn(self._populate_vt_a(15)) self._exec_vt_txn(self._populate_vt_b(14)) self._exec_vt_txn(['delete from vt_a']) self._exec_vt_txn(['delete from vt_b']) # get master events - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() master_events = [] - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - position=master_start_position): + for event, resume_timestamp in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + event=query_pb2.EventToken(shard='0', position=master_position), + shard='0'): + logging.debug('Got master event(%d): %s', resume_timestamp, event) master_events.append(event) - if event.event_token.position: + if len(master_events) == 4: break master_conn.close() # get replica events + replica_conn = self._get_vtgate_stream_conn() replica_events = [] - replica_conn = self._get_replica_stream_conn() - for event in replica_conn.stream_update('test_keyspace', '0', - topodata_pb2.REPLICA, - position=replica_start_position): + for event, resume_timestamp in replica_conn.update_stream( + 'test_keyspace', topodata_pb2.REPLICA, + event=query_pb2.EventToken(shard='0', position=replica_position), + shard='0'): + logging.debug('Got slave event(%d): %s', resume_timestamp, event) replica_events.append(event) - if event.event_token.position: + if len(replica_events) == 4: break replica_conn.close() @@ -246,16 +248,19 @@ class TestUpdateStream(unittest.TestCase): logging.debug('Test Writes: PASS') def test_ddl(self): + """Asks for all statements since we started, find the DDL.""" start_position = master_start_position logging.debug('test_ddl: starting @ %s', start_position) - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() found = False - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - position=start_position): + for event, _ in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): for statement in event.statements: if statement.sql == _create_vt_insert_test: found = True + break break master_conn.close() self.assertTrue(found, "didn't get right sql") @@ -265,13 +270,14 @@ class TestUpdateStream(unittest.TestCase): self._exec_vt_txn( ['SET INSERT_ID=1000000'] + self._populate_vt_insert_test) logging.debug('test_set_insert_id: starting @ %s', start_position) - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() expected_id = 1000000 - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - position=start_position): + for event, _ in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): for statement in event.statements: - fields, rows = update_stream.convert_statement(statement) + fields, rows = proto3_encoding.convert_stream_event_statement(statement) self.assertEqual(fields[0], 'id') self.assertEqual(rows[0][0], expected_id) expected_id += 1 @@ -285,10 +291,11 @@ class TestUpdateStream(unittest.TestCase): master_tablet.mquery('other_database', _create_vt_insert_test) self._exec_vt_txn(self._populate_vt_insert_test) logging.debug('test_database_filter: starting @ %s', start_position) - master_conn = self._get_master_stream_conn() - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - position=start_position): + master_conn = self._get_vtgate_stream_conn() + for event, _ in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): for statement in event.statements: self.assertNotEqual(statement.category, 2, # query_pb2.StreamEvent.DDL "query using other_database wasn't filtered out") @@ -307,17 +314,26 @@ class TestUpdateStream(unittest.TestCase): self.fail("Update stream service should be 'Disabled' but is '%s'" % v['UpdateStreamState']) - # Make sure we can't start a new request. start_position = _get_repl_current_position() - replica_conn = self._get_replica_stream_conn() + + # Make sure we can't start a new request to vttablet directly. + _, stderr = utils.run_vtctl(['VtTabletUpdateStream', + '-position', start_position, + replica_tablet.tablet_alias], + expect_fail=True) + self.assertIn('operation not allowed in state NOT_SERVING', stderr) + + # Make sure we can't start a new request through vtgate. + replica_conn = self._get_vtgate_stream_conn() try: - for event in replica_conn.stream_update('test_keyspace', '0', - topodata_pb2.REPLICA, - position=start_position): - self.assertFail('got event: %s' % str(event)) - self.assertFail('stream_update terminated with no exception') + for event, resume_timestamp in replica_conn.update_stream( + 'test_keyspace', topodata_pb2.REPLICA, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): + self.assertFail('got event(%d): %s' % (resume_timestamp, str(event))) + self.assertFail('update_stream terminated with no exception') except dbexceptions.DatabaseError as e: - self.assertIn('operation not allowed in state NOT_SERVING', str(e)) + self.assertIn('no valid tablet', str(e)) # Go back to replica. utils.run_vtctl( @@ -346,14 +362,16 @@ class TestUpdateStream(unittest.TestCase): # Start an Update Stream from the slave. When we get the data, go to spare. # That should interrupt the streaming RPC. - replica_conn = self._get_replica_stream_conn() + replica_conn = self._get_vtgate_stream_conn() first = True txn_count = 0 try: - for event in replica_conn.stream_update('test_keyspace', '0', - topodata_pb2.REPLICA, - position=start_position): - logging.debug('test_update_stream_interrupt got event: %s', event) + for event, resume_timestamp in replica_conn.update_stream( + 'test_keyspace', topodata_pb2.REPLICA, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): + logging.debug('test_update_stream_interrupt got event(%d): %s', + resume_timestamp, event) if first: utils.run_vtctl( ['ChangeSlaveType', replica_tablet.tablet_alias, 'spare']) @@ -363,7 +381,7 @@ class TestUpdateStream(unittest.TestCase): if event.event_token.position: txn_count += 1 - self.assertFail('stream_update terminated with no exception') + self.assertFail('update_stream terminated with no exception') except dbexceptions.DatabaseError as e: self.assertIn('context canceled', str(e)) self.assertFalse(first) @@ -378,12 +396,13 @@ class TestUpdateStream(unittest.TestCase): master_tablet.mquery('vt_test_keyspace', 'flush logs') self._exec_vt_txn(self._populate_vt_a(15)) self._exec_vt_txn(['delete from vt_a']) - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() master_txn_count = 0 logs_correct = False - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - position=start_position): + for event, _ in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + event=query_pb2.EventToken(shard='0', position=start_position), + shard='0'): if event.event_token.position: master_txn_count += 1 position = event.event_token.position @@ -477,13 +496,14 @@ class TestUpdateStream(unittest.TestCase): self._exec_vt_txn(['delete from vt_b']) # make sure we only get events related to vt_b. - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() count = 0 - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - timestamp=starting_timestamp+2): - logging.debug('_test_timestamp_start: got event: %s', - str(event)) + for (event, resume_timestamp) in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + timestamp=starting_timestamp+2, + shard='0'): + logging.debug('_test_timestamp_start: got event: %s @ %d', + str(event), resume_timestamp) # we might get a couple extra events from the rotation, ignore these. if event.statements[0].category == 0: # Statement.Category.Error continue @@ -497,13 +517,14 @@ class TestUpdateStream(unittest.TestCase): def test_timestamp_start_too_old(self): """Ask the server to start streaming from a timestamp 4h ago.""" starting_timestamp = long(time.time()) - 4*60*60 - master_conn = self._get_master_stream_conn() + master_conn = self._get_vtgate_stream_conn() try: - for event in master_conn.stream_update('test_keyspace', '0', - topodata_pb2.MASTER, - timestamp=starting_timestamp): - self.assertFail('got an event: %s' % str(event)) - except dbexceptions.OperationalError as e: + for (event, resume_timestamp) in master_conn.update_stream( + 'test_keyspace', topodata_pb2.MASTER, + timestamp=starting_timestamp, + shard='0'): + self.assertFail('got an event: %s %d' % (str(event), resume_timestamp)) + except dbexceptions.QueryNotServed as e: self.assertIn('retry: cannot find relevant binlogs on this server', str(e))