зеркало из https://github.com/github/vitess-gh.git
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.
This commit is contained in:
Родитель
50e82c1f1b
Коммит
b48a84ca64
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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]
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -92,6 +92,11 @@ func init() {
|
|||
commandVtTabletStreamHealth,
|
||||
"[-count <count, default 1>] [-connect_timeout <connect timeout>] <tablet alias>",
|
||||
"Executes the StreamHealth streaming query to a vttablet process. Will stop after getting <count> answers."})
|
||||
addCommand(queriesGroupName, command{
|
||||
"VtTabletUpdateStream",
|
||||
commandVtTabletUpdateStream,
|
||||
"[-count <count, default 1>] [-connect_timeout <connect timeout>] [-position <position>] [-timestamp <timestamp>] <tablet alias>",
|
||||
"Executes the UpdateStream streaming query to a vttablet process. Will stop after getting <count> 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 <tablet alias> 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 {
|
||||
|
|
|
@ -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() {
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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") {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -1,13 +0,0 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: binlogdata.proto
|
||||
|
||||
namespace Vitess\Proto\Binlogdata\StreamEvent {
|
||||
|
||||
class Category extends \DrSlump\Protobuf\Enum {
|
||||
const SE_ERR = 0;
|
||||
const SE_DML = 1;
|
||||
const SE_DDL = 2;
|
||||
const SE_POS = 3;
|
||||
}
|
||||
}
|
|
@ -1,74 +0,0 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: binlogdata.proto
|
||||
|
||||
namespace Vitess\Proto\Binlogdata {
|
||||
|
||||
class StreamUpdateResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Binlogdata\StreamEvent */
|
||||
public $stream_event = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'binlogdata.StreamUpdateResponse');
|
||||
|
||||
// OPTIONAL MESSAGE stream_event = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <stream_event> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasStreamEvent(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <stream_event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Binlogdata\StreamUpdateResponse
|
||||
*/
|
||||
public function clearStreamEvent(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <stream_event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Binlogdata\StreamEvent
|
||||
*/
|
||||
public function getStreamEvent(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <stream_event> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -0,0 +1,143 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: query.proto
|
||||
|
||||
namespace Vitess\Proto\Query {
|
||||
|
||||
class StreamEvent extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Query\StreamEvent\Statement[] */
|
||||
public $statements = array();
|
||||
|
||||
/** @var \Vitess\Proto\Query\EventToken */
|
||||
public $event_token = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'query.StreamEvent');
|
||||
|
||||
// REPEATED MESSAGE statements = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <statements> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasStatements(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <statements> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\StreamEvent
|
||||
*/
|
||||
public function clearStatements(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <statements> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Query\StreamEvent\Statement
|
||||
*/
|
||||
public function getStatements($idx = NULL){
|
||||
return $this->_get(1, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <statements> 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 <statements>
|
||||
*
|
||||
* @return \Vitess\Proto\Query\StreamEvent\Statement[]
|
||||
*/
|
||||
public function getStatementsList(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <statements>
|
||||
*
|
||||
* @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 <event_token> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEventToken(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <event_token> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\StreamEvent
|
||||
*/
|
||||
public function clearEventToken(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <event_token> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\EventToken
|
||||
*/
|
||||
public function getEventToken(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <event_token> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,12 +1,12 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: binlogdata.proto
|
||||
// Source: query.proto
|
||||
|
||||
namespace Vitess\Proto\Binlogdata {
|
||||
namespace Vitess\Proto\Query\StreamEvent {
|
||||
|
||||
class StreamEvent extends \DrSlump\Protobuf\Message {
|
||||
class Statement extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var int - \Vitess\Proto\Binlogdata\StreamEvent\Category */
|
||||
/** @var int - \Vitess\Proto\Query\StreamEvent\Statement\Category */
|
||||
public $category = null;
|
||||
|
||||
/** @var string */
|
||||
|
@ -21,16 +21,13 @@ namespace Vitess\Proto\Binlogdata {
|
|||
/** @var string */
|
||||
public $sql = null;
|
||||
|
||||
/** @var \Vitess\Proto\Query\EventToken */
|
||||
public $event_token = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'binlogdata.StreamEvent');
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'query.StreamEvent.Statement');
|
||||
|
||||
// OPTIONAL ENUM category = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
|
@ -38,7 +35,7 @@ namespace Vitess\Proto\Binlogdata {
|
|||
$f->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 <category> 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 <category> 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 <category> 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 <table_name> 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 <table_name> 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 <primary_key_fields> 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 <primary_key_fields> 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 <primary_key_fields>
|
||||
*
|
||||
* @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 <primary_key_values> 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 <primary_key_values> 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 <primary_key_values>
|
||||
*
|
||||
* @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 <sql> 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 <sql> 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 <event_token> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEventToken(){
|
||||
return $this->_has(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <event_token> value
|
||||
*
|
||||
* @return \Vitess\Proto\Binlogdata\StreamEvent
|
||||
*/
|
||||
public function clearEventToken(){
|
||||
return $this->_clear(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <event_token> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\EventToken
|
||||
*/
|
||||
public function getEventToken(){
|
||||
return $this->_get(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <event_token> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,12 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: query.proto
|
||||
|
||||
namespace Vitess\Proto\Query\StreamEvent\Statement {
|
||||
|
||||
class Category extends \DrSlump\Protobuf\Enum {
|
||||
const Error = 0;
|
||||
const DML = 1;
|
||||
const DDL = 2;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,268 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: query.proto
|
||||
|
||||
namespace Vitess\Proto\Query {
|
||||
|
||||
class UpdateStreamRequest extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Vtrpc\CallerID */
|
||||
public $effective_caller_id = null;
|
||||
|
||||
/** @var \Vitess\Proto\Query\VTGateCallerID */
|
||||
public $immediate_caller_id = null;
|
||||
|
||||
/** @var \Vitess\Proto\Query\Target */
|
||||
public $target = null;
|
||||
|
||||
/** @var string */
|
||||
public $position = null;
|
||||
|
||||
/** @var int */
|
||||
public $timestamp = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'query.UpdateStreamRequest');
|
||||
|
||||
// OPTIONAL MESSAGE effective_caller_id = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <effective_caller_id> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEffectiveCallerId(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <effective_caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function clearEffectiveCallerId(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <effective_caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtrpc\CallerID
|
||||
*/
|
||||
public function getEffectiveCallerId(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <effective_caller_id> 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 <immediate_caller_id> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasImmediateCallerId(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <immediate_caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function clearImmediateCallerId(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <immediate_caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\VTGateCallerID
|
||||
*/
|
||||
public function getImmediateCallerId(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <immediate_caller_id> 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 <target> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTarget(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <target> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function clearTarget(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <target> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\Target
|
||||
*/
|
||||
public function getTarget(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <target> 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 <position> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasPosition(){
|
||||
return $this->_has(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <position> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function clearPosition(){
|
||||
return $this->_clear(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <position> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getPosition(){
|
||||
return $this->_get(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <position> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function setPosition( $value){
|
||||
return $this->_set(4, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <timestamp> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTimestamp(){
|
||||
return $this->_has(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <timestamp> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function clearTimestamp(){
|
||||
return $this->_clear(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <timestamp> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getTimestamp(){
|
||||
return $this->_get(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <timestamp> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Query\UpdateStreamRequest
|
||||
*/
|
||||
public function setTimestamp( $value){
|
||||
return $this->_set(5, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,74 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: query.proto
|
||||
|
||||
namespace Vitess\Proto\Query {
|
||||
|
||||
class UpdateStreamResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Query\StreamEvent */
|
||||
public $event = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'query.UpdateStreamResponse');
|
||||
|
||||
// OPTIONAL MESSAGE event = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <event> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEvent(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\UpdateStreamResponse
|
||||
*/
|
||||
public function clearEvent(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\StreamEvent
|
||||
*/
|
||||
public function getEvent(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <event> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,505 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class Configuration extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var int */
|
||||
public $target_replication_lag_sec = null;
|
||||
|
||||
/** @var int */
|
||||
public $max_replication_lag_sec = null;
|
||||
|
||||
/** @var int */
|
||||
public $initial_rate = null;
|
||||
|
||||
/** @var float */
|
||||
public $max_increase = null;
|
||||
|
||||
/** @var float */
|
||||
public $emergency_decrease = null;
|
||||
|
||||
/** @var int */
|
||||
public $min_duration_between_changes_sec = null;
|
||||
|
||||
/** @var int */
|
||||
public $max_duration_between_increases_sec = null;
|
||||
|
||||
/** @var int */
|
||||
public $ignore_n_slowest_replicas = null;
|
||||
|
||||
/** @var int */
|
||||
public $age_bad_rate_after_sec = null;
|
||||
|
||||
/** @var float */
|
||||
public $bad_rate_increase = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.Configuration');
|
||||
|
||||
// OPTIONAL INT64 target_replication_lag_sec = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <target_replication_lag_sec> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTargetReplicationLagSec(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <target_replication_lag_sec> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearTargetReplicationLagSec(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <target_replication_lag_sec> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getTargetReplicationLagSec(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <target_replication_lag_sec> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setTargetReplicationLagSec( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <max_replication_lag_sec> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasMaxReplicationLagSec(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <max_replication_lag_sec> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearMaxReplicationLagSec(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <max_replication_lag_sec> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getMaxReplicationLagSec(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <max_replication_lag_sec> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setMaxReplicationLagSec( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <initial_rate> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasInitialRate(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <initial_rate> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearInitialRate(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <initial_rate> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getInitialRate(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <initial_rate> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setInitialRate( $value){
|
||||
return $this->_set(3, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <max_increase> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasMaxIncrease(){
|
||||
return $this->_has(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <max_increase> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearMaxIncrease(){
|
||||
return $this->_clear(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <max_increase> value
|
||||
*
|
||||
* @return float
|
||||
*/
|
||||
public function getMaxIncrease(){
|
||||
return $this->_get(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <max_increase> value
|
||||
*
|
||||
* @param float $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setMaxIncrease( $value){
|
||||
return $this->_set(4, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <emergency_decrease> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEmergencyDecrease(){
|
||||
return $this->_has(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <emergency_decrease> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearEmergencyDecrease(){
|
||||
return $this->_clear(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <emergency_decrease> value
|
||||
*
|
||||
* @return float
|
||||
*/
|
||||
public function getEmergencyDecrease(){
|
||||
return $this->_get(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <emergency_decrease> value
|
||||
*
|
||||
* @param float $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setEmergencyDecrease( $value){
|
||||
return $this->_set(5, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <min_duration_between_changes_sec> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasMinDurationBetweenChangesSec(){
|
||||
return $this->_has(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <min_duration_between_changes_sec> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearMinDurationBetweenChangesSec(){
|
||||
return $this->_clear(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <min_duration_between_changes_sec> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getMinDurationBetweenChangesSec(){
|
||||
return $this->_get(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <min_duration_between_changes_sec> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setMinDurationBetweenChangesSec( $value){
|
||||
return $this->_set(6, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <max_duration_between_increases_sec> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasMaxDurationBetweenIncreasesSec(){
|
||||
return $this->_has(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <max_duration_between_increases_sec> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearMaxDurationBetweenIncreasesSec(){
|
||||
return $this->_clear(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <max_duration_between_increases_sec> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getMaxDurationBetweenIncreasesSec(){
|
||||
return $this->_get(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <max_duration_between_increases_sec> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setMaxDurationBetweenIncreasesSec( $value){
|
||||
return $this->_set(7, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <ignore_n_slowest_replicas> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasIgnoreNSlowestReplicas(){
|
||||
return $this->_has(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <ignore_n_slowest_replicas> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearIgnoreNSlowestReplicas(){
|
||||
return $this->_clear(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <ignore_n_slowest_replicas> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getIgnoreNSlowestReplicas(){
|
||||
return $this->_get(8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <ignore_n_slowest_replicas> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setIgnoreNSlowestReplicas( $value){
|
||||
return $this->_set(8, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <age_bad_rate_after_sec> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasAgeBadRateAfterSec(){
|
||||
return $this->_has(9);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <age_bad_rate_after_sec> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearAgeBadRateAfterSec(){
|
||||
return $this->_clear(9);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <age_bad_rate_after_sec> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getAgeBadRateAfterSec(){
|
||||
return $this->_get(9);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <age_bad_rate_after_sec> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setAgeBadRateAfterSec( $value){
|
||||
return $this->_set(9, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <bad_rate_increase> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasBadRateIncrease(){
|
||||
return $this->_has(10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <bad_rate_increase> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function clearBadRateIncrease(){
|
||||
return $this->_clear(10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <bad_rate_increase> value
|
||||
*
|
||||
* @return float
|
||||
*/
|
||||
public function getBadRateIncrease(){
|
||||
return $this->_get(10);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <bad_rate_increase> value
|
||||
*
|
||||
* @param float $value
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function setBadRateIncrease( $value){
|
||||
return $this->_set(10, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,13 +1,13 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: binlogdata.proto
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Binlogdata {
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class StreamUpdateRequest extends \DrSlump\Protobuf\Message {
|
||||
class GetConfigurationRequest extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $position = null;
|
||||
public $throttler_name = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
|
@ -15,12 +15,12 @@ namespace Vitess\Proto\Binlogdata {
|
|||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'binlogdata.StreamUpdateRequest');
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.GetConfigurationRequest');
|
||||
|
||||
// OPTIONAL STRING position = 1
|
||||
// OPTIONAL STRING throttler_name = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <position> has a value
|
||||
* Check if <throttler_name> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasPosition(){
|
||||
public function hasThrottlerName(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <position> value
|
||||
* Clear <throttler_name> value
|
||||
*
|
||||
* @return \Vitess\Proto\Binlogdata\StreamUpdateRequest
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationRequest
|
||||
*/
|
||||
public function clearPosition(){
|
||||
public function clearThrottlerName(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <position> value
|
||||
* Get <throttler_name> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getPosition(){
|
||||
public function getThrottlerName(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <position> value
|
||||
* Set <throttler_name> 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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class GetConfigurationResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry[] */
|
||||
public $configurations = array();
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.GetConfigurationResponse');
|
||||
|
||||
// REPEATED MESSAGE configurations = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <configurations> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasConfigurations(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <configurations> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse
|
||||
*/
|
||||
public function clearConfigurations(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <configurations> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry
|
||||
*/
|
||||
public function getConfigurations($idx = NULL){
|
||||
return $this->_get(1, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <configurations> 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 <configurations>
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry[]
|
||||
*/
|
||||
public function getConfigurationsList(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <configurations>
|
||||
*
|
||||
* @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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,122 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata\GetConfigurationResponse {
|
||||
|
||||
class ConfigurationsEntry extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $key = null;
|
||||
|
||||
/** @var \Vitess\Proto\Throttlerdata\Configuration */
|
||||
public $value = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.GetConfigurationResponse.ConfigurationsEntry');
|
||||
|
||||
// OPTIONAL STRING key = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <key> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKey(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <key> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry
|
||||
*/
|
||||
public function clearKey(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <key> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getKey(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <key> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry
|
||||
*/
|
||||
public function setKey( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <value> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasValue(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\GetConfigurationResponse\ConfigurationsEntry
|
||||
*/
|
||||
public function clearValue(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function getValue(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <value> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,73 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class ResetConfigurationRequest extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $throttler_name = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.ResetConfigurationRequest');
|
||||
|
||||
// OPTIONAL STRING throttler_name = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <throttler_name> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasThrottlerName(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <throttler_name> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\ResetConfigurationRequest
|
||||
*/
|
||||
public function clearThrottlerName(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <throttler_name> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getThrottlerName(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <throttler_name> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Throttlerdata\ResetConfigurationRequest
|
||||
*/
|
||||
public function setThrottlerName( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,93 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class ResetConfigurationResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string[] */
|
||||
public $names = array();
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.ResetConfigurationResponse');
|
||||
|
||||
// REPEATED STRING names = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <names> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasNames(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <names> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\ResetConfigurationResponse
|
||||
*/
|
||||
public function clearNames(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <names> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return string
|
||||
*/
|
||||
public function getNames($idx = NULL){
|
||||
return $this->_get(1, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <names> 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 <names>
|
||||
*
|
||||
* @return string[]
|
||||
*/
|
||||
public function getNamesList(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <names>
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Throttlerdata\ResetConfigurationResponse
|
||||
*/
|
||||
public function addNames( $value){
|
||||
return $this->_add(1, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,170 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class UpdateConfigurationRequest extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $throttler_name = null;
|
||||
|
||||
/** @var \Vitess\Proto\Throttlerdata\Configuration */
|
||||
public $configuration = null;
|
||||
|
||||
/** @var boolean */
|
||||
public $copy_zero_values = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.UpdateConfigurationRequest');
|
||||
|
||||
// OPTIONAL STRING throttler_name = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <throttler_name> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasThrottlerName(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <throttler_name> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest
|
||||
*/
|
||||
public function clearThrottlerName(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <throttler_name> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getThrottlerName(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <throttler_name> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest
|
||||
*/
|
||||
public function setThrottlerName( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <configuration> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasConfiguration(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <configuration> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest
|
||||
*/
|
||||
public function clearConfiguration(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <configuration> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\Configuration
|
||||
*/
|
||||
public function getConfiguration(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <configuration> 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 <copy_zero_values> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasCopyZeroValues(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <copy_zero_values> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest
|
||||
*/
|
||||
public function clearCopyZeroValues(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <copy_zero_values> value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function getCopyZeroValues(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <copy_zero_values> value
|
||||
*
|
||||
* @param boolean $value
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationRequest
|
||||
*/
|
||||
public function setCopyZeroValues( $value){
|
||||
return $this->_set(3, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,93 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: throttlerdata.proto
|
||||
|
||||
namespace Vitess\Proto\Throttlerdata {
|
||||
|
||||
class UpdateConfigurationResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string[] */
|
||||
public $names = array();
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'throttlerdata.UpdateConfigurationResponse');
|
||||
|
||||
// REPEATED STRING names = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <names> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasNames(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <names> value
|
||||
*
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationResponse
|
||||
*/
|
||||
public function clearNames(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <names> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return string
|
||||
*/
|
||||
public function getNames($idx = NULL){
|
||||
return $this->_get(1, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <names> 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 <names>
|
||||
*
|
||||
* @return string[]
|
||||
*/
|
||||
public function getNamesList(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <names>
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Throttlerdata\UpdateConfigurationResponse
|
||||
*/
|
||||
public function addNames( $value){
|
||||
return $this->_add(1, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,365 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vtgate.proto
|
||||
|
||||
namespace Vitess\Proto\Vtgate {
|
||||
|
||||
class UpdateStreamRequest extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Vtrpc\CallerID */
|
||||
public $caller_id = null;
|
||||
|
||||
/** @var string */
|
||||
public $keyspace = null;
|
||||
|
||||
/** @var string */
|
||||
public $shard = null;
|
||||
|
||||
/** @var \Vitess\Proto\Topodata\KeyRange */
|
||||
public $key_range = null;
|
||||
|
||||
/** @var int - \Vitess\Proto\Topodata\TabletType */
|
||||
public $tablet_type = null;
|
||||
|
||||
/** @var int */
|
||||
public $timestamp = null;
|
||||
|
||||
/** @var \Vitess\Proto\Query\EventToken */
|
||||
public $event = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vtgate.UpdateStreamRequest');
|
||||
|
||||
// OPTIONAL MESSAGE caller_id = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <caller_id> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasCallerId(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearCallerId(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <caller_id> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtrpc\CallerID
|
||||
*/
|
||||
public function getCallerId(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <caller_id> 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 <keyspace> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKeyspace(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <keyspace> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearKeyspace(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <keyspace> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getKeyspace(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <keyspace> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function setKeyspace( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <shard> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasShard(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <shard> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearShard(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <shard> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getShard(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <shard> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function setShard( $value){
|
||||
return $this->_set(3, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <key_range> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKeyRange(){
|
||||
return $this->_has(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <key_range> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearKeyRange(){
|
||||
return $this->_clear(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <key_range> value
|
||||
*
|
||||
* @return \Vitess\Proto\Topodata\KeyRange
|
||||
*/
|
||||
public function getKeyRange(){
|
||||
return $this->_get(4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <key_range> 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 <tablet_type> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTabletType(){
|
||||
return $this->_has(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <tablet_type> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearTabletType(){
|
||||
return $this->_clear(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <tablet_type> value
|
||||
*
|
||||
* @return int - \Vitess\Proto\Topodata\TabletType
|
||||
*/
|
||||
public function getTabletType(){
|
||||
return $this->_get(5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <tablet_type> value
|
||||
*
|
||||
* @param int - \Vitess\Proto\Topodata\TabletType $value
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function setTabletType( $value){
|
||||
return $this->_set(5, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <timestamp> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTimestamp(){
|
||||
return $this->_has(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <timestamp> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearTimestamp(){
|
||||
return $this->_clear(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <timestamp> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getTimestamp(){
|
||||
return $this->_get(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <timestamp> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function setTimestamp( $value){
|
||||
return $this->_set(6, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <event> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEvent(){
|
||||
return $this->_has(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamRequest
|
||||
*/
|
||||
public function clearEvent(){
|
||||
return $this->_clear(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\EventToken
|
||||
*/
|
||||
public function getEvent(){
|
||||
return $this->_get(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <event> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,122 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vtgate.proto
|
||||
|
||||
namespace Vitess\Proto\Vtgate {
|
||||
|
||||
class UpdateStreamResponse extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var \Vitess\Proto\Query\StreamEvent */
|
||||
public $event = null;
|
||||
|
||||
/** @var int */
|
||||
public $resume_timestamp = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vtgate.UpdateStreamResponse');
|
||||
|
||||
// OPTIONAL MESSAGE event = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->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 <event> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasEvent(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamResponse
|
||||
*/
|
||||
public function clearEvent(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <event> value
|
||||
*
|
||||
* @return \Vitess\Proto\Query\StreamEvent
|
||||
*/
|
||||
public function getEvent(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <event> 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 <resume_timestamp> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasResumeTimestamp(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <resume_timestamp> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamResponse
|
||||
*/
|
||||
public function clearResumeTimestamp(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <resume_timestamp> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getResumeTimestamp(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <resume_timestamp> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Vtgate\UpdateStreamResponse
|
||||
*/
|
||||
public function setResumeTimestamp( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 <replica_count> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasReplicaCount(){
|
||||
return $this->_has(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <replica_count> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vttest\Keyspace
|
||||
*/
|
||||
public function clearReplicaCount(){
|
||||
return $this->_clear(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <replica_count> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getReplicaCount(){
|
||||
return $this->_get(6);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <replica_count> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Vttest\Keyspace
|
||||
*/
|
||||
public function setReplicaCount( $value){
|
||||
return $this->_set(6, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <rdonly_count> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasRdonlyCount(){
|
||||
return $this->_has(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <rdonly_count> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vttest\Keyspace
|
||||
*/
|
||||
public function clearRdonlyCount(){
|
||||
return $this->_clear(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <rdonly_count> value
|
||||
*
|
||||
* @return int
|
||||
*/
|
||||
public function getRdonlyCount(){
|
||||
return $this->_get(7);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <rdonly_count> value
|
||||
*
|
||||
* @param int $value
|
||||
* @return \Vitess\Proto\Vttest\Keyspace
|
||||
*/
|
||||
public function setRdonlyCount( $value){
|
||||
return $this->_set(7, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 <cells> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasCells(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <cells> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vttest\VTTestTopology
|
||||
*/
|
||||
public function clearCells(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <cells> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return string
|
||||
*/
|
||||
public function getCells($idx = NULL){
|
||||
return $this->_get(2, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <cells> 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 <cells>
|
||||
*
|
||||
* @return string[]
|
||||
*/
|
||||
public function getCellsList(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <cells>
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vttest\VTTestTopology
|
||||
*/
|
||||
public function addCells( $value){
|
||||
return $this->_add(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {};
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 '<GRPCUpdateStreamConnection %s>' % 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)
|
|
@ -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,))
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
|
@ -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')
|
||||
|
|
Различия файлов скрыты, потому что одна или несколько строк слишком длинны
|
@ -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)
|
||||
|
|
|
@ -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__':
|
||||
|
|
|
@ -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'
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
||||
|
|
|
@ -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))
|
||||
|
||||
|
|
Загрузка…
Ссылка в новой задаче