зеркало из https://github.com/github/vitess-gh.git
Moving UpdateStream to queryservice API.
The UpdateStream call makes a lot more sense in the QueryService part of the API. It is meant to be used by vtgate. The Binlog side of the API is then only used by filtered replication. Adding vttablet support to start streaming from a timestamp, not just a replication position. It finds the starting position by loking at the binary logs. In the process, change the following things: - UpdateStream is now added to gateway.Gateway, so it flows through l2vtgate if needed. - the python client now uses the new service. - updated the tests as required. - using Context instead of sync2.ServiceManager. Removing sync2.ServiceManager as it is now unused. - tabletserver.TabletServer now remembers the ongoing UpdateStream connections, and closes them on state change.
This commit is contained in:
Родитель
9267dee840
Коммит
e815e680b5
|
@ -485,6 +485,11 @@ func (itc *internalTabletConn) StreamHealth(ctx context.Context) (tabletconn.Str
|
|||
}, nil
|
||||
}
|
||||
|
||||
// UpdateStream is part of tabletconn.TabletConn. Not implemented here.
|
||||
func (itc *internalTabletConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
return nil, fmt.Errorf("not implemented in vtcombo")
|
||||
}
|
||||
|
||||
//
|
||||
// TabletManagerClient implementation
|
||||
//
|
||||
|
|
|
@ -1,122 +0,0 @@
|
|||
// Copyright 2013, 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.
|
||||
|
||||
package sync2
|
||||
|
||||
import (
|
||||
"sync"
|
||||
)
|
||||
|
||||
// These are the three predefined states of a service.
|
||||
const (
|
||||
SERVICE_STOPPED = iota
|
||||
SERVICE_RUNNING
|
||||
SERVICE_SHUTTING_DOWN
|
||||
)
|
||||
|
||||
var stateNames = []string{
|
||||
"Stopped",
|
||||
"Running",
|
||||
"ShuttingDown",
|
||||
}
|
||||
|
||||
// ServiceManager manages the state of a service through its lifecycle.
|
||||
// It's not recommended to nest service managers because they introduce
|
||||
// race conditions in the Stop functions that can cause one service to
|
||||
// indefinitely wait for the other. You can instead pass the top level
|
||||
// ServiceContext around and manage shutdown using the single
|
||||
// ServiceManager.
|
||||
type ServiceManager struct {
|
||||
mu sync.Mutex
|
||||
wg sync.WaitGroup
|
||||
err error // err is the error returned from the service function.
|
||||
state AtomicInt64
|
||||
// shutdown is created when the service starts and is closed when the service
|
||||
// enters the SERVICE_SHUTTING_DOWN state.
|
||||
shutdown chan struct{}
|
||||
}
|
||||
|
||||
// Go tries to change the state from SERVICE_STOPPED to SERVICE_RUNNING.
|
||||
//
|
||||
// If the current state is not SERVICE_STOPPED (already running), it returns
|
||||
// false immediately.
|
||||
//
|
||||
// On successful transition, it launches the service as a goroutine and returns
|
||||
// true. The service function is responsible for returning on its own when
|
||||
// requested, either by regularly checking svc.IsRunning(), or by waiting for
|
||||
// the svc.ShuttingDown channel to be closed.
|
||||
//
|
||||
// When the service func returns, the state is reverted to SERVICE_STOPPED.
|
||||
func (svm *ServiceManager) Go(service func(svc *ServiceContext) error) bool {
|
||||
svm.mu.Lock()
|
||||
defer svm.mu.Unlock()
|
||||
if !svm.state.CompareAndSwap(SERVICE_STOPPED, SERVICE_RUNNING) {
|
||||
return false
|
||||
}
|
||||
svm.wg.Add(1)
|
||||
svm.err = nil
|
||||
svm.shutdown = make(chan struct{})
|
||||
go func(shutdown chan struct{}) {
|
||||
svm.err = service(&ServiceContext{ShuttingDown: shutdown})
|
||||
svm.state.Set(SERVICE_STOPPED)
|
||||
svm.wg.Done()
|
||||
}(svm.shutdown)
|
||||
return true
|
||||
}
|
||||
|
||||
// Stop tries to change the state from SERVICE_RUNNING to SERVICE_SHUTTING_DOWN.
|
||||
// If the current state is not SERVICE_RUNNING, it returns false immediately.
|
||||
// On successul transition, it waits for the service to finish, and returns true.
|
||||
// You are allowed to Go() again after a Stop().
|
||||
func (svm *ServiceManager) Stop() bool {
|
||||
svm.mu.Lock()
|
||||
defer svm.mu.Unlock()
|
||||
if !svm.state.CompareAndSwap(SERVICE_RUNNING, SERVICE_SHUTTING_DOWN) {
|
||||
return false
|
||||
}
|
||||
// Signal the service that we've transitioned to SERVICE_SHUTTING_DOWN.
|
||||
close(svm.shutdown)
|
||||
svm.shutdown = nil
|
||||
svm.wg.Wait()
|
||||
return true
|
||||
}
|
||||
|
||||
// Join waits for the service to terminate and returns the value returned by the
|
||||
// service function.
|
||||
func (svm *ServiceManager) Join() error {
|
||||
svm.wg.Wait()
|
||||
return svm.err
|
||||
}
|
||||
|
||||
// State returns the current state of the service.
|
||||
// This should only be used to report the current state.
|
||||
func (svm *ServiceManager) State() int64 {
|
||||
return svm.state.Get()
|
||||
}
|
||||
|
||||
// StateName returns the name of the current state.
|
||||
func (svm *ServiceManager) StateName() string {
|
||||
return stateNames[svm.State()]
|
||||
}
|
||||
|
||||
// ServiceContext is passed into the service function to give it access to
|
||||
// information about the running service.
|
||||
// You can create an empty service context, in which case it will be non-cancelable.
|
||||
type ServiceContext struct {
|
||||
// ShuttingDown is a channel that the service can select on to be notified
|
||||
// when it should shut down. The channel is closed when the state transitions
|
||||
// from SERVICE_RUNNING to SERVICE_SHUTTING_DOWN.
|
||||
ShuttingDown chan struct{}
|
||||
}
|
||||
|
||||
// IsRunning returns true if the ServiceContext.ShuttingDown channel has not
|
||||
// been closed yet.
|
||||
func (svc *ServiceContext) IsRunning() bool {
|
||||
select {
|
||||
case <-svc.ShuttingDown:
|
||||
return false
|
||||
default:
|
||||
return true
|
||||
}
|
||||
}
|
|
@ -1,187 +0,0 @@
|
|||
// Copyright 2013, 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.
|
||||
|
||||
package sync2
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
type testService struct {
|
||||
activated AtomicInt64
|
||||
t *testing.T
|
||||
}
|
||||
|
||||
func (ts *testService) service(svc *ServiceContext) error {
|
||||
if !ts.activated.CompareAndSwap(0, 1) {
|
||||
ts.t.Fatalf("service called more than once")
|
||||
}
|
||||
for svc.IsRunning() {
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
|
||||
}
|
||||
if !ts.activated.CompareAndSwap(1, 0) {
|
||||
ts.t.Fatalf("service ended more than once")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (ts *testService) selectService(svc *ServiceContext) error {
|
||||
if !ts.activated.CompareAndSwap(0, 1) {
|
||||
ts.t.Fatalf("service called more than once")
|
||||
}
|
||||
serviceLoop:
|
||||
for svc.IsRunning() {
|
||||
select {
|
||||
case <-time.After(1 * time.Second):
|
||||
ts.t.Errorf("service didn't stop when shutdown channel was closed")
|
||||
case <-svc.ShuttingDown:
|
||||
break serviceLoop
|
||||
}
|
||||
}
|
||||
if !ts.activated.CompareAndSwap(1, 0) {
|
||||
ts.t.Fatalf("service ended more than once")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestServiceManager(t *testing.T) {
|
||||
ts := &testService{t: t}
|
||||
var sm ServiceManager
|
||||
if sm.StateName() != "Stopped" {
|
||||
t.Errorf("want Stopped, got %s", sm.StateName())
|
||||
}
|
||||
result := sm.Go(ts.service)
|
||||
if !result {
|
||||
t.Errorf("want true, got false")
|
||||
}
|
||||
if sm.StateName() != "Running" {
|
||||
t.Errorf("want Running, got %s", sm.StateName())
|
||||
}
|
||||
time.Sleep(5 * time.Millisecond)
|
||||
if val := ts.activated.Get(); val != 1 {
|
||||
t.Errorf("want 1, got %d", val)
|
||||
}
|
||||
result = sm.Go(ts.service)
|
||||
if result {
|
||||
t.Errorf("want false, got true")
|
||||
}
|
||||
result = sm.Stop()
|
||||
if !result {
|
||||
t.Errorf("want true, got false")
|
||||
}
|
||||
if val := ts.activated.Get(); val != 0 {
|
||||
t.Errorf("want 0, got %d", val)
|
||||
}
|
||||
result = sm.Stop()
|
||||
if result {
|
||||
t.Errorf("want false, got true")
|
||||
}
|
||||
sm.state.Set(SERVICE_SHUTTING_DOWN)
|
||||
if sm.StateName() != "ShuttingDown" {
|
||||
t.Errorf("want ShuttingDown, got %s", sm.StateName())
|
||||
}
|
||||
}
|
||||
|
||||
func TestServiceManagerSelect(t *testing.T) {
|
||||
ts := &testService{t: t}
|
||||
var sm ServiceManager
|
||||
if sm.StateName() != "Stopped" {
|
||||
t.Errorf("want Stopped, got %s", sm.StateName())
|
||||
}
|
||||
result := sm.Go(ts.selectService)
|
||||
if !result {
|
||||
t.Errorf("want true, got false")
|
||||
}
|
||||
if sm.StateName() != "Running" {
|
||||
t.Errorf("want Running, got %s", sm.StateName())
|
||||
}
|
||||
time.Sleep(5 * time.Millisecond)
|
||||
if val := ts.activated.Get(); val != 1 {
|
||||
t.Errorf("want 1, got %d", val)
|
||||
}
|
||||
result = sm.Go(ts.service)
|
||||
if result {
|
||||
t.Errorf("want false, got true")
|
||||
}
|
||||
result = sm.Stop()
|
||||
if !result {
|
||||
t.Errorf("want true, got false")
|
||||
}
|
||||
if val := ts.activated.Get(); val != 0 {
|
||||
t.Errorf("want 0, got %d", val)
|
||||
}
|
||||
result = sm.Stop()
|
||||
if result {
|
||||
t.Errorf("want false, got true")
|
||||
}
|
||||
sm.state.Set(SERVICE_SHUTTING_DOWN)
|
||||
if sm.StateName() != "ShuttingDown" {
|
||||
t.Errorf("want ShuttingDown, got %s", sm.StateName())
|
||||
}
|
||||
}
|
||||
|
||||
func TestServiceManagerJoinNotRunning(t *testing.T) {
|
||||
done := make(chan struct{})
|
||||
var sm ServiceManager
|
||||
go func() {
|
||||
sm.Join()
|
||||
close(done)
|
||||
}()
|
||||
select {
|
||||
case <-done:
|
||||
case <-time.After(1 * time.Second):
|
||||
t.Errorf("Join() blocked even though service wasn't running.")
|
||||
}
|
||||
}
|
||||
|
||||
func TestServiceManagerJoinBlocks(t *testing.T) {
|
||||
done := make(chan struct{})
|
||||
stop := make(chan struct{})
|
||||
var sm ServiceManager
|
||||
sm.Go(func(*ServiceContext) error {
|
||||
<-stop
|
||||
return nil
|
||||
})
|
||||
go func() {
|
||||
sm.Join()
|
||||
close(done)
|
||||
}()
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
select {
|
||||
case <-done:
|
||||
t.Errorf("Join() didn't block while service was still running.")
|
||||
default:
|
||||
}
|
||||
close(stop)
|
||||
select {
|
||||
case <-done:
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
t.Errorf("Join() didn't unblock when service stopped.")
|
||||
}
|
||||
}
|
||||
|
||||
func TestServiceManagerJoinReturn(t *testing.T) {
|
||||
want := "error 123"
|
||||
var sm ServiceManager
|
||||
sm.Go(func(*ServiceContext) error {
|
||||
return fmt.Errorf("error 123")
|
||||
})
|
||||
if got := sm.Join().Error(); got != want {
|
||||
t.Errorf("Join().Error() = %#v, want %#v", got, want)
|
||||
}
|
||||
}
|
||||
|
||||
func TestServiceManagerRace(t *testing.T) {
|
||||
var sm ServiceManager
|
||||
sm.Go(func(sc *ServiceContext) error {
|
||||
if sc.ShuttingDown == nil {
|
||||
t.Errorf("ShuttingDown: nil, want non-nil")
|
||||
}
|
||||
return nil
|
||||
})
|
||||
sm.Stop()
|
||||
}
|
|
@ -10,8 +10,9 @@ import (
|
|||
"strings"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/stats"
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
|
||||
|
@ -63,11 +64,13 @@ func getStatementCategory(sql string) binlogdatapb.BinlogTransaction_Statement_C
|
|||
// A Streamer should only be used once. To start another stream, call
|
||||
// NewStreamer() again.
|
||||
type Streamer struct {
|
||||
// dbname and mysqld are set at creation.
|
||||
dbname string
|
||||
mysqld mysqlctl.MysqlDaemon
|
||||
// dbname and mysqld are set at creation and immutable.
|
||||
dbname string
|
||||
mysqld mysqlctl.MysqlDaemon
|
||||
|
||||
clientCharset *binlogdatapb.Charset
|
||||
startPos replication.Position
|
||||
timestamp int64
|
||||
sendTransaction sendTransactionFunc
|
||||
|
||||
conn *mysqlctl.SlaveConnection
|
||||
|
@ -78,23 +81,25 @@ type Streamer struct {
|
|||
// dbname specifes the database to stream events for.
|
||||
// mysqld is the local instance of mysqlctl.Mysqld.
|
||||
// charset is the default character set on the BinlogPlayer side.
|
||||
// startPos is the position to start streaming at.
|
||||
// startPos is the position to start streaming at. Incompatible with timestamp.
|
||||
// timestamp is the timestamp to start streaming at. Incompatible with startPos.
|
||||
// sendTransaction is called each time a transaction is committed or rolled back.
|
||||
func NewStreamer(dbname string, mysqld mysqlctl.MysqlDaemon, clientCharset *binlogdatapb.Charset, startPos replication.Position, sendTransaction sendTransactionFunc) *Streamer {
|
||||
func NewStreamer(dbname string, mysqld mysqlctl.MysqlDaemon, clientCharset *binlogdatapb.Charset, startPos replication.Position, timestamp int64, sendTransaction sendTransactionFunc) *Streamer {
|
||||
return &Streamer{
|
||||
dbname: dbname,
|
||||
mysqld: mysqld,
|
||||
clientCharset: clientCharset,
|
||||
startPos: startPos,
|
||||
timestamp: timestamp,
|
||||
sendTransaction: sendTransaction,
|
||||
}
|
||||
}
|
||||
|
||||
// Stream starts streaming binlog events using the settings from NewStreamer().
|
||||
func (bls *Streamer) Stream(ctx *sync2.ServiceContext) (err error) {
|
||||
func (bls *Streamer) Stream(ctx context.Context) (err error) {
|
||||
stopPos := bls.startPos
|
||||
defer func() {
|
||||
if err != nil {
|
||||
if err != nil && err != mysqlctl.ErrBinlogUnavailable {
|
||||
err = fmt.Errorf("stream error @ %v: %v", stopPos, err)
|
||||
}
|
||||
log.Infof("stream ended @ %v, err = %v", stopPos, err)
|
||||
|
@ -124,7 +129,11 @@ func (bls *Streamer) Stream(ctx *sync2.ServiceContext) (err error) {
|
|||
}
|
||||
|
||||
var events <-chan replication.BinlogEvent
|
||||
events, err = bls.conn.StartBinlogDump(bls.startPos)
|
||||
if bls.timestamp != 0 {
|
||||
events, err = bls.conn.StartBinlogDumpFromTimestamp(ctx, bls.timestamp)
|
||||
} else {
|
||||
events, err = bls.conn.StartBinlogDumpFromPosition(ctx, bls.startPos)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -140,7 +149,8 @@ func (bls *Streamer) Stream(ctx *sync2.ServiceContext) (err error) {
|
|||
//
|
||||
// If the sendTransaction func returns io.EOF, parseEvents returns ErrClientEOF.
|
||||
// If the events channel is closed, parseEvents returns ErrServerEOF.
|
||||
func (bls *Streamer) parseEvents(ctx *sync2.ServiceContext, events <-chan replication.BinlogEvent) (replication.Position, error) {
|
||||
// If the context is done, returns ctx.Err().
|
||||
func (bls *Streamer) parseEvents(ctx context.Context, events <-chan replication.BinlogEvent) (replication.Position, error) {
|
||||
var statements []*binlogdatapb.BinlogTransaction_Statement
|
||||
var format replication.BinlogFormat
|
||||
var gtid replication.GTID
|
||||
|
@ -180,7 +190,7 @@ func (bls *Streamer) parseEvents(ctx *sync2.ServiceContext, events <-chan replic
|
|||
}
|
||||
|
||||
// Parse events.
|
||||
for ctx.IsRunning() {
|
||||
for {
|
||||
var ev replication.BinlogEvent
|
||||
var ok bool
|
||||
|
||||
|
@ -191,9 +201,9 @@ func (bls *Streamer) parseEvents(ctx *sync2.ServiceContext, events <-chan replic
|
|||
log.Infof("reached end of binlog event stream")
|
||||
return pos, ErrServerEOF
|
||||
}
|
||||
case <-ctx.ShuttingDown:
|
||||
log.Infof("stopping early due to binlog Streamer service shutdown")
|
||||
return pos, nil
|
||||
case <-ctx.Done():
|
||||
log.Infof("stopping early due to binlog Streamer service shutdown or client disconnect")
|
||||
return pos, ctx.Err()
|
||||
}
|
||||
|
||||
// Validate the buffer before reading fields from it.
|
||||
|
@ -317,6 +327,4 @@ func (bls *Streamer) parseEvents(ctx *sync2.ServiceContext, events <-chan replic
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
return pos, nil
|
||||
}
|
||||
|
|
|
@ -13,7 +13,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
|
||||
|
@ -197,15 +198,11 @@ func TestStreamerParseEventsXID(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -254,15 +251,11 @@ func TestStreamerParseEventsCommit(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -277,26 +270,24 @@ func TestStreamerStop(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
// Start parseEvents(), but don't send it anything, so it just waits.
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
if err != nil {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
|
||||
done := make(chan struct{})
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
done := make(chan error)
|
||||
go func() {
|
||||
svm.Stop()
|
||||
close(done)
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
done <- err
|
||||
}()
|
||||
|
||||
// close the context, expect the parser to return
|
||||
cancel()
|
||||
|
||||
select {
|
||||
case <-done:
|
||||
case err := <-done:
|
||||
if err != context.Canceled {
|
||||
t.Errorf("wrong context interruption returned value: %v", err)
|
||||
}
|
||||
case <-time.After(1 * time.Second):
|
||||
t.Errorf("timed out waiting for binlogConnStreamer.Stop()")
|
||||
}
|
||||
|
@ -321,18 +312,10 @@ func TestStreamerParseEventsClientEOF(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return io.EOF
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
}
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err != want {
|
||||
t.Errorf("wrong error, got %#v, want %#v", err, want)
|
||||
}
|
||||
|
@ -347,17 +330,9 @@ func TestStreamerParseEventsServerEOF(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
}
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err != want {
|
||||
t.Errorf("wrong error, got %#v, want %#v", err, want)
|
||||
}
|
||||
|
@ -382,15 +357,11 @@ func TestStreamerParseEventsSendErrorXID(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return fmt.Errorf("foobar")
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -421,15 +392,10 @@ func TestStreamerParseEventsSendErrorCommit(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return fmt.Errorf("foobar")
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -456,15 +422,10 @@ func TestStreamerParseEventsInvalid(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -493,15 +454,10 @@ func TestStreamerParseEventsInvalidFormat(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -530,15 +486,10 @@ func TestStreamerParseEventsNoFormat(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -565,15 +516,10 @@ func TestStreamerParseEventsInvalidQuery(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -646,15 +592,10 @@ func TestStreamerParseEventsRollback(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -711,15 +652,10 @@ func TestStreamerParseEventsDMLWithoutBegin(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -779,15 +715,10 @@ func TestStreamerParseEventsBeginWithoutCommit(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -836,15 +767,10 @@ func TestStreamerParseEventsSetInsertID(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -873,15 +799,10 @@ func TestStreamerParseEventsInvalidIntVar(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
err := svm.Join()
|
||||
_, err := bls.parseEvents(context.Background(), events)
|
||||
if err == nil {
|
||||
t.Errorf("expected error, got none")
|
||||
return
|
||||
|
@ -932,15 +853,10 @@ func TestStreamerParseEventsOtherDB(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -990,15 +906,10 @@ func TestStreamerParseEventsOtherDBBegin(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -1027,16 +938,11 @@ func TestStreamerParseEventsBeginAgain(t *testing.T) {
|
|||
sendTransaction := func(trans *binlogdatapb.BinlogTransaction) error {
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
before := binlogStreamerErrors.Counts()["ParseEvents"]
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
after := binlogStreamerErrors.Counts()["ParseEvents"]
|
||||
|
@ -1079,15 +985,10 @@ func TestStreamerParseEventsMariadbBeginGTID(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
@ -1129,15 +1030,10 @@ func TestStreamerParseEventsMariadbStandaloneGTID(t *testing.T) {
|
|||
got = append(got, *trans)
|
||||
return nil
|
||||
}
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, sendTransaction)
|
||||
bls := NewStreamer("vt_test_keyspace", nil, nil, replication.Position{}, 0, sendTransaction)
|
||||
|
||||
go sendTestEvents(events, input)
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(func(ctx *sync2.ServiceContext) error {
|
||||
_, err := bls.parseEvents(ctx, events)
|
||||
return err
|
||||
})
|
||||
if err := svm.Join(); err != ErrServerEOF {
|
||||
if _, err := bls.parseEvents(context.Background(), events); err != ErrServerEOF {
|
||||
t.Errorf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
|
|
|
@ -22,14 +22,6 @@ This file contains the API and registration mechanism for binlog player client.
|
|||
|
||||
var binlogPlayerProtocol = flag.String("binlog_player_protocol", "grpc", "the protocol to download binlogs from a vttablet")
|
||||
|
||||
// StreamEventStream is the interface of the object returned by
|
||||
// ServeUpdateStream
|
||||
type StreamEventStream interface {
|
||||
// Recv returns the next StreamEvent, or an error if the RPC was
|
||||
// interrupted.
|
||||
Recv() (*binlogdatapb.StreamEvent, error)
|
||||
}
|
||||
|
||||
// BinlogTransactionStream is the interface of the object returned by
|
||||
// StreamTables and StreamKeyRange
|
||||
type BinlogTransactionStream interface {
|
||||
|
@ -46,10 +38,6 @@ type Client interface {
|
|||
// Close the connection
|
||||
Close()
|
||||
|
||||
// Ask the server to stream binlog updates.
|
||||
// Should return context.Canceled if the context is canceled.
|
||||
ServeUpdateStream(ctx context.Context, position string) (StreamEventStream, error)
|
||||
|
||||
// Ask the server to stream updates related to the provided tables.
|
||||
// Should return context.Canceled if the context is canceled.
|
||||
StreamTables(ctx context.Context, position string, tables []string, charset *binlogdatapb.Charset) (BinlogTransactionStream, error)
|
||||
|
|
|
@ -13,7 +13,6 @@ import (
|
|||
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/sqltypes"
|
||||
"github.com/youtube/vitess/go/vt/binlog/binlogplayer"
|
||||
"github.com/youtube/vitess/go/vt/key"
|
||||
|
||||
|
@ -50,79 +49,6 @@ func NewFakeBinlogStreamer(t *testing.T) *FakeBinlogStreamer {
|
|||
}
|
||||
}
|
||||
|
||||
//
|
||||
// ServeUpdateStream tests
|
||||
//
|
||||
|
||||
var testUpdateStreamRequest = "UpdateStream starting position"
|
||||
|
||||
var testStreamEvent = &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_DML,
|
||||
TableName: "table1",
|
||||
PrimaryKeyFields: []*querypb.Field{
|
||||
{
|
||||
Name: "id",
|
||||
Type: sqltypes.Binary,
|
||||
},
|
||||
},
|
||||
PrimaryKeyValues: []*querypb.Row{
|
||||
{
|
||||
Lengths: []int64{3},
|
||||
Values: []byte{'1', '2', '3'},
|
||||
},
|
||||
},
|
||||
Sql: []byte("test sql with invalid utf-8 character \x80"),
|
||||
EventToken: &querypb.EventToken{
|
||||
Timestamp: 372,
|
||||
Position: "StreamEvent returned position",
|
||||
},
|
||||
}
|
||||
|
||||
// ServeUpdateStream is part of the the UpdateStream interface
|
||||
func (fake *FakeBinlogStreamer) ServeUpdateStream(position string, sendReply func(reply *binlogdatapb.StreamEvent) error) error {
|
||||
if fake.panics {
|
||||
panic(fmt.Errorf("test-triggered panic"))
|
||||
}
|
||||
if position != testUpdateStreamRequest {
|
||||
fake.t.Errorf("wrong ServeUpdateStream parameter, got %v want %v", position, testUpdateStreamRequest)
|
||||
}
|
||||
sendReply(testStreamEvent)
|
||||
return nil
|
||||
}
|
||||
|
||||
func testServeUpdateStream(t *testing.T, bpc binlogplayer.Client) {
|
||||
ctx := context.Background()
|
||||
stream, err := bpc.ServeUpdateStream(ctx, testUpdateStreamRequest)
|
||||
if err != nil {
|
||||
t.Fatalf("got error: %v", err)
|
||||
}
|
||||
if se, err := stream.Recv(); err != nil {
|
||||
t.Fatalf("got error: %v", err)
|
||||
} else {
|
||||
if !reflect.DeepEqual(*se, *testStreamEvent) {
|
||||
t.Errorf("got wrong result, got \n%#v expected \n%#v", *se, *testStreamEvent)
|
||||
}
|
||||
}
|
||||
if se, err := stream.Recv(); err == nil {
|
||||
t.Fatalf("got a response when error expected: %v", se)
|
||||
}
|
||||
}
|
||||
|
||||
func testServeUpdateStreamPanics(t *testing.T, bpc binlogplayer.Client) {
|
||||
ctx := context.Background()
|
||||
stream, err := bpc.ServeUpdateStream(ctx, testUpdateStreamRequest)
|
||||
if err != nil {
|
||||
t.Fatalf("got error: %v", err)
|
||||
}
|
||||
if se, err := stream.Recv(); err == nil {
|
||||
t.Fatalf("got a response when error expected: %v", se)
|
||||
} else {
|
||||
if !strings.Contains(err.Error(), "test-triggered panic") {
|
||||
t.Errorf("wrong error from panic: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//
|
||||
// StreamKeyRange tests
|
||||
//
|
||||
|
@ -159,7 +85,7 @@ var testBinlogTransaction = &binlogdatapb.BinlogTransaction{
|
|||
}
|
||||
|
||||
// StreamKeyRange is part of the the UpdateStream interface
|
||||
func (fake *FakeBinlogStreamer) StreamKeyRange(position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error {
|
||||
func (fake *FakeBinlogStreamer) StreamKeyRange(ctx context.Context, position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error {
|
||||
if fake.panics {
|
||||
panic(fmt.Errorf("test-triggered panic"))
|
||||
}
|
||||
|
@ -223,7 +149,7 @@ var testTablesRequest = &tablesRequest{
|
|||
}
|
||||
|
||||
// StreamTables is part of the the UpdateStream interface
|
||||
func (fake *FakeBinlogStreamer) StreamTables(position string, tables []string, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error {
|
||||
func (fake *FakeBinlogStreamer) StreamTables(ctx context.Context, position string, tables []string, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error {
|
||||
if fake.panics {
|
||||
panic(fmt.Errorf("test-triggered panic"))
|
||||
}
|
||||
|
@ -286,13 +212,11 @@ func Run(t *testing.T, bpc binlogplayer.Client, tablet *topodatapb.Tablet, fake
|
|||
}
|
||||
|
||||
// no panic
|
||||
testServeUpdateStream(t, bpc)
|
||||
testStreamKeyRange(t, bpc)
|
||||
testStreamTables(t, bpc)
|
||||
|
||||
// panic now, and test
|
||||
fake.panics = true
|
||||
testServeUpdateStreamPanics(t, bpc)
|
||||
testStreamKeyRangePanics(t, bpc)
|
||||
testStreamTablesPanics(t, bpc)
|
||||
fake.panics = false
|
||||
|
|
|
@ -11,8 +11,9 @@ import (
|
|||
"strings"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/sqltypes"
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/sqlparser"
|
||||
|
@ -28,7 +29,7 @@ var (
|
|||
streamCommentStartLen = len(streamCommentStart)
|
||||
)
|
||||
|
||||
type sendEventFunc func(event *binlogdatapb.StreamEvent) error
|
||||
type sendEventFunc func(event *querypb.StreamEvent) error
|
||||
|
||||
// EventStreamer is an adapter on top of a binlog Streamer that convert
|
||||
// the events into StreamEvent objects.
|
||||
|
@ -38,20 +39,23 @@ type EventStreamer struct {
|
|||
}
|
||||
|
||||
// NewEventStreamer returns a new EventStreamer on top of a Streamer
|
||||
func NewEventStreamer(dbname string, mysqld mysqlctl.MysqlDaemon, startPos replication.Position, sendEvent sendEventFunc) *EventStreamer {
|
||||
func NewEventStreamer(dbname string, mysqld mysqlctl.MysqlDaemon, startPos replication.Position, timestamp int64, sendEvent sendEventFunc) *EventStreamer {
|
||||
evs := &EventStreamer{
|
||||
sendEvent: sendEvent,
|
||||
}
|
||||
evs.bls = NewStreamer(dbname, mysqld, nil, startPos, evs.transactionToEvent)
|
||||
evs.bls = NewStreamer(dbname, mysqld, nil, startPos, timestamp, evs.transactionToEvent)
|
||||
return evs
|
||||
}
|
||||
|
||||
// Stream starts streaming updates
|
||||
func (evs *EventStreamer) Stream(ctx *sync2.ServiceContext) error {
|
||||
func (evs *EventStreamer) Stream(ctx context.Context) error {
|
||||
return evs.bls.Stream(ctx)
|
||||
}
|
||||
|
||||
func (evs *EventStreamer) transactionToEvent(trans *binlogdatapb.BinlogTransaction) error {
|
||||
event := &querypb.StreamEvent{
|
||||
EventToken: trans.EventToken,
|
||||
}
|
||||
var err error
|
||||
var insertid int64
|
||||
for _, stmt := range trans.Statements {
|
||||
|
@ -66,55 +70,42 @@ func (evs *EventStreamer) transactionToEvent(trans *binlogdatapb.BinlogTransacti
|
|||
}
|
||||
}
|
||||
case binlogdatapb.BinlogTransaction_Statement_BL_DML:
|
||||
var dmlEvent *binlogdatapb.StreamEvent
|
||||
dmlEvent, insertid, err = evs.buildDMLEvent(string(stmt.Sql), insertid)
|
||||
var dmlStatement *querypb.StreamEvent_Statement
|
||||
dmlStatement, insertid, err = evs.buildDMLStatement(string(stmt.Sql), insertid)
|
||||
if err != nil {
|
||||
dmlEvent = &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_ERR,
|
||||
dmlStatement = &querypb.StreamEvent_Statement{
|
||||
Category: querypb.StreamEvent_Statement_Error,
|
||||
Sql: stmt.Sql,
|
||||
}
|
||||
}
|
||||
if err = evs.sendEvent(dmlEvent); err != nil {
|
||||
return err
|
||||
}
|
||||
event.Statements = append(event.Statements, dmlStatement)
|
||||
case binlogdatapb.BinlogTransaction_Statement_BL_DDL:
|
||||
ddlEvent := &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_DDL,
|
||||
ddlStatement := &querypb.StreamEvent_Statement{
|
||||
Category: querypb.StreamEvent_Statement_DDL,
|
||||
Sql: stmt.Sql,
|
||||
}
|
||||
if err = evs.sendEvent(ddlEvent); err != nil {
|
||||
return err
|
||||
}
|
||||
event.Statements = append(event.Statements, ddlStatement)
|
||||
case binlogdatapb.BinlogTransaction_Statement_BL_UNRECOGNIZED:
|
||||
unrecognized := &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_ERR,
|
||||
unrecognized := &querypb.StreamEvent_Statement{
|
||||
Category: querypb.StreamEvent_Statement_Error,
|
||||
Sql: stmt.Sql,
|
||||
}
|
||||
if err = evs.sendEvent(unrecognized); err != nil {
|
||||
return err
|
||||
}
|
||||
event.Statements = append(event.Statements, unrecognized)
|
||||
default:
|
||||
binlogStreamerErrors.Add("EventStreamer", 1)
|
||||
log.Errorf("Unrecognized event: %v: %s", stmt.Category, stmt.Sql)
|
||||
}
|
||||
}
|
||||
posEvent := &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_POS,
|
||||
EventToken: trans.EventToken,
|
||||
}
|
||||
if err = evs.sendEvent(posEvent); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
return evs.sendEvent(event)
|
||||
}
|
||||
|
||||
/*
|
||||
buildDMLEvent parses the tuples of the full stream comment.
|
||||
The _stream comment is extracted into a StreamEvent.
|
||||
buildDMLStatement parses the tuples of the full stream comment.
|
||||
The _stream comment is extracted into a StreamEvent.Statement.
|
||||
*/
|
||||
// Example query: insert into _table_(foo) values ('foo') /* _stream _table_ (eid id name ) (null 1 'bmFtZQ==' ); */
|
||||
// the "null" value is used for auto-increment columns.
|
||||
func (evs *EventStreamer) buildDMLEvent(sql string, insertid int64) (*binlogdatapb.StreamEvent, int64, error) {
|
||||
func (evs *EventStreamer) buildDMLStatement(sql string, insertid int64) (*querypb.StreamEvent_Statement, int64, error) {
|
||||
// first extract the comment
|
||||
commentIndex := strings.LastIndex(sql, streamCommentStart)
|
||||
if commentIndex == -1 {
|
||||
|
@ -123,8 +114,8 @@ func (evs *EventStreamer) buildDMLEvent(sql string, insertid int64) (*binlogdata
|
|||
dmlComment := sql[commentIndex+streamCommentStartLen:]
|
||||
|
||||
// then strat building the response
|
||||
dmlEvent := &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_DML,
|
||||
dmlStatement := &querypb.StreamEvent_Statement{
|
||||
Category: querypb.StreamEvent_Statement_DML,
|
||||
}
|
||||
tokenizer := sqlparser.NewStringTokenizer(dmlComment)
|
||||
|
||||
|
@ -133,12 +124,12 @@ func (evs *EventStreamer) buildDMLEvent(sql string, insertid int64) (*binlogdata
|
|||
if typ != sqlparser.ID {
|
||||
return nil, insertid, fmt.Errorf("expecting table name in stream comment")
|
||||
}
|
||||
dmlEvent.TableName = string(val)
|
||||
dmlStatement.TableName = string(val)
|
||||
|
||||
// then parse the PK names
|
||||
var err error
|
||||
dmlEvent.PrimaryKeyFields, err = parsePkNames(tokenizer)
|
||||
hasNegatives := make([]bool, len(dmlEvent.PrimaryKeyFields))
|
||||
dmlStatement.PrimaryKeyFields, err = parsePkNames(tokenizer)
|
||||
hasNegatives := make([]bool, len(dmlStatement.PrimaryKeyFields))
|
||||
if err != nil {
|
||||
return nil, insertid, err
|
||||
}
|
||||
|
@ -149,17 +140,17 @@ func (evs *EventStreamer) buildDMLEvent(sql string, insertid int64) (*binlogdata
|
|||
case '(':
|
||||
// pkTuple is a list of pk values
|
||||
var pkTuple *querypb.Row
|
||||
pkTuple, insertid, err = parsePkTuple(tokenizer, insertid, dmlEvent.PrimaryKeyFields, hasNegatives)
|
||||
pkTuple, insertid, err = parsePkTuple(tokenizer, insertid, dmlStatement.PrimaryKeyFields, hasNegatives)
|
||||
if err != nil {
|
||||
return nil, insertid, err
|
||||
}
|
||||
dmlEvent.PrimaryKeyValues = append(dmlEvent.PrimaryKeyValues, pkTuple)
|
||||
dmlStatement.PrimaryKeyValues = append(dmlStatement.PrimaryKeyValues, pkTuple)
|
||||
default:
|
||||
return nil, insertid, fmt.Errorf("expecting '('")
|
||||
}
|
||||
}
|
||||
|
||||
return dmlEvent, insertid, nil
|
||||
return dmlStatement, insertid, nil
|
||||
}
|
||||
|
||||
// parsePkNames parses something like (eid id name )
|
||||
|
|
|
@ -26,12 +26,10 @@ var dmlErrorCases = []string{
|
|||
}
|
||||
|
||||
func TestEventErrors(t *testing.T) {
|
||||
var got *binlogdatapb.StreamEvent
|
||||
var got *querypb.StreamEvent
|
||||
evs := &EventStreamer{
|
||||
sendEvent: func(event *binlogdatapb.StreamEvent) error {
|
||||
if event.Category != binlogdatapb.StreamEvent_SE_POS {
|
||||
got = event
|
||||
}
|
||||
sendEvent: func(event *querypb.StreamEvent) error {
|
||||
got = event
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
@ -49,9 +47,13 @@ func TestEventErrors(t *testing.T) {
|
|||
t.Errorf("%s: %v", sql, err)
|
||||
continue
|
||||
}
|
||||
want := &binlogdatapb.StreamEvent{
|
||||
Category: binlogdatapb.StreamEvent_SE_ERR,
|
||||
Sql: []byte(sql),
|
||||
want := &querypb.StreamEvent{
|
||||
Statements: []*querypb.StreamEvent_Statement{
|
||||
{
|
||||
Category: querypb.StreamEvent_Statement_Error,
|
||||
Sql: []byte(sql),
|
||||
},
|
||||
},
|
||||
}
|
||||
if !reflect.DeepEqual(got, want) {
|
||||
t.Errorf("error for SQL: '%v' got: %+v, want: %+v", sql, got, want)
|
||||
|
@ -61,7 +63,7 @@ func TestEventErrors(t *testing.T) {
|
|||
|
||||
func TestSetErrors(t *testing.T) {
|
||||
evs := &EventStreamer{
|
||||
sendEvent: func(event *binlogdatapb.StreamEvent) error {
|
||||
sendEvent: func(event *querypb.StreamEvent) error {
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
@ -105,28 +107,30 @@ func TestDMLEvent(t *testing.T) {
|
|||
},
|
||||
}
|
||||
evs := &EventStreamer{
|
||||
sendEvent: func(event *binlogdatapb.StreamEvent) error {
|
||||
switch event.Category {
|
||||
case binlogdatapb.StreamEvent_SE_DML:
|
||||
want := `category:SE_DML table_name:"_table_" primary_key_fields:<name:"eid" type:INT64 > primary_key_fields:<name:"id" type:UINT64 > primary_key_fields:<name:"name" type:VARBINARY > primary_key_values:<lengths:2 lengths:1 lengths:4 values:"101name" > primary_key_values:<lengths:2 lengths:20 lengths:4 values:"1118446744073709551615name" > `
|
||||
got := fmt.Sprintf("%v", event)
|
||||
if got != want {
|
||||
t.Errorf("got \n%s, want \n%s", got, want)
|
||||
sendEvent: func(event *querypb.StreamEvent) error {
|
||||
for _, statement := range event.Statements {
|
||||
switch statement.Category {
|
||||
case querypb.StreamEvent_Statement_DML:
|
||||
want := `category:DML table_name:"_table_" primary_key_fields:<name:"eid" type:INT64 > primary_key_fields:<name:"id" type:UINT64 > primary_key_fields:<name:"name" type:VARBINARY > primary_key_values:<lengths:2 lengths:1 lengths:4 values:"101name" > primary_key_values:<lengths:2 lengths:20 lengths:4 values:"1118446744073709551615name" > `
|
||||
got := fmt.Sprintf("%v", statement)
|
||||
if got != want {
|
||||
t.Errorf("got \n%s, want \n%s", got, want)
|
||||
}
|
||||
case querypb.StreamEvent_Statement_Error:
|
||||
want := `sql:"query" `
|
||||
got := fmt.Sprintf("%v", statement)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
default:
|
||||
t.Errorf("unexpected: %#v", event)
|
||||
}
|
||||
case binlogdatapb.StreamEvent_SE_ERR:
|
||||
want := `sql:"query" `
|
||||
got := fmt.Sprintf("%v", event)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
case binlogdatapb.StreamEvent_SE_POS:
|
||||
want := `category:SE_POS event_token:<timestamp:1 position:"MariaDB/0-41983-20" > `
|
||||
got := fmt.Sprintf("%v", event)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
default:
|
||||
t.Errorf("unexpected: %#v", event)
|
||||
}
|
||||
// then test the position
|
||||
want := `timestamp:1 position:"MariaDB/0-41983-20" `
|
||||
got := fmt.Sprintf("%v", event.EventToken)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
return nil
|
||||
},
|
||||
|
@ -154,22 +158,24 @@ func TestDDLEvent(t *testing.T) {
|
|||
},
|
||||
}
|
||||
evs := &EventStreamer{
|
||||
sendEvent: func(event *binlogdatapb.StreamEvent) error {
|
||||
switch event.Category {
|
||||
case binlogdatapb.StreamEvent_SE_DDL:
|
||||
want := `category:SE_DDL sql:"DDL" `
|
||||
got := fmt.Sprintf("%v", event)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
sendEvent: func(event *querypb.StreamEvent) error {
|
||||
for _, statement := range event.Statements {
|
||||
switch statement.Category {
|
||||
case querypb.StreamEvent_Statement_DDL:
|
||||
want := `category:DDL sql:"DDL" `
|
||||
got := fmt.Sprintf("%v", statement)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
default:
|
||||
t.Errorf("unexpected: %#v", event)
|
||||
}
|
||||
case binlogdatapb.StreamEvent_SE_POS:
|
||||
want := `category:SE_POS event_token:<timestamp:1 position:"MariaDB/0-41983-20" > `
|
||||
got := fmt.Sprintf("%v", event)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
default:
|
||||
t.Errorf("unexpected: %#v", event)
|
||||
}
|
||||
// then test the position
|
||||
want := `timestamp:1 position:"MariaDB/0-41983-20" `
|
||||
got := fmt.Sprintf("%v", event.EventToken)
|
||||
if got != want {
|
||||
t.Errorf("got %s, want %s", got, want)
|
||||
}
|
||||
return nil
|
||||
},
|
||||
|
|
|
@ -40,29 +40,6 @@ func (client *client) Close() {
|
|||
client.cc.Close()
|
||||
}
|
||||
|
||||
type serveUpdateStreamAdapter struct {
|
||||
stream binlogservicepb.UpdateStream_StreamUpdateClient
|
||||
}
|
||||
|
||||
func (s *serveUpdateStreamAdapter) Recv() (*binlogdatapb.StreamEvent, error) {
|
||||
r, err := s.stream.Recv()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return r.StreamEvent, nil
|
||||
}
|
||||
|
||||
func (client *client) ServeUpdateStream(ctx context.Context, position string) (binlogplayer.StreamEventStream, error) {
|
||||
query := &binlogdatapb.StreamUpdateRequest{
|
||||
Position: position,
|
||||
}
|
||||
stream, err := client.c.StreamUpdate(ctx, query)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &serveUpdateStreamAdapter{stream}, nil
|
||||
}
|
||||
|
||||
type serveStreamKeyRangeAdapter struct {
|
||||
stream binlogservicepb.UpdateStream_StreamKeyRangeClient
|
||||
}
|
||||
|
|
|
@ -24,20 +24,10 @@ func New(updateStream binlog.UpdateStream) *UpdateStream {
|
|||
return &UpdateStream{updateStream}
|
||||
}
|
||||
|
||||
// StreamUpdate is part of the binlogservicepb.UpdateStreamServer interface
|
||||
func (server *UpdateStream) StreamUpdate(req *binlogdatapb.StreamUpdateRequest, stream binlogservicepb.UpdateStream_StreamUpdateServer) (err error) {
|
||||
defer server.updateStream.HandlePanic(&err)
|
||||
return server.updateStream.ServeUpdateStream(req.Position, func(reply *binlogdatapb.StreamEvent) error {
|
||||
return stream.Send(&binlogdatapb.StreamUpdateResponse{
|
||||
StreamEvent: reply,
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
// StreamKeyRange is part of the binlogservicepb.UpdateStreamServer interface
|
||||
func (server *UpdateStream) StreamKeyRange(req *binlogdatapb.StreamKeyRangeRequest, stream binlogservicepb.UpdateStream_StreamKeyRangeServer) (err error) {
|
||||
defer server.updateStream.HandlePanic(&err)
|
||||
return server.updateStream.StreamKeyRange(req.Position, req.KeyRange, req.Charset, func(reply *binlogdatapb.BinlogTransaction) error {
|
||||
return server.updateStream.StreamKeyRange(stream.Context(), req.Position, req.KeyRange, req.Charset, func(reply *binlogdatapb.BinlogTransaction) error {
|
||||
return stream.Send(&binlogdatapb.StreamKeyRangeResponse{
|
||||
BinlogTransaction: reply,
|
||||
})
|
||||
|
@ -47,7 +37,7 @@ func (server *UpdateStream) StreamKeyRange(req *binlogdatapb.StreamKeyRangeReque
|
|||
// StreamTables is part of the binlogservicepb.UpdateStreamServer interface
|
||||
func (server *UpdateStream) StreamTables(req *binlogdatapb.StreamTablesRequest, stream binlogservicepb.UpdateStream_StreamTablesServer) (err error) {
|
||||
defer server.updateStream.HandlePanic(&err)
|
||||
return server.updateStream.StreamTables(req.Position, req.Tables, req.Charset, func(reply *binlogdatapb.BinlogTransaction) error {
|
||||
return server.updateStream.StreamTables(stream.Context(), req.Position, req.Tables, req.Charset, func(reply *binlogdatapb.BinlogTransaction) error {
|
||||
return stream.Send(&binlogdatapb.StreamTablesResponse{
|
||||
BinlogTransaction: reply,
|
||||
})
|
||||
|
|
|
@ -5,21 +5,19 @@
|
|||
package binlog
|
||||
|
||||
import (
|
||||
"golang.org/x/net/context"
|
||||
|
||||
binlogdatapb "github.com/youtube/vitess/go/vt/proto/binlogdata"
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
// UpdateStream is the interface for the binlog server
|
||||
type UpdateStream interface {
|
||||
// ServeUpdateStream serves the query and streams the result
|
||||
// for the full update stream
|
||||
ServeUpdateStream(position string, sendReply func(reply *binlogdatapb.StreamEvent) error) error
|
||||
|
||||
// StreamKeyRange streams events related to a KeyRange only
|
||||
StreamKeyRange(position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error
|
||||
StreamKeyRange(ctx context.Context, position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(*binlogdatapb.BinlogTransaction) error) error
|
||||
|
||||
// StreamTables streams events related to a set of Tables only
|
||||
StreamTables(position string, tables []string, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) error
|
||||
StreamTables(ctx context.Context, position string, tables []string, charset *binlogdatapb.Charset, sendReply func(*binlogdatapb.BinlogTransaction) error) error
|
||||
|
||||
// HandlePanic should be called in a defer,
|
||||
// first thing in the RPC implementation.
|
||||
|
|
|
@ -9,6 +9,8 @@ import (
|
|||
"sync"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/stats"
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"github.com/youtube/vitess/go/tb"
|
||||
|
@ -34,7 +36,6 @@ var usStateNames = map[int64]string{
|
|||
var (
|
||||
streamCount = stats.NewCounters("UpdateStreamStreamCount")
|
||||
updateStreamErrors = stats.NewCounters("UpdateStreamErrors")
|
||||
updateStreamEvents = stats.NewCounters("UpdateStreamEvents")
|
||||
keyrangeStatements = stats.NewInt("UpdateStreamKeyRangeStatements")
|
||||
keyrangeTransactions = stats.NewInt("UpdateStreamKeyRangeTransactions")
|
||||
tablesStatements = stats.NewInt("UpdateStreamTablesStatements")
|
||||
|
@ -99,36 +100,47 @@ type UpdateStreamImpl struct {
|
|||
actionLock sync.Mutex
|
||||
state sync2.AtomicInt64
|
||||
stateWaitGroup sync.WaitGroup
|
||||
streams streamList
|
||||
streams StreamList
|
||||
}
|
||||
|
||||
type streamList struct {
|
||||
// StreamList is a map of context.CancelFunc to mass-interrupt ongoing
|
||||
// calls.
|
||||
type StreamList struct {
|
||||
sync.Mutex
|
||||
streams map[*sync2.ServiceManager]bool
|
||||
currentIndex int
|
||||
streams map[int]context.CancelFunc
|
||||
}
|
||||
|
||||
func (sl *streamList) Init() {
|
||||
// Init must be called before using the list.
|
||||
func (sl *StreamList) Init() {
|
||||
sl.Lock()
|
||||
sl.streams = make(map[*sync2.ServiceManager]bool)
|
||||
sl.streams = make(map[int]context.CancelFunc)
|
||||
sl.currentIndex = 0
|
||||
sl.Unlock()
|
||||
}
|
||||
|
||||
func (sl *streamList) Add(e *sync2.ServiceManager) {
|
||||
// Add adds a CancelFunc to the map.
|
||||
func (sl *StreamList) Add(c context.CancelFunc) int {
|
||||
sl.Lock()
|
||||
sl.streams[e] = true
|
||||
defer sl.Unlock()
|
||||
|
||||
sl.currentIndex++
|
||||
sl.streams[sl.currentIndex] = c
|
||||
return sl.currentIndex
|
||||
}
|
||||
|
||||
// Delete removes a CancelFunc from the list.
|
||||
func (sl *StreamList) Delete(i int) {
|
||||
sl.Lock()
|
||||
delete(sl.streams, i)
|
||||
sl.Unlock()
|
||||
}
|
||||
|
||||
func (sl *streamList) Delete(e *sync2.ServiceManager) {
|
||||
// Stop stops all the current streams.
|
||||
func (sl *StreamList) Stop() {
|
||||
sl.Lock()
|
||||
delete(sl.streams, e)
|
||||
sl.Unlock()
|
||||
}
|
||||
|
||||
func (sl *streamList) Stop() {
|
||||
sl.Lock()
|
||||
for stream := range sl.streams {
|
||||
stream.Stop()
|
||||
for _, c := range sl.streams {
|
||||
c()
|
||||
}
|
||||
sl.Unlock()
|
||||
}
|
||||
|
@ -203,45 +215,8 @@ func (updateStream *UpdateStreamImpl) IsEnabled() bool {
|
|||
return updateStream.state.Get() == usEnabled
|
||||
}
|
||||
|
||||
// ServeUpdateStream is part of the UpdateStream interface
|
||||
func (updateStream *UpdateStreamImpl) ServeUpdateStream(position string, sendReply func(reply *binlogdatapb.StreamEvent) error) (err error) {
|
||||
pos, err := replication.DecodePosition(position)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
updateStream.actionLock.Lock()
|
||||
if !updateStream.IsEnabled() {
|
||||
updateStream.actionLock.Unlock()
|
||||
log.Errorf("Unable to serve client request: update stream service is not enabled")
|
||||
return fmt.Errorf("update stream service is not enabled")
|
||||
}
|
||||
updateStream.stateWaitGroup.Add(1)
|
||||
updateStream.actionLock.Unlock()
|
||||
defer updateStream.stateWaitGroup.Done()
|
||||
|
||||
streamCount.Add("Updates", 1)
|
||||
defer streamCount.Add("Updates", -1)
|
||||
log.Infof("ServeUpdateStream starting @ %#v", pos)
|
||||
|
||||
evs := NewEventStreamer(updateStream.dbname, updateStream.mysqld, pos, func(reply *binlogdatapb.StreamEvent) error {
|
||||
if reply.Category == binlogdatapb.StreamEvent_SE_ERR {
|
||||
updateStreamErrors.Add("UpdateStream", 1)
|
||||
} else {
|
||||
updateStreamEvents.Add(reply.Category.String(), 1)
|
||||
}
|
||||
return sendReply(reply)
|
||||
})
|
||||
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(evs.Stream)
|
||||
updateStream.streams.Add(svm)
|
||||
defer updateStream.streams.Delete(svm)
|
||||
return svm.Join()
|
||||
}
|
||||
|
||||
// StreamKeyRange is part of the UpdateStream interface
|
||||
func (updateStream *UpdateStreamImpl) StreamKeyRange(position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) (err error) {
|
||||
func (updateStream *UpdateStreamImpl) StreamKeyRange(ctx context.Context, position string, keyRange *topodatapb.KeyRange, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) (err error) {
|
||||
pos, err := replication.DecodePosition(position)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -267,17 +242,17 @@ func (updateStream *UpdateStreamImpl) StreamKeyRange(position string, keyRange *
|
|||
keyrangeTransactions.Add(1)
|
||||
return sendReply(reply)
|
||||
})
|
||||
bls := NewStreamer(updateStream.dbname, updateStream.mysqld, charset, pos, f)
|
||||
bls := NewStreamer(updateStream.dbname, updateStream.mysqld, charset, pos, 0, f)
|
||||
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(bls.Stream)
|
||||
updateStream.streams.Add(svm)
|
||||
defer updateStream.streams.Delete(svm)
|
||||
return svm.Join()
|
||||
streamCtx, cancel := context.WithCancel(ctx)
|
||||
i := updateStream.streams.Add(cancel)
|
||||
defer updateStream.streams.Delete(i)
|
||||
|
||||
return bls.Stream(streamCtx)
|
||||
}
|
||||
|
||||
// StreamTables is part of the UpdateStream interface
|
||||
func (updateStream *UpdateStreamImpl) StreamTables(position string, tables []string, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) (err error) {
|
||||
func (updateStream *UpdateStreamImpl) StreamTables(ctx context.Context, position string, tables []string, charset *binlogdatapb.Charset, sendReply func(reply *binlogdatapb.BinlogTransaction) error) (err error) {
|
||||
pos, err := replication.DecodePosition(position)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -303,13 +278,13 @@ func (updateStream *UpdateStreamImpl) StreamTables(position string, tables []str
|
|||
tablesTransactions.Add(1)
|
||||
return sendReply(reply)
|
||||
})
|
||||
bls := NewStreamer(updateStream.dbname, updateStream.mysqld, charset, pos, f)
|
||||
bls := NewStreamer(updateStream.dbname, updateStream.mysqld, charset, pos, 0, f)
|
||||
|
||||
svm := &sync2.ServiceManager{}
|
||||
svm.Go(bls.Stream)
|
||||
updateStream.streams.Add(svm)
|
||||
defer updateStream.streams.Delete(svm)
|
||||
return svm.Join()
|
||||
streamCtx, cancel := context.WithCancel(ctx)
|
||||
i := updateStream.streams.Add(cancel)
|
||||
defer updateStream.streams.Delete(i)
|
||||
|
||||
return bls.Stream(streamCtx)
|
||||
}
|
||||
|
||||
// HandlePanic is part of the UpdateStream interface
|
||||
|
|
|
@ -460,6 +460,11 @@ func (fc *fakeConn) SplitQueryV2(
|
|||
return nil, fmt.Errorf("not implemented")
|
||||
}
|
||||
|
||||
// UpdateStream implements tabletconn.TabletConn.
|
||||
func (fc *fakeConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
return nil, fmt.Errorf("not implemented")
|
||||
}
|
||||
|
||||
// Tablet returns the tablet associated with the connection.
|
||||
func (fc *fakeConn) Tablet() *topodatapb.Tablet {
|
||||
return fc.tablet
|
||||
|
|
|
@ -156,22 +156,14 @@ func (*mariaDB10) ParseReplicationPosition(s string) (replication.Position, erro
|
|||
|
||||
// SendBinlogDumpCommand implements MysqlFlavor.SendBinlogDumpCommand().
|
||||
func (*mariaDB10) SendBinlogDumpCommand(conn *SlaveConnection, startPos replication.Position) error {
|
||||
const ComBinlogDump = 0x12
|
||||
|
||||
// Tell the server that we understand GTIDs by setting our slave capability
|
||||
// to MARIA_SLAVE_CAPABILITY_GTID = 4 (MariaDB >= 10.0.1).
|
||||
if _, err := conn.ExecuteFetch("SET @mariadb_slave_capability=4", 0, false); err != nil {
|
||||
return fmt.Errorf("failed to set @mariadb_slave_capability=4: %v", err)
|
||||
}
|
||||
|
||||
// Tell the server that we understand the format of events that will be used
|
||||
// if binlog_checksum is enabled on the server.
|
||||
if _, err := conn.ExecuteFetch("SET @master_binlog_checksum=@@global.binlog_checksum", 0, false); err != nil {
|
||||
return fmt.Errorf("failed to set @master_binlog_checksum=@@global.binlog_checksum: %v", err)
|
||||
}
|
||||
|
||||
// Set the slave_connect_state variable before issuing COM_BINLOG_DUMP to
|
||||
// provide the start position in GTID form.
|
||||
// Set the slave_connect_state variable before issuing COM_BINLOG_DUMP
|
||||
// to provide the start position in GTID form.
|
||||
query := fmt.Sprintf("SET @slave_connect_state='%s'", startPos)
|
||||
if _, err := conn.ExecuteFetch(query, 0, false); err != nil {
|
||||
return fmt.Errorf("failed to set @slave_connect_state='%s': %v", startPos, err)
|
||||
|
|
|
@ -153,12 +153,6 @@ func (flavor *mysql56) SendBinlogDumpCommand(conn *SlaveConnection, startPos rep
|
|||
return fmt.Errorf("startPos.GTIDSet is wrong type - expected Mysql56GTIDSet, got: %#v", startPos.GTIDSet)
|
||||
}
|
||||
|
||||
// Tell the server that we understand the format of events that will be used
|
||||
// if binlog_checksum is enabled on the server.
|
||||
if _, err := conn.ExecuteFetch("SET @master_binlog_checksum=@@global.binlog_checksum", 0, false); err != nil {
|
||||
return fmt.Errorf("failed to set @master_binlog_checksum=@@global.binlog_checksum: %v", err)
|
||||
}
|
||||
|
||||
// Build the command.
|
||||
buf := makeBinlogDumpGTIDCommand(0, conn.slaveID, gtidSet)
|
||||
return conn.SendCommand(ComBinlogDumpGTID, buf)
|
||||
|
|
|
@ -8,16 +8,24 @@ import (
|
|||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/mysql"
|
||||
"github.com/youtube/vitess/go/pools"
|
||||
"github.com/youtube/vitess/go/sqldb"
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"github.com/youtube/vitess/go/vt/dbconfigs"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrBinlogUnavailable is returned by this library when we
|
||||
// cannot find a suitable binlog to satisfy the request.
|
||||
ErrBinlogUnavailable = fmt.Errorf("cannot find relevant binlogs on this server")
|
||||
)
|
||||
|
||||
// SlaveConnection represents a connection to mysqld that pretends to be a slave
|
||||
// connecting for replication. Each such connection must identify itself to
|
||||
// mysqld with a server ID that is unique both among other SlaveConnections and
|
||||
|
@ -26,7 +34,8 @@ type SlaveConnection struct {
|
|||
sqldb.Conn
|
||||
mysqld *Mysqld
|
||||
slaveID uint32
|
||||
svm sync2.ServiceManager
|
||||
cancel context.CancelFunc
|
||||
wg sync.WaitGroup
|
||||
}
|
||||
|
||||
// NewSlaveConnection creates a new slave connection to the mysqld instance.
|
||||
|
@ -37,12 +46,7 @@ type SlaveConnection struct {
|
|||
// 2) No real slave servers will have IDs in the range 1-N where N is the peak
|
||||
// number of concurrent fake slave connections we will ever make.
|
||||
func (mysqld *Mysqld) NewSlaveConnection() (*SlaveConnection, error) {
|
||||
params, err := dbconfigs.MysqlParams(mysqld.dba)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
conn, err := sqldb.Connect(params)
|
||||
conn, err := mysqld.connectForReplication()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -56,15 +60,41 @@ func (mysqld *Mysqld) NewSlaveConnection() (*SlaveConnection, error) {
|
|||
return sc, nil
|
||||
}
|
||||
|
||||
// connectForReplication create a MySQL connection ready to use for replication.
|
||||
func (mysqld *Mysqld) connectForReplication() (sqldb.Conn, error) {
|
||||
params, err := dbconfigs.MysqlParams(mysqld.dba)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
conn, err := sqldb.Connect(params)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Tell the server that we understand the format of events
|
||||
// that will be used if binlog_checksum is enabled on the server.
|
||||
if _, err := conn.ExecuteFetch("SET @master_binlog_checksum=@@global.binlog_checksum", 0, false); err != nil {
|
||||
return nil, fmt.Errorf("failed to set @master_binlog_checksum=@@global.binlog_checksum: %v", err)
|
||||
}
|
||||
|
||||
return conn, nil
|
||||
}
|
||||
|
||||
// slaveIDPool is the IDPool for server IDs used to connect as a slave.
|
||||
var slaveIDPool = pools.NewIDPool()
|
||||
|
||||
// StartBinlogDump requests a replication binlog dump from the master mysqld
|
||||
// and then immediately returns a channel on which received binlog events will
|
||||
// be sent. The stream will continue, waiting for new events if necessary,
|
||||
// until the connection is closed, either by the master or by calling
|
||||
// SlaveConnection.Close(). At that point, the channel will also be closed.
|
||||
func (sc *SlaveConnection) StartBinlogDump(startPos replication.Position) (<-chan replication.BinlogEvent, error) {
|
||||
// StartBinlogDumpFromPosition requests a replication binlog dump from
|
||||
// the master mysqld at the given Position and then sends binlog
|
||||
// events to the provided channel.
|
||||
// The stream will continue in the background, waiting for new events if
|
||||
// necessary, until the connection is closed, either by the master or
|
||||
// by canceling the context.
|
||||
//
|
||||
// Note the context is valid and used until eventChan is closed.
|
||||
func (sc *SlaveConnection) StartBinlogDumpFromPosition(ctx context.Context, startPos replication.Position) (<-chan replication.BinlogEvent, error) {
|
||||
ctx, sc.cancel = context.WithCancel(ctx)
|
||||
|
||||
flavor, err := sc.mysqld.flavor()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("StartBinlogDump needs flavor: %v", err)
|
||||
|
@ -83,24 +113,28 @@ func (sc *SlaveConnection) StartBinlogDump(startPos replication.Position) (<-cha
|
|||
return nil, err
|
||||
}
|
||||
|
||||
// FIXME(alainjobart) I think we can use a buffered channel for better performance.
|
||||
eventChan := make(chan replication.BinlogEvent)
|
||||
|
||||
// Start reading events.
|
||||
sc.svm.Go(func(svc *sync2.ServiceContext) error {
|
||||
defer close(eventChan)
|
||||
|
||||
for svc.IsRunning() {
|
||||
sc.wg.Add(1)
|
||||
go func() {
|
||||
defer func() {
|
||||
close(eventChan)
|
||||
sc.wg.Done()
|
||||
}()
|
||||
for {
|
||||
if buf[0] == 254 {
|
||||
// The master is telling us to stop.
|
||||
log.Infof("received EOF packet in binlog dump: %#v", buf)
|
||||
return nil
|
||||
return
|
||||
}
|
||||
|
||||
select {
|
||||
// Skip the first byte because it's only used for signaling EOF.
|
||||
case eventChan <- flavor.MakeBinlogEvent(buf[1:]):
|
||||
case <-svc.ShuttingDown:
|
||||
return nil
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
|
||||
buf, err = sc.Conn.ReadPacket()
|
||||
|
@ -110,14 +144,189 @@ func (sc *SlaveConnection) StartBinlogDump(startPos replication.Position) (<-cha
|
|||
// This is not necessarily an error. It could just be that we closed
|
||||
// the connection from outside.
|
||||
log.Infof("connection closed during binlog stream (possibly intentional): %v", err)
|
||||
return err
|
||||
return
|
||||
}
|
||||
log.Errorf("read error while streaming binlog events: %v", err)
|
||||
return err
|
||||
return
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}()
|
||||
|
||||
return eventChan, nil
|
||||
}
|
||||
|
||||
// StartBinlogDumpFromTimestamp requests a replication binlog dump from
|
||||
// the master mysqld at the given timestamp and then sends binlog
|
||||
// events to the provided channel.
|
||||
//
|
||||
// The startup phase will list all the binary logs, and find the one
|
||||
// that has events starting strictly before the provided timestamp. It
|
||||
// will then start from there, and skip all events that are before the
|
||||
// provided timestamp.
|
||||
//
|
||||
// The stream will continue in the background, waiting for new events if
|
||||
// necessary, until the connection is closed, either by the master or
|
||||
// by canceling the context.
|
||||
//
|
||||
// Note the context is valid and used until eventChan is closed.
|
||||
func (sc *SlaveConnection) StartBinlogDumpFromTimestamp(ctx context.Context, timestamp int64) (<-chan replication.BinlogEvent, error) {
|
||||
ctx, sc.cancel = context.WithCancel(ctx)
|
||||
|
||||
flavor, err := sc.mysqld.flavor()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("StartBinlogDump needs flavor: %v", err)
|
||||
}
|
||||
|
||||
// List the binlogs.
|
||||
binlogs, err := sc.Conn.ExecuteFetch("SHOW BINARY LOGS", 1000, false)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to SHOW BINARY LOGS: %v", err)
|
||||
}
|
||||
|
||||
// Start with the most recent binlog file until we find the right event.
|
||||
var binlogIndex int
|
||||
var firstEvent replication.BinlogEvent
|
||||
for binlogIndex = len(binlogs.Rows) - 1; binlogIndex >= 0; binlogIndex-- {
|
||||
// Exit the loop early if context is canceled.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
default:
|
||||
}
|
||||
|
||||
// Start dumping the logs. The position is '4' to skip the
|
||||
// Binlog File Header. See this page for more info:
|
||||
// https://dev.mysql.com/doc/internals/en/binlog-file.html
|
||||
binlog := binlogs.Rows[binlogIndex][0].String()
|
||||
cmd := makeBinlogDumpCommand(4, 0, sc.slaveID, binlog)
|
||||
if err := sc.Conn.SendCommand(ComBinlogDump, cmd); err != nil {
|
||||
return nil, fmt.Errorf("failed to send the ComBinlogDump command: %v", err)
|
||||
}
|
||||
|
||||
// Get the first event to get its timestamp. We skip ROTATE
|
||||
// events, as they don't have timestamps.
|
||||
for {
|
||||
buf, err := sc.Conn.ReadPacket()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("couldn't start binlog dump of binlog %v: %v", binlog, err)
|
||||
}
|
||||
|
||||
// Why would the master tell us to stop here?
|
||||
if buf[0] == 254 {
|
||||
return nil, fmt.Errorf("received EOF packet for first packet of binlog %v", binlog)
|
||||
}
|
||||
|
||||
// Parse the full event.
|
||||
firstEvent = flavor.MakeBinlogEvent(buf[1:])
|
||||
if !firstEvent.IsValid() {
|
||||
return nil, fmt.Errorf("first event from binlog %v is not valid", binlog)
|
||||
}
|
||||
if !firstEvent.IsRotate() {
|
||||
break
|
||||
}
|
||||
}
|
||||
if int64(firstEvent.Timestamp()) < timestamp {
|
||||
// The first event in this binlog has a smaller
|
||||
// timestamp than what we need, we found a good
|
||||
// starting point.
|
||||
break
|
||||
}
|
||||
|
||||
// The timestamp is higher, we need to try the older files.
|
||||
// Close and re-open our connection.
|
||||
sc.Conn.Close()
|
||||
conn, err := sc.mysqld.connectForReplication()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sc.Conn = conn
|
||||
}
|
||||
if binlogIndex == -1 {
|
||||
// We haven't found a suitable binlog
|
||||
log.Errorf("couldn't find an old enough binlog to match timestamp >= %v (looked at %v files)", timestamp, len(binlogs.Rows))
|
||||
return nil, ErrBinlogUnavailable
|
||||
}
|
||||
|
||||
// Now skip all events that have a smaller timestamp
|
||||
var event replication.BinlogEvent
|
||||
for {
|
||||
buf, err := sc.Conn.ReadPacket()
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error reading packet while skipping binlog events: %v", err)
|
||||
}
|
||||
if buf[0] == 254 {
|
||||
// The master is telling us to stop.
|
||||
return nil, fmt.Errorf("received EOF packet in binlog dump while skipping packets: %#v", buf)
|
||||
}
|
||||
|
||||
event = flavor.MakeBinlogEvent(buf[1:])
|
||||
if !event.IsValid() {
|
||||
return nil, fmt.Errorf("event from binlog is not valid (while skipping)")
|
||||
}
|
||||
if int64(event.Timestamp()) >= timestamp {
|
||||
// we found the first event to send
|
||||
break
|
||||
}
|
||||
|
||||
// Exit the loop early if context is canceled.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return nil, ctx.Err()
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
// Now just loop sending and reading events.
|
||||
// FIXME(alainjobart) I think we can use a buffered channel for better performance.
|
||||
eventChan := make(chan replication.BinlogEvent)
|
||||
|
||||
// Start reading events.
|
||||
sc.wg.Add(1)
|
||||
go func() {
|
||||
defer func() {
|
||||
close(eventChan)
|
||||
sc.wg.Done()
|
||||
}()
|
||||
|
||||
// Send the first binlog event, it has the format description.
|
||||
select {
|
||||
case eventChan <- firstEvent:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
|
||||
// Then send the rest.
|
||||
for {
|
||||
select {
|
||||
case eventChan <- event:
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
|
||||
buf, err := sc.Conn.ReadPacket()
|
||||
if err != nil {
|
||||
if sqlErr, ok := err.(*sqldb.SQLError); ok && sqlErr.Number() == mysql.ErrServerLost {
|
||||
// ErrServerLost = Lost connection to MySQL server during query
|
||||
// This is not necessarily an error. It could just be that we closed
|
||||
// the connection from outside.
|
||||
log.Infof("connection closed during binlog stream (possibly intentional): %v", err)
|
||||
return
|
||||
}
|
||||
log.Errorf("read error while streaming binlog events: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
if buf[0] == 254 {
|
||||
// The master is telling us to stop.
|
||||
log.Infof("received EOF packet in binlog dump: %#v", buf)
|
||||
return
|
||||
}
|
||||
|
||||
// Skip the first byte because it's only used
|
||||
// for signaling EOF.
|
||||
event = flavor.MakeBinlogEvent(buf[1:])
|
||||
}
|
||||
}()
|
||||
|
||||
return eventChan, nil
|
||||
}
|
||||
|
@ -131,7 +340,8 @@ func (sc *SlaveConnection) Close() {
|
|||
sc.Conn.Shutdown()
|
||||
|
||||
log.Infof("waiting for slave dump thread to end")
|
||||
sc.svm.Stop()
|
||||
sc.cancel()
|
||||
sc.wg.Wait()
|
||||
|
||||
log.Infof("closing slave MySQL client, recycling slaveID %v", sc.slaveID)
|
||||
sc.Conn.Close()
|
||||
|
@ -157,3 +367,6 @@ func makeBinlogDumpCommand(pos uint32, flags uint16, serverID uint32, filename s
|
|||
|
||||
return buf.Bytes()
|
||||
}
|
||||
|
||||
// ComBinlogDump is the command id for COM_BINLOG_DUMP.
|
||||
const ComBinlogDump = 0x12
|
||||
|
|
|
@ -11,9 +11,6 @@ It is generated from these files:
|
|||
It has these top-level messages:
|
||||
Charset
|
||||
BinlogTransaction
|
||||
StreamEvent
|
||||
StreamUpdateRequest
|
||||
StreamUpdateResponse
|
||||
StreamKeyRangeRequest
|
||||
StreamKeyRangeResponse
|
||||
StreamTablesRequest
|
||||
|
@ -76,34 +73,6 @@ func (BinlogTransaction_Statement_Category) EnumDescriptor() ([]byte, []int) {
|
|||
return fileDescriptor0, []int{1, 0, 0}
|
||||
}
|
||||
|
||||
// the category of this event
|
||||
type StreamEvent_Category int32
|
||||
|
||||
const (
|
||||
StreamEvent_SE_ERR StreamEvent_Category = 0
|
||||
StreamEvent_SE_DML StreamEvent_Category = 1
|
||||
StreamEvent_SE_DDL StreamEvent_Category = 2
|
||||
StreamEvent_SE_POS StreamEvent_Category = 3
|
||||
)
|
||||
|
||||
var StreamEvent_Category_name = map[int32]string{
|
||||
0: "SE_ERR",
|
||||
1: "SE_DML",
|
||||
2: "SE_DDL",
|
||||
3: "SE_POS",
|
||||
}
|
||||
var StreamEvent_Category_value = map[string]int32{
|
||||
"SE_ERR": 0,
|
||||
"SE_DML": 1,
|
||||
"SE_DDL": 2,
|
||||
"SE_POS": 3,
|
||||
}
|
||||
|
||||
func (x StreamEvent_Category) String() string {
|
||||
return proto.EnumName(StreamEvent_Category_name, int32(x))
|
||||
}
|
||||
func (StreamEvent_Category) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{2, 0} }
|
||||
|
||||
// Charset is the per-statement charset info from a QUERY_EVENT binlog entry.
|
||||
type Charset struct {
|
||||
// @@session.character_set_client
|
||||
|
@ -168,76 +137,6 @@ func (m *BinlogTransaction_Statement) GetCharset() *Charset {
|
|||
return nil
|
||||
}
|
||||
|
||||
// StreamEvent describes an update stream event inside the binlogs.
|
||||
// It is streamed by vttablet for the Update Stream feature.
|
||||
// For each BinlogTransaction (transactional unit), we will send any
|
||||
// number of SE_ERR, SE_DML and SE_DDL statements, followed by one SE_POS.
|
||||
type StreamEvent struct {
|
||||
Category StreamEvent_Category `protobuf:"varint,1,opt,name=category,enum=binlogdata.StreamEvent_Category" json:"category,omitempty"`
|
||||
// table_name, primary_key_fields and primary_key_values are set for SE_DML.
|
||||
TableName string `protobuf:"bytes,2,opt,name=table_name,json=tableName" json:"table_name,omitempty"`
|
||||
PrimaryKeyFields []*query.Field `protobuf:"bytes,3,rep,name=primary_key_fields,json=primaryKeyFields" json:"primary_key_fields,omitempty"`
|
||||
PrimaryKeyValues []*query.Row `protobuf:"bytes,4,rep,name=primary_key_values,json=primaryKeyValues" json:"primary_key_values,omitempty"`
|
||||
// sql is set for SE_DDL or SE_ERR.
|
||||
Sql []byte `protobuf:"bytes,5,opt,name=sql,proto3" json:"sql,omitempty"`
|
||||
// The Event Token for this event. Only set for SE_POS.
|
||||
EventToken *query.EventToken `protobuf:"bytes,8,opt,name=event_token,json=eventToken" json:"event_token,omitempty"`
|
||||
}
|
||||
|
||||
func (m *StreamEvent) Reset() { *m = StreamEvent{} }
|
||||
func (m *StreamEvent) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamEvent) ProtoMessage() {}
|
||||
func (*StreamEvent) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
|
||||
|
||||
func (m *StreamEvent) GetPrimaryKeyFields() []*query.Field {
|
||||
if m != nil {
|
||||
return m.PrimaryKeyFields
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StreamEvent) GetPrimaryKeyValues() []*query.Row {
|
||||
if m != nil {
|
||||
return m.PrimaryKeyValues
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StreamEvent) GetEventToken() *query.EventToken {
|
||||
if m != nil {
|
||||
return m.EventToken
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// StreamUpdateRequest is the payload to StreamUpdate
|
||||
type StreamUpdateRequest struct {
|
||||
// where to start
|
||||
Position string `protobuf:"bytes,1,opt,name=position" json:"position,omitempty"`
|
||||
}
|
||||
|
||||
func (m *StreamUpdateRequest) Reset() { *m = StreamUpdateRequest{} }
|
||||
func (m *StreamUpdateRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamUpdateRequest) ProtoMessage() {}
|
||||
func (*StreamUpdateRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
|
||||
|
||||
// StreamUpdateResponse is the response from StreamUpdate
|
||||
type StreamUpdateResponse struct {
|
||||
StreamEvent *StreamEvent `protobuf:"bytes,1,opt,name=stream_event,json=streamEvent" json:"stream_event,omitempty"`
|
||||
}
|
||||
|
||||
func (m *StreamUpdateResponse) Reset() { *m = StreamUpdateResponse{} }
|
||||
func (m *StreamUpdateResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamUpdateResponse) ProtoMessage() {}
|
||||
func (*StreamUpdateResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
|
||||
|
||||
func (m *StreamUpdateResponse) GetStreamEvent() *StreamEvent {
|
||||
if m != nil {
|
||||
return m.StreamEvent
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// StreamKeyRangeRequest is the payload to StreamKeyRange
|
||||
type StreamKeyRangeRequest struct {
|
||||
// where to start
|
||||
|
@ -251,7 +150,7 @@ type StreamKeyRangeRequest struct {
|
|||
func (m *StreamKeyRangeRequest) Reset() { *m = StreamKeyRangeRequest{} }
|
||||
func (m *StreamKeyRangeRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamKeyRangeRequest) ProtoMessage() {}
|
||||
func (*StreamKeyRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
|
||||
func (*StreamKeyRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
|
||||
|
||||
func (m *StreamKeyRangeRequest) GetKeyRange() *topodata.KeyRange {
|
||||
if m != nil {
|
||||
|
@ -275,7 +174,7 @@ type StreamKeyRangeResponse struct {
|
|||
func (m *StreamKeyRangeResponse) Reset() { *m = StreamKeyRangeResponse{} }
|
||||
func (m *StreamKeyRangeResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamKeyRangeResponse) ProtoMessage() {}
|
||||
func (*StreamKeyRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{6} }
|
||||
func (*StreamKeyRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
|
||||
|
||||
func (m *StreamKeyRangeResponse) GetBinlogTransaction() *BinlogTransaction {
|
||||
if m != nil {
|
||||
|
@ -297,7 +196,7 @@ type StreamTablesRequest struct {
|
|||
func (m *StreamTablesRequest) Reset() { *m = StreamTablesRequest{} }
|
||||
func (m *StreamTablesRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamTablesRequest) ProtoMessage() {}
|
||||
func (*StreamTablesRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{7} }
|
||||
func (*StreamTablesRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{4} }
|
||||
|
||||
func (m *StreamTablesRequest) GetCharset() *Charset {
|
||||
if m != nil {
|
||||
|
@ -314,7 +213,7 @@ type StreamTablesResponse struct {
|
|||
func (m *StreamTablesResponse) Reset() { *m = StreamTablesResponse{} }
|
||||
func (m *StreamTablesResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamTablesResponse) ProtoMessage() {}
|
||||
func (*StreamTablesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{8} }
|
||||
func (*StreamTablesResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{5} }
|
||||
|
||||
func (m *StreamTablesResponse) GetBinlogTransaction() *BinlogTransaction {
|
||||
if m != nil {
|
||||
|
@ -327,62 +226,47 @@ func init() {
|
|||
proto.RegisterType((*Charset)(nil), "binlogdata.Charset")
|
||||
proto.RegisterType((*BinlogTransaction)(nil), "binlogdata.BinlogTransaction")
|
||||
proto.RegisterType((*BinlogTransaction_Statement)(nil), "binlogdata.BinlogTransaction.Statement")
|
||||
proto.RegisterType((*StreamEvent)(nil), "binlogdata.StreamEvent")
|
||||
proto.RegisterType((*StreamUpdateRequest)(nil), "binlogdata.StreamUpdateRequest")
|
||||
proto.RegisterType((*StreamUpdateResponse)(nil), "binlogdata.StreamUpdateResponse")
|
||||
proto.RegisterType((*StreamKeyRangeRequest)(nil), "binlogdata.StreamKeyRangeRequest")
|
||||
proto.RegisterType((*StreamKeyRangeResponse)(nil), "binlogdata.StreamKeyRangeResponse")
|
||||
proto.RegisterType((*StreamTablesRequest)(nil), "binlogdata.StreamTablesRequest")
|
||||
proto.RegisterType((*StreamTablesResponse)(nil), "binlogdata.StreamTablesResponse")
|
||||
proto.RegisterEnum("binlogdata.BinlogTransaction_Statement_Category", BinlogTransaction_Statement_Category_name, BinlogTransaction_Statement_Category_value)
|
||||
proto.RegisterEnum("binlogdata.StreamEvent_Category", StreamEvent_Category_name, StreamEvent_Category_value)
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor0) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 682 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x54, 0xcd, 0x6e, 0xda, 0x4a,
|
||||
0x14, 0x0e, 0xd8, 0x01, 0xfb, 0x98, 0x9b, 0x4c, 0x26, 0xf7, 0xe6, 0x5a, 0x48, 0x91, 0x90, 0x37,
|
||||
0x97, 0xcd, 0xa5, 0x2d, 0xdd, 0x54, 0x51, 0x37, 0x35, 0xb8, 0x51, 0x12, 0x03, 0xd5, 0x40, 0xba,
|
||||
0xe8, 0xc6, 0x1a, 0x60, 0x92, 0xa2, 0x80, 0x0d, 0x9e, 0x49, 0x5a, 0x1e, 0xa2, 0xab, 0xbe, 0x46,
|
||||
0x9f, 0xad, 0xcf, 0x50, 0x79, 0x3c, 0x36, 0x4e, 0xd2, 0x1f, 0xba, 0xe8, 0xee, 0x9b, 0x33, 0xdf,
|
||||
0x77, 0x7c, 0xce, 0x77, 0xce, 0x18, 0xd0, 0x78, 0x16, 0xce, 0xa3, 0xeb, 0x29, 0x15, 0xb4, 0xb5,
|
||||
0x8c, 0x23, 0x11, 0x61, 0xd8, 0x44, 0xea, 0xd6, 0xea, 0x96, 0xc5, 0xeb, 0xf4, 0xa2, 0xbe, 0x27,
|
||||
0xa2, 0x65, 0xb4, 0x21, 0x3a, 0x3d, 0xa8, 0x76, 0xde, 0xd3, 0x98, 0x33, 0x81, 0x8f, 0xa0, 0x32,
|
||||
0x99, 0xcf, 0x58, 0x28, 0xec, 0x52, 0xa3, 0xd4, 0xdc, 0x25, 0xea, 0x84, 0x31, 0xe8, 0x93, 0x28,
|
||||
0x0c, 0xed, 0xb2, 0x8c, 0x4a, 0x9c, 0x70, 0x39, 0x8b, 0xef, 0x58, 0x6c, 0x6b, 0x29, 0x37, 0x3d,
|
||||
0x39, 0x5f, 0x34, 0x38, 0x70, 0xe5, 0xa7, 0x47, 0x31, 0x0d, 0x39, 0x9d, 0x88, 0x59, 0x14, 0xe2,
|
||||
0x53, 0x00, 0x2e, 0xa8, 0x60, 0x0b, 0x16, 0x0a, 0x6e, 0x97, 0x1a, 0x5a, 0xd3, 0x6a, 0xff, 0xd7,
|
||||
0x2a, 0x14, 0xfd, 0x48, 0xd2, 0x1a, 0x66, 0x7c, 0x52, 0x90, 0xe2, 0x36, 0x58, 0xec, 0x8e, 0x85,
|
||||
0x22, 0x10, 0xd1, 0x0d, 0x0b, 0x6d, 0xbd, 0x51, 0x6a, 0x5a, 0xed, 0x83, 0x56, 0xda, 0xa0, 0x97,
|
||||
0xdc, 0x8c, 0x92, 0x0b, 0x02, 0x2c, 0xc7, 0xf5, 0x4f, 0x65, 0x30, 0xf3, 0x6c, 0xd8, 0x07, 0x63,
|
||||
0x42, 0x05, 0xbb, 0x8e, 0xe2, 0xb5, 0x6c, 0x73, 0xaf, 0xfd, 0x74, 0xcb, 0x42, 0x5a, 0x1d, 0xa5,
|
||||
0x23, 0x79, 0x06, 0xfc, 0x3f, 0x54, 0x27, 0xa9, 0x7b, 0xd2, 0x1d, 0xab, 0x7d, 0x58, 0x4c, 0xa6,
|
||||
0x8c, 0x25, 0x19, 0x07, 0x23, 0xd0, 0xf8, 0x6a, 0x2e, 0x2d, 0xab, 0x91, 0x04, 0x3a, 0x2b, 0x30,
|
||||
0xb2, 0xb4, 0xf8, 0x10, 0xf6, 0x5d, 0x3f, 0xb8, 0xec, 0x13, 0xaf, 0x33, 0x38, 0xed, 0x9f, 0xbd,
|
||||
0xf3, 0xba, 0x68, 0x07, 0xd7, 0xc0, 0x70, 0xfd, 0xc0, 0xf5, 0x4e, 0xcf, 0xfa, 0xa8, 0x84, 0xff,
|
||||
0x02, 0xd3, 0xf5, 0x83, 0xce, 0xa0, 0xd7, 0x3b, 0x1b, 0xa1, 0x32, 0xde, 0x07, 0xcb, 0xf5, 0x03,
|
||||
0x32, 0xf0, 0x7d, 0xf7, 0x55, 0xe7, 0x02, 0x69, 0x18, 0xa0, 0xe2, 0xfa, 0x41, 0xb7, 0xe7, 0x23,
|
||||
0x3d, 0xc3, 0x5d, 0x1f, 0xed, 0x2a, 0x3c, 0xf4, 0x46, 0xa8, 0x72, 0xae, 0x1b, 0x65, 0xa4, 0x9d,
|
||||
0xeb, 0x86, 0x86, 0x74, 0xe7, 0x6b, 0x19, 0xac, 0xa1, 0x88, 0x19, 0x5d, 0x48, 0xf3, 0xf0, 0xcb,
|
||||
0x47, 0xee, 0x34, 0x8a, 0x0d, 0x15, 0xa8, 0xdf, 0x73, 0xe3, 0x18, 0x40, 0xd0, 0xf1, 0x9c, 0x05,
|
||||
0x21, 0x5d, 0x30, 0x69, 0x88, 0x49, 0x4c, 0x19, 0xe9, 0xd3, 0x05, 0xc3, 0x27, 0x80, 0x97, 0xf1,
|
||||
0x6c, 0x41, 0xe3, 0x75, 0x70, 0xc3, 0xd6, 0xc1, 0xd5, 0x8c, 0xcd, 0xa7, 0xdc, 0xd6, 0xe4, 0x36,
|
||||
0xd4, 0xd4, 0x0c, 0x5f, 0x27, 0x41, 0x82, 0x14, 0xef, 0x82, 0xad, 0x65, 0x80, 0xe3, 0x17, 0xf7,
|
||||
0xb5, 0x77, 0x74, 0x7e, 0xcb, 0xb8, 0xad, 0x4b, 0x2d, 0x28, 0x2d, 0x89, 0x3e, 0x14, 0x95, 0x6f,
|
||||
0x25, 0x27, 0xf3, 0x7c, 0x37, 0xf7, 0xfc, 0xe1, 0x12, 0x19, 0x5b, 0x2c, 0x91, 0x73, 0x52, 0x98,
|
||||
0x13, 0x40, 0x65, 0xe8, 0x05, 0x1e, 0x21, 0x68, 0x47, 0xe1, 0xc4, 0xf0, 0x52, 0x86, 0xbb, 0x3e,
|
||||
0x2a, 0x2b, 0xfc, 0x66, 0x30, 0x94, 0x56, 0x57, 0x50, 0xf5, 0x5c, 0x37, 0xaa, 0xc8, 0x70, 0x9e,
|
||||
0xc1, 0x61, 0x6a, 0xe2, 0xe5, 0x72, 0x4a, 0x05, 0x23, 0x6c, 0x75, 0xcb, 0xb8, 0xc0, 0x75, 0x30,
|
||||
0x96, 0x11, 0x9f, 0x25, 0x0b, 0x27, 0x7d, 0x37, 0x49, 0x7e, 0x76, 0x08, 0xfc, 0x7d, 0x5f, 0xc2,
|
||||
0x97, 0x51, 0xc8, 0x13, 0x3b, 0x6b, 0x5c, 0xc6, 0x03, 0x59, 0xa7, 0xd4, 0x59, 0xed, 0x7f, 0x7f,
|
||||
0x30, 0x2f, 0x62, 0xf1, 0xcd, 0xc1, 0xf9, 0x5c, 0x82, 0x7f, 0xd2, 0xcb, 0x0b, 0xb6, 0x26, 0x34,
|
||||
0xbc, 0xde, 0xa6, 0x12, 0xfc, 0x04, 0xcc, 0xc4, 0xfc, 0x38, 0xe1, 0xab, 0x7d, 0xc7, 0xad, 0xfc,
|
||||
0x7f, 0x92, 0x67, 0x32, 0x6e, 0x14, 0x2a, 0x3e, 0x0f, 0xed, 0xd7, 0xcf, 0xc3, 0xb9, 0x82, 0xa3,
|
||||
0x87, 0x45, 0xa9, 0x5e, 0x7d, 0xc0, 0xa9, 0x30, 0x10, 0x9b, 0xa7, 0xa9, 0x3a, 0x3e, 0xfe, 0xe9,
|
||||
0xfb, 0x25, 0x07, 0xe3, 0x87, 0x21, 0xe7, 0x63, 0x36, 0x84, 0x51, 0xb2, 0x9b, 0x7c, 0x9b, 0xd6,
|
||||
0x8f, 0xa0, 0x22, 0x17, 0x99, 0xdb, 0xe5, 0x86, 0xd6, 0x34, 0x89, 0x3a, 0xfd, 0x6e, 0x87, 0xd3,
|
||||
0x6c, 0x96, 0xd9, 0x97, 0xff, 0x44, 0x7f, 0xe3, 0x8a, 0xfc, 0xb5, 0x3f, 0xff, 0x16, 0x00, 0x00,
|
||||
0xff, 0xff, 0xe6, 0x7e, 0x8f, 0xb0, 0x17, 0x06, 0x00, 0x00,
|
||||
// 499 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xb4, 0x53, 0xdd, 0x8e, 0xd2, 0x40,
|
||||
0x14, 0xb6, 0xb4, 0xcb, 0x96, 0xd3, 0x75, 0x77, 0x18, 0x94, 0x34, 0x24, 0x26, 0xa4, 0x37, 0x72,
|
||||
0x63, 0x35, 0xf5, 0x09, 0x2c, 0x10, 0xc2, 0xee, 0x00, 0xc9, 0x80, 0x37, 0xde, 0x34, 0x85, 0x3d,
|
||||
0x22, 0x81, 0x9d, 0x81, 0xce, 0xec, 0x46, 0x1e, 0xc2, 0x2b, 0x5f, 0xc3, 0x87, 0x34, 0xfd, 0xa1,
|
||||
0xe0, 0x9a, 0xe8, 0x7a, 0xe1, 0xdd, 0x77, 0xce, 0xf9, 0xce, 0xcf, 0xf7, 0x4d, 0x0b, 0x64, 0xbe,
|
||||
0x12, 0x1b, 0xb9, 0xbc, 0x8d, 0x75, 0xec, 0x6f, 0x13, 0xa9, 0x25, 0x85, 0x63, 0xa6, 0xe5, 0xec,
|
||||
0xee, 0x31, 0xd9, 0xe7, 0x85, 0xd6, 0xa5, 0x96, 0x5b, 0x79, 0x24, 0x7a, 0x23, 0x38, 0xef, 0x7e,
|
||||
0x89, 0x13, 0x85, 0x9a, 0x36, 0xa1, 0xba, 0xd8, 0xac, 0x50, 0x68, 0xd7, 0x68, 0x1b, 0x9d, 0x33,
|
||||
0x5e, 0x44, 0x94, 0x82, 0xb5, 0x90, 0x42, 0xb8, 0x95, 0x2c, 0x9b, 0xe1, 0x94, 0xab, 0x30, 0x79,
|
||||
0xc0, 0xc4, 0x35, 0x73, 0x6e, 0x1e, 0x79, 0x3f, 0x4c, 0xa8, 0x87, 0xd9, 0xea, 0x59, 0x12, 0x0b,
|
||||
0x15, 0x2f, 0xf4, 0x4a, 0x0a, 0x3a, 0x00, 0x50, 0x3a, 0xd6, 0x78, 0x87, 0x42, 0x2b, 0xd7, 0x68,
|
||||
0x9b, 0x1d, 0x27, 0x78, 0xed, 0x9f, 0x1c, 0xfd, 0x5b, 0x8b, 0x3f, 0x3d, 0xf0, 0xf9, 0x49, 0x2b,
|
||||
0x0d, 0xc0, 0xc1, 0x07, 0x14, 0x3a, 0xd2, 0x72, 0x8d, 0xc2, 0xb5, 0xda, 0x46, 0xc7, 0x09, 0xea,
|
||||
0x7e, 0x2e, 0xb0, 0x9f, 0x56, 0x66, 0x69, 0x81, 0x03, 0x96, 0xb8, 0xf5, 0xad, 0x02, 0xb5, 0x72,
|
||||
0x1a, 0x65, 0x60, 0x2f, 0x62, 0x8d, 0x4b, 0x99, 0xec, 0x33, 0x99, 0x97, 0xc1, 0xbb, 0x27, 0x1e,
|
||||
0xe2, 0x77, 0x8b, 0x3e, 0x5e, 0x4e, 0xa0, 0x6f, 0xe0, 0x7c, 0x91, 0xbb, 0x97, 0xb9, 0xe3, 0x04,
|
||||
0x8d, 0xd3, 0x61, 0x85, 0xb1, 0xfc, 0xc0, 0xa1, 0x04, 0x4c, 0xb5, 0xdb, 0x64, 0x96, 0x5d, 0xf0,
|
||||
0x14, 0x7a, 0x3b, 0xb0, 0x0f, 0x63, 0x69, 0x03, 0xae, 0x42, 0x16, 0x7d, 0x1c, 0xf3, 0x7e, 0x77,
|
||||
0x32, 0x18, 0x0f, 0x3f, 0xf5, 0x7b, 0xe4, 0x19, 0xbd, 0x00, 0x3b, 0x64, 0x51, 0xd8, 0x1f, 0x0c,
|
||||
0xc7, 0xc4, 0xa0, 0xcf, 0xa1, 0x16, 0xb2, 0xa8, 0x3b, 0x19, 0x8d, 0x86, 0x33, 0x52, 0xa1, 0x57,
|
||||
0xe0, 0x84, 0x2c, 0xe2, 0x13, 0xc6, 0xc2, 0x0f, 0xdd, 0x1b, 0x62, 0x52, 0x80, 0x6a, 0xc8, 0xa2,
|
||||
0xde, 0x88, 0x11, 0xeb, 0x80, 0x7b, 0x8c, 0x9c, 0x15, 0x78, 0xda, 0x9f, 0x91, 0xea, 0xb5, 0x65,
|
||||
0x57, 0x88, 0x79, 0x6d, 0xd9, 0x26, 0xb1, 0xbc, 0xef, 0x06, 0xbc, 0x9c, 0xea, 0x04, 0xe3, 0xbb,
|
||||
0x1b, 0xdc, 0xf3, 0x58, 0x2c, 0x91, 0xe3, 0xee, 0x1e, 0x95, 0xa6, 0x2d, 0xb0, 0xb7, 0x52, 0xad,
|
||||
0x52, 0x0b, 0x32, 0x9f, 0x6a, 0xbc, 0x8c, 0xe9, 0x5b, 0xa8, 0xad, 0x71, 0x1f, 0x25, 0x29, 0xbf,
|
||||
0xd0, 0x4d, 0xfd, 0xf2, 0xbb, 0x2a, 0x27, 0xd9, 0xeb, 0x02, 0x9d, 0xda, 0x64, 0xfe, 0xdd, 0x26,
|
||||
0xef, 0x33, 0x34, 0x1f, 0x1f, 0xa5, 0xb6, 0x52, 0x28, 0xa4, 0x0c, 0x68, 0xde, 0x18, 0xe9, 0xe3,
|
||||
0x13, 0x65, 0xf7, 0x39, 0xc1, 0xab, 0x3f, 0xbe, 0x23, 0xaf, 0xcf, 0x1f, 0xa7, 0xbc, 0xaf, 0xd0,
|
||||
0xc8, 0xf7, 0xcc, 0xe2, 0xf9, 0x06, 0xd5, 0x53, 0xa4, 0x37, 0xa1, 0xaa, 0x33, 0xb2, 0x5b, 0x69,
|
||||
0x9b, 0x9d, 0x1a, 0x2f, 0xa2, 0x7f, 0x55, 0x78, 0x0b, 0x2f, 0x7e, 0xdd, 0xfc, 0x3f, 0xf4, 0xcd,
|
||||
0xab, 0xd9, 0x2f, 0xfe, 0xfe, 0x67, 0x00, 0x00, 0x00, 0xff, 0xff, 0x61, 0x8a, 0x02, 0x79, 0x1f,
|
||||
0x04, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -44,8 +44,6 @@ const _ = grpc.SupportPackageIsVersion3
|
|||
// Client API for UpdateStream service
|
||||
|
||||
type UpdateStreamClient interface {
|
||||
// StreamUpdate streams the binlog events, to know which objects have changed.
|
||||
StreamUpdate(ctx context.Context, in *binlogdata.StreamUpdateRequest, opts ...grpc.CallOption) (UpdateStream_StreamUpdateClient, error)
|
||||
// StreamKeyRange returns the binlog transactions related to
|
||||
// the specified Keyrange.
|
||||
StreamKeyRange(ctx context.Context, in *binlogdata.StreamKeyRangeRequest, opts ...grpc.CallOption) (UpdateStream_StreamKeyRangeClient, error)
|
||||
|
@ -62,40 +60,8 @@ func NewUpdateStreamClient(cc *grpc.ClientConn) UpdateStreamClient {
|
|||
return &updateStreamClient{cc}
|
||||
}
|
||||
|
||||
func (c *updateStreamClient) StreamUpdate(ctx context.Context, in *binlogdata.StreamUpdateRequest, opts ...grpc.CallOption) (UpdateStream_StreamUpdateClient, error) {
|
||||
stream, err := grpc.NewClientStream(ctx, &_UpdateStream_serviceDesc.Streams[0], c.cc, "/binlogservice.UpdateStream/StreamUpdate", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &updateStreamStreamUpdateClient{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 UpdateStream_StreamUpdateClient interface {
|
||||
Recv() (*binlogdata.StreamUpdateResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type updateStreamStreamUpdateClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *updateStreamStreamUpdateClient) Recv() (*binlogdata.StreamUpdateResponse, error) {
|
||||
m := new(binlogdata.StreamUpdateResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *updateStreamClient) StreamKeyRange(ctx context.Context, in *binlogdata.StreamKeyRangeRequest, opts ...grpc.CallOption) (UpdateStream_StreamKeyRangeClient, error) {
|
||||
stream, err := grpc.NewClientStream(ctx, &_UpdateStream_serviceDesc.Streams[1], c.cc, "/binlogservice.UpdateStream/StreamKeyRange", opts...)
|
||||
stream, err := grpc.NewClientStream(ctx, &_UpdateStream_serviceDesc.Streams[0], c.cc, "/binlogservice.UpdateStream/StreamKeyRange", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -127,7 +93,7 @@ func (x *updateStreamStreamKeyRangeClient) Recv() (*binlogdata.StreamKeyRangeRes
|
|||
}
|
||||
|
||||
func (c *updateStreamClient) StreamTables(ctx context.Context, in *binlogdata.StreamTablesRequest, opts ...grpc.CallOption) (UpdateStream_StreamTablesClient, error) {
|
||||
stream, err := grpc.NewClientStream(ctx, &_UpdateStream_serviceDesc.Streams[2], c.cc, "/binlogservice.UpdateStream/StreamTables", opts...)
|
||||
stream, err := grpc.NewClientStream(ctx, &_UpdateStream_serviceDesc.Streams[1], c.cc, "/binlogservice.UpdateStream/StreamTables", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -161,8 +127,6 @@ func (x *updateStreamStreamTablesClient) Recv() (*binlogdata.StreamTablesRespons
|
|||
// Server API for UpdateStream service
|
||||
|
||||
type UpdateStreamServer interface {
|
||||
// StreamUpdate streams the binlog events, to know which objects have changed.
|
||||
StreamUpdate(*binlogdata.StreamUpdateRequest, UpdateStream_StreamUpdateServer) error
|
||||
// StreamKeyRange returns the binlog transactions related to
|
||||
// the specified Keyrange.
|
||||
StreamKeyRange(*binlogdata.StreamKeyRangeRequest, UpdateStream_StreamKeyRangeServer) error
|
||||
|
@ -175,27 +139,6 @@ func RegisterUpdateStreamServer(s *grpc.Server, srv UpdateStreamServer) {
|
|||
s.RegisterService(&_UpdateStream_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _UpdateStream_StreamUpdate_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(binlogdata.StreamUpdateRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(UpdateStreamServer).StreamUpdate(m, &updateStreamStreamUpdateServer{stream})
|
||||
}
|
||||
|
||||
type UpdateStream_StreamUpdateServer interface {
|
||||
Send(*binlogdata.StreamUpdateResponse) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type updateStreamStreamUpdateServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *updateStreamStreamUpdateServer) Send(m *binlogdata.StreamUpdateResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func _UpdateStream_StreamKeyRange_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(binlogdata.StreamKeyRangeRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
|
@ -243,11 +186,6 @@ var _UpdateStream_serviceDesc = grpc.ServiceDesc{
|
|||
HandlerType: (*UpdateStreamServer)(nil),
|
||||
Methods: []grpc.MethodDesc{},
|
||||
Streams: []grpc.StreamDesc{
|
||||
{
|
||||
StreamName: "StreamUpdate",
|
||||
Handler: _UpdateStream_StreamUpdate_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
{
|
||||
StreamName: "StreamKeyRange",
|
||||
Handler: _UpdateStream_StreamKeyRange_Handler,
|
||||
|
@ -265,16 +203,15 @@ var _UpdateStream_serviceDesc = grpc.ServiceDesc{
|
|||
func init() { proto.RegisterFile("binlogservice.proto", fileDescriptor0) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 164 bytes of a gzipped FileDescriptorProto
|
||||
// 149 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xe2, 0x12, 0x4e, 0xca, 0xcc, 0xcb,
|
||||
0xc9, 0x4f, 0x2f, 0x4e, 0x2d, 0x2a, 0xcb, 0x4c, 0x4e, 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17,
|
||||
0xe2, 0x45, 0x11, 0x94, 0x12, 0x80, 0x70, 0x53, 0x12, 0x4b, 0x12, 0x21, 0x0a, 0x8c, 0x66, 0x32,
|
||||
0x71, 0xf1, 0x84, 0x16, 0xa4, 0x24, 0x96, 0xa4, 0x06, 0x97, 0x14, 0xa5, 0x26, 0xe6, 0x0a, 0x85,
|
||||
0x72, 0xf1, 0x40, 0x58, 0x10, 0x51, 0x21, 0x79, 0x3d, 0x24, 0x3d, 0xc8, 0x32, 0x41, 0xa9, 0x85,
|
||||
0xa5, 0xa9, 0xc5, 0x25, 0x52, 0x0a, 0xb8, 0x15, 0x14, 0x17, 0xe4, 0xe7, 0x15, 0xa7, 0x2a, 0x31,
|
||||
0x18, 0x30, 0x0a, 0x45, 0x73, 0xf1, 0x41, 0xe4, 0xbc, 0x53, 0x2b, 0x83, 0x12, 0xf3, 0xd2, 0x53,
|
||||
0x85, 0x14, 0x31, 0xf5, 0xc1, 0xe4, 0x60, 0x46, 0x2b, 0xe1, 0x53, 0x82, 0x64, 0x38, 0xdc, 0xcd,
|
||||
0x21, 0x89, 0x49, 0x39, 0xa9, 0xc5, 0xd8, 0xdc, 0x0c, 0x91, 0xc1, 0xe3, 0x66, 0x98, 0x02, 0x84,
|
||||
0xb1, 0x49, 0x6c, 0xe0, 0x20, 0x32, 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0x4c, 0x73, 0x00, 0xff,
|
||||
0x5a, 0x01, 0x00, 0x00,
|
||||
0xe2, 0x45, 0x11, 0x94, 0x12, 0x80, 0x70, 0x53, 0x12, 0x4b, 0x12, 0x21, 0x0a, 0x8c, 0x0e, 0x31,
|
||||
0x72, 0xf1, 0x84, 0x16, 0xa4, 0x24, 0x96, 0xa4, 0x06, 0x97, 0x14, 0xa5, 0x26, 0xe6, 0x0a, 0x45,
|
||||
0x73, 0xf1, 0x41, 0x58, 0xde, 0xa9, 0x95, 0x41, 0x89, 0x79, 0xe9, 0xa9, 0x42, 0x8a, 0x7a, 0x48,
|
||||
0xba, 0x50, 0xe5, 0x82, 0x52, 0x0b, 0x4b, 0x53, 0x8b, 0x4b, 0xa4, 0x94, 0xf0, 0x29, 0x29, 0x2e,
|
||||
0xc8, 0xcf, 0x2b, 0x4e, 0x55, 0x62, 0x30, 0x60, 0x14, 0x0a, 0xe5, 0xe2, 0x81, 0xc8, 0x86, 0x24,
|
||||
0x26, 0xe5, 0xa4, 0x16, 0x0b, 0xc9, 0x63, 0xea, 0x83, 0xc8, 0xc0, 0x0c, 0x56, 0xc0, 0xad, 0x00,
|
||||
0x61, 0x6c, 0x12, 0x1b, 0xd8, 0x2f, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa3, 0x4c, 0x2b,
|
||||
0x54, 0x03, 0x01, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@ It has these top-level messages:
|
|||
Field
|
||||
Row
|
||||
QueryResult
|
||||
StreamEvent
|
||||
ExecuteRequest
|
||||
ExecuteResponse
|
||||
ExecuteBatchRequest
|
||||
|
@ -40,6 +41,8 @@ It has these top-level messages:
|
|||
StreamHealthRequest
|
||||
RealtimeStats
|
||||
StreamHealthResponse
|
||||
UpdateStreamRequest
|
||||
UpdateStreamResponse
|
||||
*/
|
||||
package query
|
||||
|
||||
|
@ -260,6 +263,33 @@ func (x Type) String() string {
|
|||
}
|
||||
func (Type) EnumDescriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
|
||||
|
||||
// The category of one statement.
|
||||
type StreamEvent_Statement_Category int32
|
||||
|
||||
const (
|
||||
StreamEvent_Statement_Error StreamEvent_Statement_Category = 0
|
||||
StreamEvent_Statement_DML StreamEvent_Statement_Category = 1
|
||||
StreamEvent_Statement_DDL StreamEvent_Statement_Category = 2
|
||||
)
|
||||
|
||||
var StreamEvent_Statement_Category_name = map[int32]string{
|
||||
0: "Error",
|
||||
1: "DML",
|
||||
2: "DDL",
|
||||
}
|
||||
var StreamEvent_Statement_Category_value = map[string]int32{
|
||||
"Error": 0,
|
||||
"DML": 1,
|
||||
"DDL": 2,
|
||||
}
|
||||
|
||||
func (x StreamEvent_Statement_Category) String() string {
|
||||
return proto.EnumName(StreamEvent_Statement_Category_name, int32(x))
|
||||
}
|
||||
func (StreamEvent_Statement_Category) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor0, []int{9, 0, 0}
|
||||
}
|
||||
|
||||
type SplitQueryRequest_Algorithm int32
|
||||
|
||||
const (
|
||||
|
@ -280,7 +310,7 @@ func (x SplitQueryRequest_Algorithm) String() string {
|
|||
return proto.EnumName(SplitQueryRequest_Algorithm_name, int32(x))
|
||||
}
|
||||
func (SplitQueryRequest_Algorithm) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor0, []int{25, 0}
|
||||
return fileDescriptor0, []int{26, 0}
|
||||
}
|
||||
|
||||
// Target describes what the client expects the tablet is.
|
||||
|
@ -448,6 +478,66 @@ func (m *QueryResult) GetRows() []*Row {
|
|||
return nil
|
||||
}
|
||||
|
||||
// StreamEvent describes a set of transformations that happened as a
|
||||
// single transactional unit on a server. It is streamed back by the
|
||||
// Update Stream calls.
|
||||
type StreamEvent struct {
|
||||
// The statements in this transaction.
|
||||
Statements []*StreamEvent_Statement `protobuf:"bytes,1,rep,name=statements" json:"statements,omitempty"`
|
||||
// The Event Token for this event.
|
||||
EventToken *EventToken `protobuf:"bytes,2,opt,name=event_token,json=eventToken" json:"event_token,omitempty"`
|
||||
}
|
||||
|
||||
func (m *StreamEvent) Reset() { *m = StreamEvent{} }
|
||||
func (m *StreamEvent) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamEvent) ProtoMessage() {}
|
||||
func (*StreamEvent) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
|
||||
|
||||
func (m *StreamEvent) GetStatements() []*StreamEvent_Statement {
|
||||
if m != nil {
|
||||
return m.Statements
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StreamEvent) GetEventToken() *EventToken {
|
||||
if m != nil {
|
||||
return m.EventToken
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// One individual Statement in a transaction.
|
||||
type StreamEvent_Statement struct {
|
||||
Category StreamEvent_Statement_Category `protobuf:"varint,1,opt,name=category,enum=query.StreamEvent_Statement_Category" json:"category,omitempty"`
|
||||
// table_name, primary_key_fields and primary_key_values are set for DML.
|
||||
TableName string `protobuf:"bytes,2,opt,name=table_name,json=tableName" json:"table_name,omitempty"`
|
||||
PrimaryKeyFields []*Field `protobuf:"bytes,3,rep,name=primary_key_fields,json=primaryKeyFields" json:"primary_key_fields,omitempty"`
|
||||
PrimaryKeyValues []*Row `protobuf:"bytes,4,rep,name=primary_key_values,json=primaryKeyValues" json:"primary_key_values,omitempty"`
|
||||
// sql is set for all queries.
|
||||
// FIXME(alainjobart) we may not need it for DMLs.
|
||||
Sql []byte `protobuf:"bytes,5,opt,name=sql,proto3" json:"sql,omitempty"`
|
||||
}
|
||||
|
||||
func (m *StreamEvent_Statement) Reset() { *m = StreamEvent_Statement{} }
|
||||
func (m *StreamEvent_Statement) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamEvent_Statement) ProtoMessage() {}
|
||||
func (*StreamEvent_Statement) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9, 0} }
|
||||
|
||||
func (m *StreamEvent_Statement) GetPrimaryKeyFields() []*Field {
|
||||
if m != nil {
|
||||
return m.PrimaryKeyFields
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StreamEvent_Statement) GetPrimaryKeyValues() []*Row {
|
||||
if m != nil {
|
||||
return m.PrimaryKeyValues
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ExecuteRequest is the payload to Execute
|
||||
type ExecuteRequest struct {
|
||||
EffectiveCallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=effective_caller_id,json=effectiveCallerId" json:"effective_caller_id,omitempty"`
|
||||
|
@ -460,7 +550,7 @@ type ExecuteRequest struct {
|
|||
func (m *ExecuteRequest) Reset() { *m = ExecuteRequest{} }
|
||||
func (m *ExecuteRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*ExecuteRequest) ProtoMessage() {}
|
||||
func (*ExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{9} }
|
||||
func (*ExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
|
||||
|
||||
func (m *ExecuteRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -498,7 +588,7 @@ type ExecuteResponse struct {
|
|||
func (m *ExecuteResponse) Reset() { *m = ExecuteResponse{} }
|
||||
func (m *ExecuteResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*ExecuteResponse) ProtoMessage() {}
|
||||
func (*ExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{10} }
|
||||
func (*ExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
|
||||
|
||||
func (m *ExecuteResponse) GetResult() *QueryResult {
|
||||
if m != nil {
|
||||
|
@ -520,7 +610,7 @@ type ExecuteBatchRequest struct {
|
|||
func (m *ExecuteBatchRequest) Reset() { *m = ExecuteBatchRequest{} }
|
||||
func (m *ExecuteBatchRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*ExecuteBatchRequest) ProtoMessage() {}
|
||||
func (*ExecuteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{11} }
|
||||
func (*ExecuteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
|
||||
|
||||
func (m *ExecuteBatchRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -558,7 +648,7 @@ type ExecuteBatchResponse struct {
|
|||
func (m *ExecuteBatchResponse) Reset() { *m = ExecuteBatchResponse{} }
|
||||
func (m *ExecuteBatchResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*ExecuteBatchResponse) ProtoMessage() {}
|
||||
func (*ExecuteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{12} }
|
||||
func (*ExecuteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
|
||||
|
||||
func (m *ExecuteBatchResponse) GetResults() []*QueryResult {
|
||||
if m != nil {
|
||||
|
@ -578,7 +668,7 @@ type StreamExecuteRequest struct {
|
|||
func (m *StreamExecuteRequest) Reset() { *m = StreamExecuteRequest{} }
|
||||
func (m *StreamExecuteRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamExecuteRequest) ProtoMessage() {}
|
||||
func (*StreamExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{13} }
|
||||
func (*StreamExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
|
||||
|
||||
func (m *StreamExecuteRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -616,7 +706,7 @@ type StreamExecuteResponse struct {
|
|||
func (m *StreamExecuteResponse) Reset() { *m = StreamExecuteResponse{} }
|
||||
func (m *StreamExecuteResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamExecuteResponse) ProtoMessage() {}
|
||||
func (*StreamExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{14} }
|
||||
func (*StreamExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
|
||||
|
||||
func (m *StreamExecuteResponse) GetResult() *QueryResult {
|
||||
if m != nil {
|
||||
|
@ -635,7 +725,7 @@ type BeginRequest struct {
|
|||
func (m *BeginRequest) Reset() { *m = BeginRequest{} }
|
||||
func (m *BeginRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginRequest) ProtoMessage() {}
|
||||
func (*BeginRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{15} }
|
||||
func (*BeginRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
|
||||
|
||||
func (m *BeginRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -666,7 +756,7 @@ type BeginResponse struct {
|
|||
func (m *BeginResponse) Reset() { *m = BeginResponse{} }
|
||||
func (m *BeginResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginResponse) ProtoMessage() {}
|
||||
func (*BeginResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{16} }
|
||||
func (*BeginResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
|
||||
|
||||
// CommitRequest is the payload to Commit
|
||||
type CommitRequest struct {
|
||||
|
@ -679,7 +769,7 @@ type CommitRequest struct {
|
|||
func (m *CommitRequest) Reset() { *m = CommitRequest{} }
|
||||
func (m *CommitRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*CommitRequest) ProtoMessage() {}
|
||||
func (*CommitRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{17} }
|
||||
func (*CommitRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
|
||||
|
||||
func (m *CommitRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -709,7 +799,7 @@ type CommitResponse struct {
|
|||
func (m *CommitResponse) Reset() { *m = CommitResponse{} }
|
||||
func (m *CommitResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*CommitResponse) ProtoMessage() {}
|
||||
func (*CommitResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{18} }
|
||||
func (*CommitResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
|
||||
|
||||
// RollbackRequest is the payload to Rollback
|
||||
type RollbackRequest struct {
|
||||
|
@ -722,7 +812,7 @@ type RollbackRequest struct {
|
|||
func (m *RollbackRequest) Reset() { *m = RollbackRequest{} }
|
||||
func (m *RollbackRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*RollbackRequest) ProtoMessage() {}
|
||||
func (*RollbackRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{19} }
|
||||
func (*RollbackRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
|
||||
|
||||
func (m *RollbackRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -752,7 +842,7 @@ type RollbackResponse struct {
|
|||
func (m *RollbackResponse) Reset() { *m = RollbackResponse{} }
|
||||
func (m *RollbackResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*RollbackResponse) ProtoMessage() {}
|
||||
func (*RollbackResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{20} }
|
||||
func (*RollbackResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
|
||||
|
||||
// BeginExecuteRequest is the payload to BeginExecute
|
||||
type BeginExecuteRequest struct {
|
||||
|
@ -765,7 +855,7 @@ type BeginExecuteRequest struct {
|
|||
func (m *BeginExecuteRequest) Reset() { *m = BeginExecuteRequest{} }
|
||||
func (m *BeginExecuteRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginExecuteRequest) ProtoMessage() {}
|
||||
func (*BeginExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{21} }
|
||||
func (*BeginExecuteRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
|
||||
|
||||
func (m *BeginExecuteRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -809,7 +899,7 @@ type BeginExecuteResponse struct {
|
|||
func (m *BeginExecuteResponse) Reset() { *m = BeginExecuteResponse{} }
|
||||
func (m *BeginExecuteResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginExecuteResponse) ProtoMessage() {}
|
||||
func (*BeginExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{22} }
|
||||
func (*BeginExecuteResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
|
||||
|
||||
func (m *BeginExecuteResponse) GetError() *vtrpc.RPCError {
|
||||
if m != nil {
|
||||
|
@ -837,7 +927,7 @@ type BeginExecuteBatchRequest struct {
|
|||
func (m *BeginExecuteBatchRequest) Reset() { *m = BeginExecuteBatchRequest{} }
|
||||
func (m *BeginExecuteBatchRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginExecuteBatchRequest) ProtoMessage() {}
|
||||
func (*BeginExecuteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{23} }
|
||||
func (*BeginExecuteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
|
||||
|
||||
func (m *BeginExecuteBatchRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -881,7 +971,7 @@ type BeginExecuteBatchResponse struct {
|
|||
func (m *BeginExecuteBatchResponse) Reset() { *m = BeginExecuteBatchResponse{} }
|
||||
func (m *BeginExecuteBatchResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*BeginExecuteBatchResponse) ProtoMessage() {}
|
||||
func (*BeginExecuteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{24} }
|
||||
func (*BeginExecuteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
|
||||
|
||||
func (m *BeginExecuteBatchResponse) GetError() *vtrpc.RPCError {
|
||||
if m != nil {
|
||||
|
@ -923,7 +1013,7 @@ type SplitQueryRequest struct {
|
|||
func (m *SplitQueryRequest) Reset() { *m = SplitQueryRequest{} }
|
||||
func (m *SplitQueryRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*SplitQueryRequest) ProtoMessage() {}
|
||||
func (*SplitQueryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{25} }
|
||||
func (*SplitQueryRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
|
||||
|
||||
func (m *SplitQueryRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
|
@ -964,7 +1054,7 @@ type QuerySplit struct {
|
|||
func (m *QuerySplit) Reset() { *m = QuerySplit{} }
|
||||
func (m *QuerySplit) String() string { return proto.CompactTextString(m) }
|
||||
func (*QuerySplit) ProtoMessage() {}
|
||||
func (*QuerySplit) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{26} }
|
||||
func (*QuerySplit) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
|
||||
|
||||
func (m *QuerySplit) GetQuery() *BoundQuery {
|
||||
if m != nil {
|
||||
|
@ -982,7 +1072,7 @@ type SplitQueryResponse struct {
|
|||
func (m *SplitQueryResponse) Reset() { *m = SplitQueryResponse{} }
|
||||
func (m *SplitQueryResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*SplitQueryResponse) ProtoMessage() {}
|
||||
func (*SplitQueryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{27} }
|
||||
func (*SplitQueryResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
|
||||
|
||||
func (m *SplitQueryResponse) GetQueries() []*QuerySplit {
|
||||
if m != nil {
|
||||
|
@ -998,7 +1088,7 @@ type StreamHealthRequest struct {
|
|||
func (m *StreamHealthRequest) Reset() { *m = StreamHealthRequest{} }
|
||||
func (m *StreamHealthRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamHealthRequest) ProtoMessage() {}
|
||||
func (*StreamHealthRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{28} }
|
||||
func (*StreamHealthRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{29} }
|
||||
|
||||
// RealtimeStats contains information about the tablet status
|
||||
type RealtimeStats struct {
|
||||
|
@ -1034,7 +1124,7 @@ type RealtimeStats struct {
|
|||
func (m *RealtimeStats) Reset() { *m = RealtimeStats{} }
|
||||
func (m *RealtimeStats) String() string { return proto.CompactTextString(m) }
|
||||
func (*RealtimeStats) ProtoMessage() {}
|
||||
func (*RealtimeStats) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{29} }
|
||||
func (*RealtimeStats) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{30} }
|
||||
|
||||
// StreamHealthResponse is streamed by StreamHealth on a regular basis
|
||||
type StreamHealthResponse struct {
|
||||
|
@ -1058,7 +1148,7 @@ type StreamHealthResponse struct {
|
|||
func (m *StreamHealthResponse) Reset() { *m = StreamHealthResponse{} }
|
||||
func (m *StreamHealthResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StreamHealthResponse) ProtoMessage() {}
|
||||
func (*StreamHealthResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{30} }
|
||||
func (*StreamHealthResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{31} }
|
||||
|
||||
func (m *StreamHealthResponse) GetTarget() *Target {
|
||||
if m != nil {
|
||||
|
@ -1074,6 +1164,63 @@ func (m *StreamHealthResponse) GetRealtimeStats() *RealtimeStats {
|
|||
return nil
|
||||
}
|
||||
|
||||
// UpdateStreamRequest is the payload for UpdateStream. Only one of
|
||||
// position and timestamp can be set.
|
||||
type UpdateStreamRequest struct {
|
||||
EffectiveCallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=effective_caller_id,json=effectiveCallerId" json:"effective_caller_id,omitempty"`
|
||||
ImmediateCallerId *VTGateCallerID `protobuf:"bytes,2,opt,name=immediate_caller_id,json=immediateCallerId" json:"immediate_caller_id,omitempty"`
|
||||
Target *Target `protobuf:"bytes,3,opt,name=target" json:"target,omitempty"`
|
||||
// If position is set, we will start the streaming from that replication
|
||||
// position. Incompatible with timestamp.
|
||||
Position string `protobuf:"bytes,4,opt,name=position" json:"position,omitempty"`
|
||||
// If timestamp is set, we will start the streaming from the first
|
||||
// event in the binlogs that have that timestamp. Incompatible with position.
|
||||
Timestamp int64 `protobuf:"varint,5,opt,name=timestamp" json:"timestamp,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{32} }
|
||||
|
||||
func (m *UpdateStreamRequest) GetEffectiveCallerId() *vtrpc.CallerID {
|
||||
if m != nil {
|
||||
return m.EffectiveCallerId
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *UpdateStreamRequest) GetImmediateCallerId() *VTGateCallerID {
|
||||
if m != nil {
|
||||
return m.ImmediateCallerId
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *UpdateStreamRequest) GetTarget() *Target {
|
||||
if m != nil {
|
||||
return m.Target
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// UpdateStreamResponse is returned by UpdateStream
|
||||
type UpdateStreamResponse struct {
|
||||
Event *StreamEvent `protobuf:"bytes,1,opt,name=event" json:"event,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{33} }
|
||||
|
||||
func (m *UpdateStreamResponse) GetEvent() *StreamEvent {
|
||||
if m != nil {
|
||||
return m.Event
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterType((*Target)(nil), "query.Target")
|
||||
proto.RegisterType((*VTGateCallerID)(nil), "query.VTGateCallerID")
|
||||
|
@ -1084,6 +1231,8 @@ func init() {
|
|||
proto.RegisterType((*Field)(nil), "query.Field")
|
||||
proto.RegisterType((*Row)(nil), "query.Row")
|
||||
proto.RegisterType((*QueryResult)(nil), "query.QueryResult")
|
||||
proto.RegisterType((*StreamEvent)(nil), "query.StreamEvent")
|
||||
proto.RegisterType((*StreamEvent_Statement)(nil), "query.StreamEvent.Statement")
|
||||
proto.RegisterType((*ExecuteRequest)(nil), "query.ExecuteRequest")
|
||||
proto.RegisterType((*ExecuteResponse)(nil), "query.ExecuteResponse")
|
||||
proto.RegisterType((*ExecuteBatchRequest)(nil), "query.ExecuteBatchRequest")
|
||||
|
@ -1106,120 +1255,136 @@ func init() {
|
|||
proto.RegisterType((*StreamHealthRequest)(nil), "query.StreamHealthRequest")
|
||||
proto.RegisterType((*RealtimeStats)(nil), "query.RealtimeStats")
|
||||
proto.RegisterType((*StreamHealthResponse)(nil), "query.StreamHealthResponse")
|
||||
proto.RegisterType((*UpdateStreamRequest)(nil), "query.UpdateStreamRequest")
|
||||
proto.RegisterType((*UpdateStreamResponse)(nil), "query.UpdateStreamResponse")
|
||||
proto.RegisterEnum("query.Flag", Flag_name, Flag_value)
|
||||
proto.RegisterEnum("query.Type", Type_name, Type_value)
|
||||
proto.RegisterEnum("query.StreamEvent_Statement_Category", StreamEvent_Statement_Category_name, StreamEvent_Statement_Category_value)
|
||||
proto.RegisterEnum("query.SplitQueryRequest_Algorithm", SplitQueryRequest_Algorithm_name, SplitQueryRequest_Algorithm_value)
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("query.proto", fileDescriptor0) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 1699 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xec, 0x58, 0xeb, 0x73, 0x23, 0x47,
|
||||
0x11, 0xcf, 0xea, 0x65, 0xa9, 0x65, 0xc9, 0x73, 0x63, 0x1f, 0x08, 0x27, 0x45, 0xcc, 0xe6, 0x81,
|
||||
0xb9, 0x5c, 0xb9, 0x0e, 0x9d, 0x31, 0x29, 0x9e, 0x91, 0xec, 0xf5, 0x45, 0x85, 0xac, 0xd3, 0x8d,
|
||||
0x56, 0x2e, 0x8e, 0xa2, 0x6a, 0x6b, 0x2d, 0x8d, 0xed, 0x2d, 0xaf, 0x76, 0xd7, 0x33, 0xb3, 0x76,
|
||||
0xf4, 0xcd, 0x84, 0xf7, 0x33, 0xa1, 0x20, 0x84, 0x47, 0xf1, 0x89, 0xbf, 0x83, 0xa2, 0x8a, 0xff,
|
||||
0x85, 0xa2, 0xf8, 0x00, 0xfc, 0x0b, 0xd4, 0xcc, 0xce, 0xae, 0xe4, 0x58, 0xe1, 0x20, 0xdf, 0x2e,
|
||||
0xc9, 0x27, 0xcd, 0x74, 0xf7, 0x74, 0xf7, 0xef, 0xd7, 0xdd, 0xb3, 0xab, 0x85, 0xea, 0x79, 0x4c,
|
||||
0xd9, 0x74, 0x2b, 0x62, 0xa1, 0x08, 0x71, 0x51, 0x6d, 0xd6, 0xeb, 0x22, 0x8c, 0xc2, 0xb1, 0x2b,
|
||||
0xdc, 0x44, 0xbc, 0x5e, 0xbd, 0x10, 0x2c, 0x1a, 0x25, 0x1b, 0xf3, 0x1c, 0x4a, 0xb6, 0xcb, 0x4e,
|
||||
0xa8, 0xc0, 0xeb, 0x50, 0x3e, 0xa3, 0x53, 0x1e, 0xb9, 0x23, 0xda, 0x30, 0x36, 0x8c, 0xcd, 0x0a,
|
||||
0xc9, 0xf6, 0x78, 0x0d, 0x8a, 0xfc, 0xd4, 0x65, 0xe3, 0x46, 0x4e, 0x29, 0x92, 0x0d, 0xfe, 0x02,
|
||||
0x54, 0x85, 0x7b, 0xe4, 0x53, 0xe1, 0x88, 0x69, 0x44, 0x1b, 0xf9, 0x0d, 0x63, 0xb3, 0xde, 0x5c,
|
||||
0xdb, 0xca, 0xc2, 0xd9, 0x4a, 0x69, 0x4f, 0x23, 0x4a, 0x40, 0x64, 0x6b, 0xf3, 0x2e, 0xd4, 0x0f,
|
||||
0xed, 0x07, 0xae, 0xa0, 0xbb, 0xae, 0xef, 0x53, 0xd6, 0xd9, 0x93, 0xa1, 0x63, 0x4e, 0x59, 0xe0,
|
||||
0x4e, 0xb2, 0xd0, 0xe9, 0xde, 0xfc, 0x36, 0x80, 0x75, 0x41, 0x03, 0x61, 0x87, 0x67, 0x34, 0xc0,
|
||||
0xcf, 0x41, 0x45, 0x78, 0x13, 0xca, 0x85, 0x3b, 0x89, 0x94, 0x69, 0x9e, 0xcc, 0x04, 0xef, 0x93,
|
||||
0xe6, 0x3a, 0x94, 0xa3, 0x90, 0x7b, 0xc2, 0x0b, 0x03, 0x95, 0x63, 0x85, 0x64, 0x7b, 0xf3, 0x6b,
|
||||
0x50, 0x3c, 0x74, 0xfd, 0x98, 0xe2, 0xe7, 0xa1, 0xa0, 0x40, 0x18, 0x0a, 0x44, 0x75, 0x2b, 0xe1,
|
||||
0x51, 0xe5, 0xae, 0x14, 0xd2, 0xf7, 0x85, 0xb4, 0x54, 0xbe, 0x97, 0x49, 0xb2, 0x31, 0xcf, 0x60,
|
||||
0xb9, 0xed, 0x05, 0xe3, 0x43, 0x97, 0x79, 0x12, 0xe0, 0x07, 0x74, 0x83, 0x5f, 0x84, 0x92, 0x5a,
|
||||
0xf0, 0x46, 0x7e, 0x23, 0xbf, 0x59, 0x6d, 0x2e, 0xeb, 0x83, 0x2a, 0x37, 0xa2, 0x75, 0xe6, 0x5f,
|
||||
0x0d, 0x80, 0x76, 0x18, 0x07, 0xe3, 0x47, 0x52, 0x89, 0x11, 0xe4, 0xf9, 0xb9, 0xaf, 0x09, 0x93,
|
||||
0x4b, 0xfc, 0x0d, 0xa8, 0x1f, 0x79, 0xc1, 0xd8, 0xb9, 0xd0, 0xe9, 0xf0, 0x46, 0x4e, 0xb9, 0x7b,
|
||||
0x51, 0xbb, 0x9b, 0x1d, 0xde, 0x9a, 0xcf, 0x9a, 0x5b, 0x81, 0x60, 0x53, 0x52, 0x3b, 0x9a, 0x97,
|
||||
0xad, 0x0f, 0x01, 0xdf, 0x34, 0x92, 0x41, 0xcf, 0xe8, 0x34, 0x0d, 0x7a, 0x46, 0xa7, 0xf8, 0x73,
|
||||
0xf3, 0x88, 0xaa, 0xcd, 0xd5, 0x34, 0xd6, 0xdc, 0x59, 0x0d, 0xf3, 0x4b, 0xb9, 0x57, 0x0d, 0xf3,
|
||||
0x2b, 0x50, 0xdc, 0xf7, 0xa8, 0x3f, 0xc6, 0x18, 0x0a, 0x73, 0x05, 0x57, 0xeb, 0x8c, 0xbe, 0xdc,
|
||||
0xfb, 0xd0, 0x67, 0x7e, 0x11, 0xf2, 0x24, 0xbc, 0xc4, 0x0d, 0x58, 0xf2, 0x69, 0x70, 0x22, 0x4e,
|
||||
0x79, 0xc3, 0xd8, 0xc8, 0x6f, 0x62, 0x92, 0x6e, 0xf1, 0x27, 0x32, 0x26, 0x13, 0x82, 0x53, 0xee,
|
||||
0xde, 0x31, 0xa0, 0xaa, 0x90, 0x13, 0xca, 0x63, 0x5f, 0x48, 0xc6, 0x8f, 0x65, 0x1a, 0x89, 0x83,
|
||||
0x19, 0xe3, 0x2a, 0x37, 0xa2, 0x75, 0xf8, 0x05, 0xa8, 0xb1, 0xf0, 0x92, 0x3b, 0xee, 0xf1, 0x31,
|
||||
0x1d, 0x09, 0x9a, 0x34, 0x56, 0x81, 0x2c, 0x4b, 0x61, 0x4b, 0xcb, 0xf0, 0xb3, 0x50, 0xf1, 0x02,
|
||||
0x4e, 0x99, 0x70, 0xbc, 0xb1, 0x6a, 0xb0, 0x02, 0x29, 0x27, 0x82, 0xce, 0x18, 0x7f, 0x1a, 0x0a,
|
||||
0xd2, 0xb8, 0x51, 0x50, 0x51, 0x40, 0x47, 0x21, 0xe1, 0x25, 0x51, 0x72, 0xf3, 0xad, 0x1c, 0xd4,
|
||||
0xad, 0x37, 0xe8, 0x28, 0x16, 0x94, 0xd0, 0xf3, 0x98, 0x72, 0x81, 0xbf, 0x0e, 0xab, 0x54, 0xf9,
|
||||
0xf6, 0x2e, 0xa8, 0x33, 0x52, 0x33, 0x22, 0x3d, 0x1b, 0x8a, 0xde, 0x95, 0xad, 0x64, 0x7a, 0xd3,
|
||||
0xd9, 0x21, 0xb7, 0x32, 0x5b, 0x2d, 0x1a, 0x63, 0x0b, 0x56, 0xbd, 0xc9, 0x84, 0x8e, 0x3d, 0x57,
|
||||
0xcc, 0x3b, 0x48, 0xea, 0x73, 0x3b, 0x6d, 0xad, 0x6b, 0x23, 0x48, 0x6e, 0x65, 0x27, 0x32, 0x37,
|
||||
0x2f, 0x41, 0x49, 0xa8, 0xab, 0x41, 0x81, 0xaa, 0x36, 0x6b, 0x69, 0x39, 0x94, 0x90, 0x68, 0x25,
|
||||
0xfe, 0x2c, 0x24, 0xf7, 0x4c, 0xa3, 0xa0, 0xac, 0x6e, 0xdd, 0xe8, 0x35, 0x92, 0xe8, 0xf1, 0x4b,
|
||||
0x50, 0x17, 0xcc, 0x0d, 0xb8, 0x3b, 0x92, 0xa3, 0x27, 0x33, 0x2a, 0xaa, 0x01, 0xae, 0xcd, 0x49,
|
||||
0x3b, 0x63, 0xf3, 0xab, 0xb0, 0x92, 0x11, 0xc2, 0xa3, 0x30, 0xe0, 0x14, 0xdf, 0x81, 0x12, 0x53,
|
||||
0x65, 0xd3, 0x24, 0x60, 0x1d, 0x63, 0xae, 0xa0, 0x44, 0x5b, 0x98, 0x7f, 0xc9, 0xc1, 0xaa, 0x3e,
|
||||
0xdf, 0x76, 0xc5, 0xe8, 0xf4, 0x29, 0x65, 0xf5, 0x15, 0x58, 0x92, 0x72, 0x8f, 0xa6, 0xad, 0xb3,
|
||||
0x80, 0xd7, 0xd4, 0x42, 0x32, 0xeb, 0x72, 0x67, 0x8e, 0x46, 0xc5, 0x6c, 0x99, 0xd4, 0x5c, 0x6e,
|
||||
0xcf, 0x84, 0x0b, 0x0a, 0x50, 0x5a, 0x54, 0x80, 0x3d, 0x58, 0xbb, 0x4e, 0xa0, 0xae, 0xc2, 0x5d,
|
||||
0x58, 0x4a, 0x38, 0x4e, 0x67, 0x66, 0x51, 0x19, 0x52, 0x13, 0xf3, 0x5f, 0x06, 0xac, 0x0d, 0x04,
|
||||
0xa3, 0xee, 0xe4, 0xa3, 0xd1, 0xde, 0xe6, 0x2e, 0xdc, 0x7e, 0x0f, 0xde, 0x0f, 0xd0, 0xbd, 0x7f,
|
||||
0x36, 0x60, 0xb9, 0x4d, 0x4f, 0xbc, 0xe0, 0xe9, 0x64, 0xcb, 0xdc, 0x81, 0x9a, 0x4e, 0x5f, 0x83,
|
||||
0xbf, 0xd9, 0x73, 0xc6, 0xa2, 0x9e, 0xfb, 0x9b, 0x01, 0xb5, 0xdd, 0x70, 0x32, 0xf1, 0xc4, 0x53,
|
||||
0xda, 0x26, 0x37, 0x71, 0x16, 0x16, 0xe1, 0x44, 0x50, 0x4f, 0x61, 0x26, 0x04, 0x99, 0x7f, 0x37,
|
||||
0x60, 0x85, 0x84, 0xbe, 0x7f, 0xe4, 0x8e, 0xce, 0x3e, 0xdc, 0xd8, 0x31, 0xa0, 0x19, 0x50, 0x8d,
|
||||
0xfe, 0x9f, 0x06, 0xac, 0xaa, 0x86, 0xf9, 0x88, 0x5c, 0x12, 0x6f, 0x1b, 0xb0, 0x76, 0x1d, 0x6f,
|
||||
0x36, 0x27, 0x45, 0xca, 0x58, 0xc8, 0xde, 0x03, 0x91, 0xf4, 0x77, 0x2d, 0x29, 0x26, 0x89, 0x76,
|
||||
0xee, 0x2e, 0xc9, 0x3d, 0xe9, 0x2e, 0x59, 0x50, 0x96, 0xfc, 0xa2, 0xb2, 0xfc, 0x31, 0x07, 0x8d,
|
||||
0xf9, 0x94, 0x3e, 0x7e, 0x6a, 0x5e, 0x7b, 0x6a, 0x9a, 0xef, 0x1a, 0xf0, 0xa9, 0x05, 0xfc, 0xfc,
|
||||
0x7f, 0x75, 0x9b, 0x7b, 0x76, 0xe6, 0x9e, 0xf8, 0xec, 0xfc, 0x5f, 0x2b, 0xf7, 0x66, 0x01, 0x6e,
|
||||
0x0d, 0x22, 0xdf, 0x13, 0xda, 0xc9, 0x87, 0xfb, 0xf5, 0xf1, 0x33, 0xb0, 0xcc, 0x25, 0x58, 0x67,
|
||||
0x14, 0xfa, 0xf1, 0x44, 0x16, 0x2b, 0xbf, 0x59, 0x21, 0x55, 0x25, 0xdb, 0x55, 0x22, 0xfc, 0x3c,
|
||||
0x54, 0x53, 0x93, 0x38, 0x10, 0xfa, 0xed, 0x06, 0xb4, 0x45, 0x1c, 0x08, 0xbc, 0x0d, 0x9f, 0x0c,
|
||||
0xe2, 0x89, 0xa3, 0xde, 0xe9, 0x23, 0xca, 0x1c, 0xe5, 0xd9, 0x89, 0x5c, 0x26, 0x1a, 0x65, 0x65,
|
||||
0xbc, 0x1a, 0xc4, 0x13, 0x12, 0x5e, 0xf2, 0x3e, 0x65, 0x2a, 0x78, 0xdf, 0x65, 0x02, 0xbf, 0x06,
|
||||
0x15, 0xd7, 0x3f, 0x09, 0x99, 0x27, 0x4e, 0x27, 0x8d, 0x8a, 0xfa, 0x6b, 0x62, 0xea, 0x34, 0x6f,
|
||||
0xd0, 0xbf, 0xd5, 0x4a, 0x2d, 0xc9, 0xec, 0x10, 0x7e, 0x05, 0x70, 0xcc, 0xa9, 0x93, 0x24, 0x97,
|
||||
0x04, 0xbd, 0x68, 0x36, 0x40, 0xb5, 0xdb, 0x4a, 0xcc, 0xe9, 0xcc, 0xcd, 0x61, 0xd3, 0xbc, 0x0b,
|
||||
0x95, 0xcc, 0x09, 0x46, 0xb0, 0x6c, 0x3d, 0x1a, 0xb6, 0xba, 0xce, 0xa0, 0xdf, 0xed, 0xd8, 0x03,
|
||||
0xf4, 0x0c, 0xae, 0x41, 0x65, 0x7f, 0xd8, 0xed, 0x3a, 0x83, 0xdd, 0x56, 0x0f, 0x19, 0x26, 0x01,
|
||||
0x50, 0x07, 0x95, 0x8b, 0x19, 0x9b, 0xc6, 0x13, 0xd8, 0x7c, 0x16, 0x2a, 0x2c, 0xbc, 0xd4, 0x44,
|
||||
0xe5, 0x14, 0xf6, 0x32, 0x0b, 0x2f, 0x15, 0x4d, 0x66, 0x0b, 0xf0, 0x3c, 0x30, 0xdd, 0xea, 0x73,
|
||||
0xc3, 0x65, 0x5c, 0x1b, 0xae, 0x59, 0xfc, 0x6c, 0xb8, 0xcc, 0xdb, 0xb0, 0x9a, 0xbc, 0x0d, 0xbd,
|
||||
0x4e, 0x5d, 0x5f, 0xa4, 0xf7, 0x89, 0xf9, 0xa7, 0x1c, 0xd4, 0x88, 0x94, 0x78, 0x13, 0x3a, 0x10,
|
||||
0xae, 0xe0, 0xb2, 0xac, 0xa7, 0xca, 0xc4, 0x99, 0xcd, 0x51, 0x85, 0x54, 0x13, 0x99, 0x9a, 0x21,
|
||||
0xdc, 0x84, 0xdb, 0x9c, 0x8e, 0xc2, 0x60, 0xcc, 0x9d, 0x23, 0x7a, 0x2a, 0xff, 0xe0, 0x4e, 0x5c,
|
||||
0x2e, 0x28, 0x53, 0x79, 0xd7, 0xc8, 0xaa, 0x56, 0xb6, 0x95, 0xee, 0x40, 0xa9, 0xf0, 0x3d, 0x58,
|
||||
0x3b, 0xf2, 0x02, 0x3f, 0x3c, 0x71, 0x22, 0xdf, 0x9d, 0x52, 0xc6, 0x35, 0x54, 0xd9, 0x8b, 0x45,
|
||||
0x82, 0x13, 0x5d, 0x3f, 0x51, 0x25, 0xbd, 0xf1, 0x2d, 0xb8, 0xb3, 0x30, 0x8a, 0x73, 0xec, 0xf9,
|
||||
0x82, 0x32, 0x3a, 0x76, 0x18, 0x8d, 0x7c, 0x6f, 0xe4, 0xaa, 0xab, 0x22, 0x79, 0xc2, 0xbd, 0xbc,
|
||||
0x20, 0xf4, 0xbe, 0x36, 0x27, 0x33, 0x6b, 0xc9, 0xf6, 0x28, 0x8a, 0x9d, 0x98, 0xbb, 0x27, 0x54,
|
||||
0xdd, 0x32, 0x06, 0x29, 0x8f, 0xa2, 0x78, 0x28, 0xf7, 0xf2, 0x2f, 0xf5, 0x79, 0xc4, 0x55, 0xb7,
|
||||
0x1a, 0x44, 0x2e, 0xcd, 0x7f, 0x64, 0xef, 0xce, 0x29, 0x7b, 0xd9, 0x6d, 0x93, 0xce, 0x94, 0xf1,
|
||||
0xdf, 0x66, 0xaa, 0x01, 0x4b, 0x9c, 0xb2, 0x0b, 0x2f, 0x38, 0x51, 0x14, 0x95, 0x49, 0xba, 0xc5,
|
||||
0x03, 0x78, 0x59, 0x7f, 0xb2, 0xa1, 0x6f, 0x08, 0xca, 0x02, 0xd7, 0xf7, 0xa7, 0x12, 0x97, 0xcb,
|
||||
0x68, 0x20, 0xe8, 0xd8, 0x99, 0x7d, 0x5c, 0x49, 0x6e, 0x9c, 0x17, 0x12, 0x6b, 0x2b, 0x33, 0x26,
|
||||
0x99, 0xad, 0x9d, 0x7d, 0x76, 0xf9, 0x32, 0xd4, 0x99, 0xae, 0xa9, 0xc3, 0x65, 0x51, 0xf5, 0x2c,
|
||||
0xaf, 0xa5, 0xff, 0x76, 0xe7, 0x0b, 0x4e, 0x6a, 0x6c, 0x7e, 0x7b, 0xe7, 0x0c, 0x0a, 0xfb, 0xbe,
|
||||
0x7b, 0x82, 0xcb, 0x50, 0xe8, 0x3d, 0xec, 0x59, 0xe8, 0x19, 0xbc, 0x02, 0xd0, 0x19, 0x74, 0x7a,
|
||||
0xb6, 0xf5, 0x80, 0xb4, 0xba, 0xe8, 0x2a, 0x97, 0x08, 0x86, 0xbd, 0x41, 0xe7, 0x41, 0xcf, 0xda,
|
||||
0x43, 0x57, 0x05, 0xbc, 0x0c, 0x4b, 0x9d, 0xc1, 0x7e, 0xf7, 0x61, 0xcb, 0x46, 0x57, 0x65, 0x5c,
|
||||
0x83, 0x72, 0x67, 0xf0, 0x68, 0xf8, 0xd0, 0x96, 0x4a, 0x84, 0xab, 0x50, 0xea, 0x0c, 0x6c, 0xeb,
|
||||
0x9b, 0x36, 0xba, 0xda, 0x48, 0x74, 0xed, 0x4e, 0xaf, 0x45, 0x1e, 0xa3, 0xab, 0xd7, 0xee, 0xfc,
|
||||
0x3b, 0x07, 0x05, 0x7b, 0x1a, 0x51, 0x39, 0x44, 0x3d, 0x39, 0x44, 0xf6, 0xe3, 0xbe, 0x0c, 0x59,
|
||||
0x81, 0x42, 0xa7, 0x67, 0xbf, 0x8a, 0xbe, 0x93, 0xc3, 0x00, 0xc5, 0xa1, 0x5a, 0xbf, 0x59, 0x92,
|
||||
0xeb, 0x4e, 0xcf, 0xfe, 0xfc, 0x0e, 0xfa, 0x6e, 0x4e, 0xba, 0x1d, 0x26, 0x9b, 0xef, 0xa5, 0x8a,
|
||||
0xe6, 0x36, 0xfa, 0x7e, 0xa6, 0x68, 0x6e, 0xa3, 0x1f, 0xa4, 0x8a, 0xfb, 0x4d, 0xf4, 0xc3, 0x4c,
|
||||
0x71, 0xbf, 0x89, 0x7e, 0x94, 0x2a, 0x76, 0xb6, 0xd1, 0x8f, 0x33, 0xc5, 0xce, 0x36, 0xfa, 0x49,
|
||||
0x49, 0x62, 0x51, 0x48, 0xee, 0x37, 0xd1, 0x4f, 0xcb, 0xd9, 0x6e, 0x67, 0x1b, 0xfd, 0xac, 0x8c,
|
||||
0xeb, 0x50, 0xb1, 0x3b, 0x07, 0xd6, 0xc0, 0x6e, 0x1d, 0xf4, 0xd1, 0xcf, 0x91, 0x4c, 0x73, 0xaf,
|
||||
0x65, 0x5b, 0xe8, 0x17, 0x6a, 0x29, 0x55, 0xe8, 0x2d, 0x24, 0x31, 0x4a, 0xa9, 0xda, 0xbe, 0xad,
|
||||
0x34, 0x8f, 0xad, 0x16, 0x41, 0xbf, 0x2c, 0xe1, 0x2a, 0x2c, 0xed, 0x59, 0xbb, 0x9d, 0x83, 0x56,
|
||||
0x17, 0x61, 0x75, 0x42, 0xb2, 0xf2, 0xab, 0x7b, 0x72, 0xd9, 0xee, 0x3e, 0x6c, 0xa3, 0x5f, 0xf7,
|
||||
0x65, 0xc0, 0xc3, 0x16, 0xd9, 0x7d, 0xbd, 0x45, 0xd0, 0x3b, 0xf7, 0x64, 0xc0, 0xc3, 0x16, 0xd1,
|
||||
0x7c, 0xfd, 0xa6, 0x2f, 0x0d, 0x95, 0xea, 0xdd, 0x7b, 0x32, 0x69, 0x2d, 0xff, 0x6d, 0x1f, 0x97,
|
||||
0x21, 0xdf, 0xee, 0xd8, 0xe8, 0x77, 0x2a, 0x9a, 0xd5, 0x1b, 0x1e, 0xa0, 0xdf, 0x23, 0x29, 0x1c,
|
||||
0x58, 0x36, 0xfa, 0x83, 0x14, 0x16, 0xed, 0x61, 0xbf, 0x6b, 0xa1, 0xe7, 0xda, 0xeb, 0xd0, 0x18,
|
||||
0x85, 0x93, 0xad, 0x69, 0x18, 0x8b, 0xf8, 0x88, 0x6e, 0x5d, 0x78, 0x82, 0x72, 0x9e, 0x7c, 0x7b,
|
||||
0x3c, 0x2a, 0xa9, 0x9f, 0xfb, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0xc8, 0x3c, 0x4c, 0xa7, 0xb5,
|
||||
0x14, 0x00, 0x00,
|
||||
// 1917 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xec, 0x58, 0xed, 0x6f, 0x23, 0x47,
|
||||
0x19, 0xbf, 0xf5, 0x5b, 0xec, 0xc7, 0x71, 0x32, 0x37, 0xc9, 0x81, 0x49, 0x0f, 0x1a, 0xb6, 0xbd,
|
||||
0x36, 0x5c, 0x4f, 0xd6, 0xe1, 0x0b, 0xe1, 0x54, 0x0a, 0x9c, 0x9d, 0x38, 0x57, 0xab, 0x8e, 0xcf,
|
||||
0x37, 0x5e, 0x47, 0x1c, 0x42, 0x5a, 0x6d, 0xec, 0x39, 0x67, 0x95, 0xf5, 0xee, 0x66, 0x67, 0x36,
|
||||
0xa9, 0xbf, 0x85, 0xf2, 0xfe, 0xda, 0x22, 0x28, 0xe5, 0x45, 0x20, 0x24, 0xfe, 0x0e, 0x84, 0xc4,
|
||||
0xff, 0x82, 0x10, 0x1f, 0x80, 0xff, 0x00, 0xa1, 0x99, 0x9d, 0x5d, 0xaf, 0x73, 0x6e, 0x0f, 0xfa,
|
||||
0xed, 0x5a, 0x3e, 0x79, 0xe6, 0x79, 0x9e, 0x79, 0x5e, 0x7e, 0xcf, 0x8b, 0x77, 0x06, 0xca, 0xa7,
|
||||
0x21, 0x0d, 0xa6, 0x35, 0x3f, 0xf0, 0xb8, 0x87, 0xf3, 0x72, 0xb3, 0xb1, 0xc2, 0x3d, 0xdf, 0x1b,
|
||||
0x59, 0xdc, 0x8a, 0xc8, 0x1b, 0xe5, 0x33, 0x1e, 0xf8, 0xc3, 0x68, 0xa3, 0x9f, 0x42, 0xc1, 0xb0,
|
||||
0x82, 0x31, 0xe5, 0x78, 0x03, 0x8a, 0x27, 0x74, 0xca, 0x7c, 0x6b, 0x48, 0xab, 0xda, 0xa6, 0xb6,
|
||||
0x55, 0x22, 0xc9, 0x1e, 0xaf, 0x43, 0x9e, 0x1d, 0x5b, 0xc1, 0xa8, 0x9a, 0x91, 0x8c, 0x68, 0x83,
|
||||
0xbf, 0x00, 0x65, 0x6e, 0x1d, 0x39, 0x94, 0x9b, 0x7c, 0xea, 0xd3, 0x6a, 0x76, 0x53, 0xdb, 0x5a,
|
||||
0xa9, 0xaf, 0xd7, 0x12, 0x73, 0x86, 0x64, 0x1a, 0x53, 0x9f, 0x12, 0xe0, 0xc9, 0x5a, 0xbf, 0x05,
|
||||
0x2b, 0x87, 0xc6, 0x7d, 0x8b, 0xd3, 0x5d, 0xcb, 0x71, 0x68, 0xd0, 0xde, 0x13, 0xa6, 0x43, 0x46,
|
||||
0x03, 0xd7, 0x9a, 0x24, 0xa6, 0xe3, 0xbd, 0xfe, 0x0d, 0x80, 0xd6, 0x19, 0x75, 0xb9, 0xe1, 0x9d,
|
||||
0x50, 0x17, 0x5f, 0x87, 0x12, 0xb7, 0x27, 0x94, 0x71, 0x6b, 0xe2, 0x4b, 0xd1, 0x2c, 0x99, 0x11,
|
||||
0xde, 0xc7, 0xcd, 0x0d, 0x28, 0xfa, 0x1e, 0xb3, 0xb9, 0xed, 0xb9, 0xd2, 0xc7, 0x12, 0x49, 0xf6,
|
||||
0xfa, 0x57, 0x20, 0x7f, 0x68, 0x39, 0x21, 0xc5, 0xcf, 0x43, 0x4e, 0x06, 0xa1, 0xc9, 0x20, 0xca,
|
||||
0xb5, 0x08, 0x47, 0xe9, 0xbb, 0x64, 0x08, 0xdd, 0x67, 0x42, 0x52, 0xea, 0x5e, 0x26, 0xd1, 0x46,
|
||||
0x3f, 0x81, 0xe5, 0xa6, 0xed, 0x8e, 0x0e, 0xad, 0xc0, 0x16, 0x01, 0x7e, 0x48, 0x35, 0xf8, 0x45,
|
||||
0x28, 0xc8, 0x05, 0xab, 0x66, 0x37, 0xb3, 0x5b, 0xe5, 0xfa, 0xb2, 0x3a, 0x28, 0x7d, 0x23, 0x8a,
|
||||
0xa7, 0xff, 0x45, 0x03, 0x68, 0x7a, 0xa1, 0x3b, 0x7a, 0x28, 0x98, 0x18, 0x41, 0x96, 0x9d, 0x3a,
|
||||
0x0a, 0x30, 0xb1, 0xc4, 0x6f, 0xc0, 0xca, 0x91, 0xed, 0x8e, 0xcc, 0x33, 0xe5, 0x0e, 0xab, 0x66,
|
||||
0xa4, 0xba, 0x17, 0x95, 0xba, 0xd9, 0xe1, 0x5a, 0xda, 0x6b, 0xd6, 0x72, 0x79, 0x30, 0x25, 0x95,
|
||||
0xa3, 0x34, 0x6d, 0x63, 0x00, 0xf8, 0x49, 0x21, 0x61, 0xf4, 0x84, 0x4e, 0x63, 0xa3, 0x27, 0x74,
|
||||
0x8a, 0x3f, 0x97, 0x8e, 0xa8, 0x5c, 0x5f, 0x8b, 0x6d, 0xa5, 0xce, 0xaa, 0x30, 0x5f, 0xcd, 0xdc,
|
||||
0xd5, 0xf4, 0xd7, 0x20, 0xbf, 0x6f, 0x53, 0x67, 0x84, 0x31, 0xe4, 0x52, 0x09, 0x97, 0xeb, 0x04,
|
||||
0xbe, 0xcc, 0xfb, 0xc0, 0xa7, 0x7f, 0x11, 0xb2, 0xc4, 0x3b, 0xc7, 0x55, 0x58, 0x72, 0xa8, 0x3b,
|
||||
0xe6, 0xc7, 0xac, 0xaa, 0x6d, 0x66, 0xb7, 0x30, 0x89, 0xb7, 0xf8, 0x13, 0x09, 0x92, 0x11, 0xc0,
|
||||
0x31, 0x76, 0xef, 0x6a, 0x50, 0x96, 0x91, 0x13, 0xca, 0x42, 0x87, 0x0b, 0xc4, 0x1f, 0x0b, 0x37,
|
||||
0x22, 0x05, 0x33, 0xc4, 0xa5, 0x6f, 0x44, 0xf1, 0xf0, 0x0b, 0x50, 0x09, 0xbc, 0x73, 0x66, 0x5a,
|
||||
0x8f, 0x1f, 0xd3, 0x21, 0xa7, 0x51, 0x61, 0xe5, 0xc8, 0xb2, 0x20, 0x36, 0x14, 0x0d, 0x3f, 0x07,
|
||||
0x25, 0xdb, 0x65, 0x34, 0xe0, 0xa6, 0x3d, 0x92, 0x05, 0x96, 0x23, 0xc5, 0x88, 0xd0, 0x1e, 0xe1,
|
||||
0xcf, 0x40, 0x4e, 0x08, 0x57, 0x73, 0xd2, 0x0a, 0x28, 0x2b, 0xc4, 0x3b, 0x27, 0x92, 0xae, 0xff,
|
||||
0x21, 0x0b, 0xe5, 0x3e, 0x0f, 0xa8, 0x35, 0x91, 0x55, 0x8e, 0x5f, 0x03, 0x60, 0xdc, 0xe2, 0x74,
|
||||
0x42, 0x5d, 0x1e, 0xfb, 0x76, 0x5d, 0x9d, 0x4a, 0xc9, 0xd5, 0xfa, 0xb1, 0x10, 0x49, 0xc9, 0xe3,
|
||||
0x3a, 0x94, 0xa9, 0x60, 0x9b, 0x5c, 0x74, 0x8b, 0xca, 0xc8, 0x55, 0x75, 0x7c, 0xd6, 0x46, 0x04,
|
||||
0x68, 0xb2, 0xde, 0xf8, 0x7d, 0x06, 0x4a, 0x89, 0x36, 0xdc, 0x80, 0xe2, 0xd0, 0xe2, 0x74, 0xec,
|
||||
0x05, 0x53, 0x55, 0xc4, 0x37, 0x3e, 0xc8, 0x7a, 0x6d, 0x57, 0x09, 0x93, 0xe4, 0x18, 0xfe, 0x34,
|
||||
0x44, 0xdd, 0x6e, 0xca, 0xf4, 0x46, 0xad, 0x58, 0x92, 0x94, 0xae, 0xc8, 0xf1, 0xab, 0x80, 0xfd,
|
||||
0xc0, 0x9e, 0x58, 0xc1, 0xd4, 0x3c, 0xa1, 0x53, 0x53, 0x65, 0x21, 0xbb, 0x20, 0x0b, 0x48, 0xc9,
|
||||
0xbd, 0x41, 0xa7, 0xfb, 0x51, 0x3e, 0xee, 0xce, 0x9f, 0x55, 0x99, 0x7e, 0x12, 0xdb, 0xd4, 0x49,
|
||||
0xd9, 0x42, 0x2c, 0x6e, 0x96, 0xbc, 0x2c, 0x0a, 0xb1, 0xd4, 0x5f, 0x86, 0x62, 0xec, 0x3c, 0x2e,
|
||||
0x41, 0xbe, 0x15, 0x04, 0x5e, 0x80, 0xae, 0xe0, 0x25, 0xc8, 0xee, 0x1d, 0x74, 0x90, 0x26, 0x17,
|
||||
0x7b, 0x1d, 0x94, 0xd1, 0xdf, 0xce, 0xc0, 0x4a, 0xeb, 0x4d, 0x3a, 0x0c, 0x39, 0x25, 0xf4, 0x34,
|
||||
0xa4, 0x8c, 0xe3, 0xaf, 0xc2, 0x1a, 0x95, 0xe9, 0xb7, 0xcf, 0xa8, 0x39, 0x94, 0x63, 0x4c, 0x24,
|
||||
0x5f, 0x93, 0x78, 0xaf, 0xd6, 0xa2, 0x01, 0x1b, 0x8f, 0x37, 0x72, 0x35, 0x91, 0x55, 0xa4, 0x11,
|
||||
0x6e, 0xc1, 0x9a, 0x3d, 0x99, 0xd0, 0x91, 0x6d, 0xf1, 0xb4, 0x82, 0x28, 0x61, 0xd7, 0xe2, 0xee,
|
||||
0x9f, 0x9b, 0x92, 0xe4, 0x6a, 0x72, 0x22, 0x51, 0x73, 0x03, 0x0a, 0x5c, 0x4e, 0x6f, 0x59, 0x77,
|
||||
0xe5, 0x7a, 0x25, 0xee, 0x18, 0x49, 0x24, 0x8a, 0x89, 0x5f, 0x86, 0xe8, 0xaf, 0xa0, 0x9a, 0x9b,
|
||||
0x2b, 0x88, 0xd9, 0x38, 0x20, 0x11, 0x1f, 0xdf, 0x80, 0x15, 0x1e, 0x58, 0x2e, 0xb3, 0x86, 0x62,
|
||||
0x3a, 0x0a, 0x8f, 0xf2, 0x72, 0xc6, 0x56, 0x52, 0xd4, 0xf6, 0x48, 0xff, 0x32, 0xac, 0x26, 0x80,
|
||||
0x30, 0xdf, 0x73, 0x19, 0xc5, 0x37, 0xa1, 0x10, 0xc8, 0xce, 0x52, 0x20, 0x60, 0x65, 0x23, 0xd5,
|
||||
0x73, 0x44, 0x49, 0xe8, 0x7f, 0xce, 0xc0, 0x9a, 0x3a, 0xdf, 0xb4, 0xf8, 0xf0, 0xf8, 0x19, 0x45,
|
||||
0xf5, 0x15, 0x58, 0x12, 0x74, 0x3b, 0xa9, 0xc0, 0x05, 0xb8, 0xc6, 0x12, 0x02, 0x59, 0x8b, 0x99,
|
||||
0x29, 0x18, 0x25, 0xb2, 0x45, 0x52, 0xb1, 0x98, 0x31, 0x23, 0x2e, 0x48, 0x40, 0x61, 0x51, 0x02,
|
||||
0xf6, 0x60, 0x7d, 0x1e, 0x40, 0x95, 0x85, 0x5b, 0xb0, 0x14, 0x61, 0x1c, 0x8f, 0x8e, 0x45, 0x69,
|
||||
0x88, 0x45, 0xf4, 0x7f, 0x6a, 0xb0, 0xae, 0xba, 0xfa, 0x63, 0x51, 0xde, 0xfa, 0x2e, 0x5c, 0xbb,
|
||||
0x14, 0xef, 0x87, 0xa8, 0xde, 0x3f, 0x69, 0xb0, 0xdc, 0xa4, 0x63, 0xdb, 0x7d, 0x36, 0xd1, 0xd2,
|
||||
0x77, 0xa0, 0xa2, 0xdc, 0x57, 0xc1, 0x3f, 0x59, 0x73, 0xda, 0xa2, 0x9a, 0xfb, 0xab, 0x06, 0x95,
|
||||
0x5d, 0x6f, 0x32, 0xb1, 0xf9, 0x33, 0x5a, 0x26, 0x4f, 0xc6, 0x99, 0x5b, 0x14, 0x27, 0x82, 0x95,
|
||||
0x38, 0xcc, 0x08, 0x20, 0xfd, 0x6f, 0x1a, 0xac, 0x12, 0xcf, 0x71, 0x8e, 0xac, 0xe1, 0xc9, 0x47,
|
||||
0x3b, 0x76, 0x0c, 0x68, 0x16, 0xa8, 0x8a, 0xfe, 0x1f, 0x1a, 0xac, 0xc9, 0x82, 0xf9, 0x98, 0x0c,
|
||||
0x89, 0x77, 0x34, 0x58, 0x9f, 0x8f, 0x37, 0xe9, 0x93, 0x3c, 0x15, 0x1f, 0x09, 0x97, 0x42, 0x24,
|
||||
0xbd, 0x5d, 0xf9, 0xed, 0x40, 0x22, 0x6e, 0x6a, 0x96, 0x64, 0x9e, 0x36, 0x4b, 0x16, 0xa4, 0x25,
|
||||
0xbb, 0x28, 0x2d, 0xbf, 0xcb, 0x40, 0x35, 0xed, 0xd2, 0xff, 0xff, 0x35, 0xe7, 0xfe, 0x35, 0xf5,
|
||||
0xf7, 0x34, 0xf8, 0xd4, 0x02, 0x7c, 0xfe, 0xb7, 0xbc, 0xa5, 0xfe, 0x3b, 0x33, 0x4f, 0xfd, 0xef,
|
||||
0xfc, 0x6f, 0x33, 0xf7, 0x56, 0x0e, 0xae, 0xf6, 0x7d, 0xc7, 0xe6, 0x4a, 0xc9, 0x47, 0xfb, 0xf3,
|
||||
0xf1, 0xb3, 0xb0, 0xcc, 0x44, 0xb0, 0xe6, 0xd0, 0x73, 0xc2, 0x89, 0x48, 0x56, 0x76, 0xab, 0x44,
|
||||
0xca, 0x92, 0xb6, 0x2b, 0x49, 0xf8, 0x79, 0x28, 0xc7, 0x22, 0xa1, 0xcb, 0xd5, 0xd7, 0x0d, 0x28,
|
||||
0x89, 0xd0, 0xe5, 0x78, 0x1b, 0x3e, 0xe9, 0x86, 0x13, 0x53, 0x5e, 0xbb, 0x7c, 0x1a, 0x98, 0x52,
|
||||
0xb3, 0xe9, 0x5b, 0x01, 0xaf, 0x16, 0xa5, 0xf0, 0x9a, 0x1b, 0x4e, 0x88, 0x77, 0xce, 0x7a, 0x34,
|
||||
0x90, 0xc6, 0x7b, 0x56, 0xc0, 0xf1, 0x3d, 0x28, 0x59, 0xce, 0xd8, 0x0b, 0x6c, 0x7e, 0x3c, 0xa9,
|
||||
0x96, 0xe4, 0xbd, 0x45, 0x8f, 0xef, 0x2d, 0x97, 0xe1, 0xaf, 0x35, 0x62, 0x49, 0x32, 0x3b, 0x84,
|
||||
0x5f, 0x01, 0x1c, 0x32, 0x6a, 0x46, 0xce, 0x45, 0x46, 0xcf, 0xea, 0x55, 0x90, 0xe5, 0xb6, 0x1a,
|
||||
0x32, 0x3a, 0x53, 0x73, 0x58, 0xd7, 0x6f, 0x41, 0x29, 0x51, 0x82, 0x11, 0x2c, 0xb7, 0x1e, 0x0e,
|
||||
0x1a, 0x1d, 0xb3, 0xdf, 0xeb, 0xb4, 0x8d, 0x3e, 0xba, 0x82, 0x2b, 0x50, 0xda, 0x1f, 0x74, 0x3a,
|
||||
0x66, 0x7f, 0xb7, 0xd1, 0x45, 0x9a, 0x4e, 0x00, 0xe4, 0x41, 0xa9, 0x62, 0x86, 0xa6, 0xf6, 0x14,
|
||||
0x34, 0x9f, 0x83, 0x52, 0xe0, 0x9d, 0x2b, 0xa0, 0x32, 0x32, 0xf6, 0x62, 0xe0, 0x9d, 0x4b, 0x98,
|
||||
0xf4, 0x06, 0xe0, 0x74, 0x60, 0xaa, 0xd4, 0x53, 0xcd, 0xa5, 0xcd, 0x35, 0xd7, 0xcc, 0x7e, 0xd2,
|
||||
0x5c, 0xfa, 0x35, 0x58, 0x8b, 0xbe, 0x86, 0x5e, 0xa7, 0x96, 0xc3, 0xe3, 0x79, 0xa2, 0xff, 0x31,
|
||||
0x03, 0x15, 0x22, 0x28, 0xf6, 0x84, 0x8a, 0x7b, 0x1e, 0x13, 0x69, 0x3d, 0x96, 0x22, 0xe6, 0xac,
|
||||
0x8f, 0x4a, 0xa4, 0x1c, 0xd1, 0x64, 0x0f, 0xe1, 0x3a, 0x5c, 0x63, 0x74, 0xe8, 0xb9, 0x23, 0x66,
|
||||
0x1e, 0xd1, 0x63, 0xdb, 0x1d, 0x99, 0x13, 0x8b, 0x71, 0x1a, 0x48, 0xbf, 0x2b, 0x64, 0x4d, 0x31,
|
||||
0x9b, 0x92, 0x77, 0x20, 0x59, 0xf8, 0x36, 0xac, 0x1f, 0xd9, 0xae, 0xe3, 0x8d, 0x4d, 0xdf, 0xb1,
|
||||
0xa6, 0x34, 0x60, 0x2a, 0x54, 0x51, 0x8b, 0x79, 0x82, 0x23, 0x5e, 0x2f, 0x62, 0x45, 0xb5, 0xf1,
|
||||
0x75, 0xb8, 0xb9, 0xd0, 0x8a, 0xf9, 0xd8, 0x76, 0x38, 0x0d, 0xe8, 0xc8, 0x0c, 0xa8, 0xef, 0xd8,
|
||||
0x43, 0x4b, 0x8e, 0x8a, 0xe8, 0x1f, 0xee, 0xa5, 0x05, 0xa6, 0xf7, 0x95, 0x38, 0x99, 0x49, 0x0b,
|
||||
0xb4, 0x87, 0x7e, 0x68, 0x86, 0xcc, 0x1a, 0x53, 0x39, 0x65, 0x34, 0x52, 0x1c, 0xfa, 0xe1, 0x40,
|
||||
0xec, 0xc5, 0xed, 0xf1, 0xd4, 0x67, 0xb2, 0x5a, 0x35, 0x22, 0x96, 0xfa, 0xdf, 0x93, 0x6f, 0xe7,
|
||||
0x18, 0xbd, 0x64, 0xda, 0xc4, 0x3d, 0xa5, 0x7d, 0x50, 0x4f, 0x55, 0x61, 0x89, 0xd1, 0xe0, 0xcc,
|
||||
0x76, 0xc7, 0x12, 0xa2, 0x22, 0x89, 0xb7, 0xb8, 0x0f, 0x2f, 0xa9, 0x57, 0x35, 0xfa, 0x26, 0xa7,
|
||||
0x81, 0x6b, 0x39, 0xce, 0x54, 0xc4, 0x65, 0x05, 0xd4, 0xe5, 0x74, 0x64, 0xce, 0xde, 0xbf, 0xa2,
|
||||
0x89, 0xf3, 0x42, 0x24, 0xdd, 0x4a, 0x84, 0x49, 0x22, 0x6b, 0x24, 0x2f, 0x63, 0x5f, 0x82, 0x95,
|
||||
0x40, 0xe5, 0xd4, 0x64, 0x22, 0xa9, 0xaa, 0x97, 0xd7, 0xe3, 0x4b, 0x73, 0x3a, 0xe1, 0xa4, 0x12,
|
||||
0xa4, 0xb7, 0xfa, 0xbf, 0x35, 0x58, 0x1b, 0xf8, 0x23, 0x8b, 0xd3, 0x28, 0xe2, 0x67, 0x74, 0x8c,
|
||||
0xa5, 0xdf, 0x01, 0x73, 0xf3, 0xef, 0x80, 0xf3, 0xef, 0x8a, 0xf9, 0x4b, 0xef, 0x8a, 0xfa, 0x3d,
|
||||
0x58, 0x9f, 0x8f, 0x5f, 0xe5, 0x7a, 0x0b, 0xf2, 0xf2, 0x21, 0xe5, 0xd2, 0xad, 0x21, 0xf5, 0x52,
|
||||
0x42, 0x22, 0x81, 0x9b, 0x27, 0x90, 0xdb, 0x77, 0xac, 0x31, 0x2e, 0x42, 0xae, 0xfb, 0xa0, 0xdb,
|
||||
0x42, 0x57, 0xf0, 0x2a, 0x40, 0xbb, 0xdf, 0xee, 0x1a, 0xad, 0xfb, 0xa4, 0xd1, 0x41, 0x17, 0x99,
|
||||
0x88, 0x30, 0xe8, 0xf6, 0xdb, 0xf7, 0xbb, 0xad, 0x3d, 0x74, 0x91, 0xc3, 0xcb, 0xb0, 0xd4, 0xee,
|
||||
0xef, 0x77, 0x1e, 0x34, 0x0c, 0x74, 0x51, 0xc4, 0x15, 0x28, 0xb6, 0xfb, 0x0f, 0x07, 0x0f, 0x0c,
|
||||
0xc1, 0x44, 0xb8, 0x0c, 0x85, 0x76, 0xdf, 0x68, 0x7d, 0xcd, 0x40, 0x17, 0x9b, 0x11, 0xaf, 0xd9,
|
||||
0xee, 0x36, 0xc8, 0x23, 0x74, 0x71, 0xef, 0xe6, 0xbf, 0x32, 0x90, 0x33, 0xa6, 0x3e, 0x15, 0x73,
|
||||
0xa8, 0x2b, 0xe6, 0x90, 0xf1, 0xa8, 0x27, 0x4c, 0x96, 0x20, 0xd7, 0xee, 0x1a, 0x77, 0xd1, 0x37,
|
||||
0x33, 0x18, 0x20, 0x3f, 0x90, 0xeb, 0xb7, 0x0a, 0x62, 0xdd, 0xee, 0x1a, 0x9f, 0xdf, 0x41, 0xdf,
|
||||
0xca, 0x08, 0xb5, 0x83, 0x68, 0xf3, 0xed, 0x98, 0x51, 0xdf, 0x46, 0xdf, 0x49, 0x18, 0xf5, 0x6d,
|
||||
0xf4, 0xdd, 0x98, 0x71, 0xa7, 0x8e, 0xbe, 0x97, 0x30, 0xee, 0xd4, 0xd1, 0xf7, 0x63, 0xc6, 0xce,
|
||||
0x36, 0xfa, 0x41, 0xc2, 0xd8, 0xd9, 0x46, 0x3f, 0x2c, 0x88, 0x58, 0x64, 0x24, 0x77, 0xea, 0xe8,
|
||||
0x47, 0xc5, 0x64, 0xb7, 0xb3, 0x8d, 0x7e, 0x5c, 0xc4, 0x2b, 0x50, 0x32, 0xda, 0x07, 0xad, 0xbe,
|
||||
0xd1, 0x38, 0xe8, 0xa1, 0x9f, 0x20, 0xe1, 0xe6, 0x5e, 0xc3, 0x68, 0xa1, 0x9f, 0xca, 0xa5, 0x60,
|
||||
0xa1, 0xb7, 0x91, 0x88, 0x51, 0x50, 0xe5, 0xf6, 0x1d, 0xc9, 0x79, 0xd4, 0x6a, 0x10, 0xf4, 0xb3,
|
||||
0x02, 0x2e, 0xc3, 0xd2, 0x5e, 0x6b, 0xb7, 0x7d, 0xd0, 0xe8, 0x20, 0x2c, 0x4f, 0x08, 0x54, 0x7e,
|
||||
0x7e, 0x5b, 0x2c, 0x9b, 0x9d, 0x07, 0x4d, 0xf4, 0x8b, 0x9e, 0x30, 0x78, 0xd8, 0x20, 0xbb, 0xaf,
|
||||
0x37, 0x08, 0x7a, 0xf7, 0xb6, 0x30, 0x78, 0xd8, 0x20, 0x0a, 0xaf, 0x5f, 0xf6, 0x84, 0xa0, 0x64,
|
||||
0xbd, 0x77, 0x5b, 0x38, 0xad, 0xe8, 0xbf, 0xea, 0xe1, 0x22, 0x64, 0x9b, 0x6d, 0x03, 0xfd, 0x5a,
|
||||
0x5a, 0x6b, 0x75, 0x07, 0x07, 0xe8, 0x37, 0x48, 0x10, 0xfb, 0x2d, 0x03, 0xfd, 0x56, 0x10, 0xf3,
|
||||
0xc6, 0xa0, 0xd7, 0x69, 0xa1, 0xeb, 0xcd, 0x0d, 0xa8, 0x0e, 0xbd, 0x49, 0x6d, 0xea, 0x85, 0x3c,
|
||||
0x3c, 0xa2, 0xb5, 0x33, 0x9b, 0x53, 0xc6, 0xa2, 0x17, 0xf6, 0xa3, 0x82, 0xfc, 0xb9, 0xf3, 0x9f,
|
||||
0x00, 0x00, 0x00, 0xff, 0xff, 0x4b, 0x9e, 0x76, 0xa9, 0x9b, 0x17, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -71,6 +71,8 @@ type QueryClient interface {
|
|||
// StreamHealth runs a streaming RPC to the tablet, that returns the
|
||||
// current health of the tablet on a regular basis.
|
||||
StreamHealth(ctx context.Context, in *query.StreamHealthRequest, opts ...grpc.CallOption) (Query_StreamHealthClient, error)
|
||||
// UpdateStream asks the server to return a stream of the updates that have been applied to its database.
|
||||
UpdateStream(ctx context.Context, in *query.UpdateStreamRequest, opts ...grpc.CallOption) (Query_UpdateStreamClient, error)
|
||||
}
|
||||
|
||||
type queryClient struct {
|
||||
|
@ -217,6 +219,38 @@ func (x *queryStreamHealthClient) Recv() (*query.StreamHealthResponse, error) {
|
|||
return m, nil
|
||||
}
|
||||
|
||||
func (c *queryClient) UpdateStream(ctx context.Context, in *query.UpdateStreamRequest, opts ...grpc.CallOption) (Query_UpdateStreamClient, error) {
|
||||
stream, err := grpc.NewClientStream(ctx, &_Query_serviceDesc.Streams[2], c.cc, "/queryservice.Query/UpdateStream", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &queryUpdateStreamClient{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 Query_UpdateStreamClient interface {
|
||||
Recv() (*query.UpdateStreamResponse, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type queryUpdateStreamClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *queryUpdateStreamClient) Recv() (*query.UpdateStreamResponse, error) {
|
||||
m := new(query.UpdateStreamResponse)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
// Server API for Query service
|
||||
|
||||
type QueryServer interface {
|
||||
|
@ -247,6 +281,8 @@ type QueryServer interface {
|
|||
// StreamHealth runs a streaming RPC to the tablet, that returns the
|
||||
// current health of the tablet on a regular basis.
|
||||
StreamHealth(*query.StreamHealthRequest, Query_StreamHealthServer) error
|
||||
// UpdateStream asks the server to return a stream of the updates that have been applied to its database.
|
||||
UpdateStream(*query.UpdateStreamRequest, Query_UpdateStreamServer) error
|
||||
}
|
||||
|
||||
func RegisterQueryServer(s *grpc.Server, srv QueryServer) {
|
||||
|
@ -439,6 +475,27 @@ func (x *queryStreamHealthServer) Send(m *query.StreamHealthResponse) error {
|
|||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func _Query_UpdateStream_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
m := new(query.UpdateStreamRequest)
|
||||
if err := stream.RecvMsg(m); err != nil {
|
||||
return err
|
||||
}
|
||||
return srv.(QueryServer).UpdateStream(m, &queryUpdateStreamServer{stream})
|
||||
}
|
||||
|
||||
type Query_UpdateStreamServer interface {
|
||||
Send(*query.UpdateStreamResponse) error
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type queryUpdateStreamServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *queryUpdateStreamServer) Send(m *query.UpdateStreamResponse) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
var _Query_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "queryservice.Query",
|
||||
HandlerType: (*QueryServer)(nil),
|
||||
|
@ -487,6 +544,11 @@ var _Query_serviceDesc = grpc.ServiceDesc{
|
|||
Handler: _Query_StreamHealth_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
{
|
||||
StreamName: "UpdateStream",
|
||||
Handler: _Query_UpdateStream_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
},
|
||||
Metadata: fileDescriptor0,
|
||||
}
|
||||
|
@ -494,24 +556,25 @@ var _Query_serviceDesc = grpc.ServiceDesc{
|
|||
func init() { proto.RegisterFile("queryservice.proto", fileDescriptor0) }
|
||||
|
||||
var fileDescriptor0 = []byte{
|
||||
// 294 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x74, 0x92, 0xcf, 0x4a, 0xc3, 0x40,
|
||||
0x10, 0xc6, 0xf5, 0x90, 0x2a, 0x63, 0x3c, 0x38, 0x5a, 0xff, 0xa4, 0x82, 0xe2, 0x03, 0x14, 0x51,
|
||||
0x41, 0x10, 0xbc, 0xb4, 0x08, 0x8a, 0x20, 0xd8, 0x5e, 0xbc, 0xa6, 0x61, 0xd0, 0x60, 0xd2, 0x4d,
|
||||
0x93, 0x8d, 0xe8, 0xf3, 0xf9, 0x62, 0xe2, 0x6e, 0x66, 0xb3, 0xbb, 0xae, 0xc7, 0xf9, 0x7d, 0xdf,
|
||||
0x7c, 0xcc, 0xee, 0x0c, 0xe0, 0xaa, 0xa5, 0xfa, 0xab, 0xa1, 0xfa, 0x23, 0xcf, 0x68, 0x5c, 0xd5,
|
||||
0x42, 0x0a, 0x8c, 0x6d, 0x96, 0x6c, 0xa9, 0x4a, 0x4b, 0x17, 0xdf, 0x11, 0x44, 0xcf, 0xbf, 0x35,
|
||||
0xde, 0xc0, 0xc6, 0xdd, 0x27, 0x65, 0xad, 0x24, 0x1c, 0x8e, 0xb5, 0xa5, 0xab, 0x67, 0xb4, 0x6a,
|
||||
0xa9, 0x91, 0xc9, 0xbe, 0x8f, 0x9b, 0x4a, 0x2c, 0x1b, 0x3a, 0x5b, 0xc3, 0x07, 0x88, 0x3b, 0x38,
|
||||
0x49, 0x65, 0xf6, 0x86, 0x89, 0xeb, 0x54, 0x90, 0x53, 0x46, 0x41, 0xcd, 0x44, 0x3d, 0xc1, 0xf6,
|
||||
0x5c, 0xd6, 0x94, 0x96, 0x3c, 0x0c, 0xfb, 0x1d, 0xca, 0x61, 0xc7, 0x61, 0x91, 0xd3, 0xce, 0xd7,
|
||||
0xf1, 0x0a, 0xa2, 0x09, 0xbd, 0xe6, 0x4b, 0xdc, 0xed, 0xac, 0xaa, 0xe2, 0xfe, 0x3d, 0x17, 0x9a,
|
||||
0x29, 0xae, 0x61, 0x30, 0x15, 0x65, 0x99, 0x4b, 0x64, 0x87, 0x2e, 0xb9, 0x6f, 0xe8, 0x51, 0xd3,
|
||||
0x78, 0x0b, 0x9b, 0x33, 0x51, 0x14, 0x8b, 0x34, 0x7b, 0x47, 0xfe, 0x2f, 0x06, 0xdc, 0x7c, 0xf0,
|
||||
0x87, 0xdb, 0x1f, 0xa9, 0x46, 0xe1, 0xc7, 0x27, 0xf6, 0x7c, 0xde, 0xdb, 0x47, 0x41, 0xcd, 0x44,
|
||||
0xbd, 0xc0, 0x8e, 0xad, 0xe8, 0xc5, 0x9c, 0x04, 0x7a, 0x9c, 0xed, 0x9c, 0xfe, 0x6f, 0x30, 0xc9,
|
||||
0x53, 0x80, 0x79, 0x55, 0xe4, 0x52, 0xdf, 0xcd, 0x21, 0xaf, 0xc0, 0x20, 0xce, 0x3a, 0x0a, 0x28,
|
||||
0x26, 0xe4, 0x11, 0x62, 0xbd, 0xb4, 0x7b, 0x4a, 0x0b, 0xd9, 0x9f, 0x8c, 0x0d, 0xfd, 0x97, 0xba,
|
||||
0x5a, 0xbf, 0xe4, 0xc5, 0x40, 0x1d, 0xf3, 0xe5, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc1, 0xe2,
|
||||
0xe9, 0x0d, 0xfd, 0x02, 0x00, 0x00,
|
||||
// 312 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x74, 0x93, 0xdf, 0x4a, 0xf3, 0x40,
|
||||
0x10, 0xc5, 0xbf, 0xef, 0xa2, 0x51, 0xc6, 0x78, 0xe1, 0x68, 0xfd, 0x93, 0x0a, 0x8a, 0x0f, 0x50,
|
||||
0x44, 0x05, 0x41, 0xf0, 0xa6, 0x45, 0x50, 0x0a, 0x82, 0x2d, 0x82, 0xb7, 0x69, 0x1c, 0x34, 0x98,
|
||||
0x74, 0xd3, 0x64, 0x23, 0xfa, 0x16, 0x3e, 0xb2, 0xb8, 0x9b, 0xd9, 0xee, 0xae, 0xeb, 0xe5, 0xfc,
|
||||
0xce, 0x99, 0xc3, 0x4c, 0x66, 0x03, 0xb8, 0x6c, 0xa9, 0xfe, 0x6c, 0xa8, 0x7e, 0xcf, 0x33, 0x1a,
|
||||
0x56, 0xb5, 0x90, 0x02, 0x63, 0x9b, 0x25, 0x1b, 0xaa, 0xd2, 0xd2, 0xd9, 0x57, 0x04, 0xbd, 0x87,
|
||||
0x9f, 0x1a, 0xaf, 0x60, 0xed, 0xe6, 0x83, 0xb2, 0x56, 0x12, 0xf6, 0x87, 0xda, 0xd2, 0xd5, 0x53,
|
||||
0x5a, 0xb6, 0xd4, 0xc8, 0x64, 0xd7, 0xc7, 0x4d, 0x25, 0x16, 0x0d, 0x9d, 0xfc, 0xc3, 0x3b, 0x88,
|
||||
0x3b, 0x38, 0x4a, 0x65, 0xf6, 0x8a, 0x89, 0xeb, 0x54, 0x90, 0x53, 0x06, 0x41, 0xcd, 0x44, 0xdd,
|
||||
0xc3, 0xe6, 0x4c, 0xd6, 0x94, 0x96, 0x3c, 0x0c, 0xfb, 0x1d, 0xca, 0x61, 0x87, 0x61, 0x91, 0xd3,
|
||||
0x4e, 0xff, 0xe3, 0x05, 0xf4, 0x46, 0xf4, 0x92, 0x2f, 0x70, 0xbb, 0xb3, 0xaa, 0x8a, 0xfb, 0x77,
|
||||
0x5c, 0x68, 0xa6, 0xb8, 0x84, 0x68, 0x2c, 0xca, 0x32, 0x97, 0xc8, 0x0e, 0x5d, 0x72, 0x5f, 0xdf,
|
||||
0xa3, 0xa6, 0xf1, 0x1a, 0xd6, 0xa7, 0xa2, 0x28, 0xe6, 0x69, 0xf6, 0x86, 0xfc, 0xbd, 0x18, 0x70,
|
||||
0xf3, 0xde, 0x2f, 0x6e, 0x7f, 0x48, 0x35, 0x0a, 0x2f, 0x9f, 0xd8, 0xf3, 0x79, 0xbb, 0x0f, 0x82,
|
||||
0x9a, 0x89, 0x7a, 0x82, 0x2d, 0x5b, 0xd1, 0x87, 0x39, 0x0a, 0xf4, 0x38, 0xd7, 0x39, 0xfe, 0xdb,
|
||||
0x60, 0x92, 0xc7, 0x00, 0xb3, 0xaa, 0xc8, 0xa5, 0x7e, 0x37, 0xfb, 0x7c, 0x02, 0x83, 0x38, 0xeb,
|
||||
0x20, 0xa0, 0x98, 0x90, 0x09, 0xc4, 0xfa, 0x68, 0xb7, 0x94, 0x16, 0x72, 0xf5, 0x64, 0x6c, 0xe8,
|
||||
0x6f, 0xea, 0x6a, 0xd6, 0x91, 0x27, 0x10, 0x3f, 0x56, 0xcf, 0xa9, 0x24, 0xed, 0x30, 0x61, 0x36,
|
||||
0xf4, 0xc3, 0x5c, 0x6d, 0x15, 0x36, 0x8f, 0xd4, 0x9f, 0x71, 0xfe, 0x1d, 0x00, 0x00, 0xff, 0xff,
|
||||
0x29, 0x7f, 0x6d, 0xa7, 0x4a, 0x03, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -81,11 +81,6 @@ func (fbc *fakeBinlogClient) Dial(tablet *topodatapb.Tablet, connTimeout time.Du
|
|||
func (fbc *fakeBinlogClient) Close() {
|
||||
}
|
||||
|
||||
// ServeUpdateStream is part of the binlogplayer.Client interface
|
||||
func (fbc *fakeBinlogClient) ServeUpdateStream(ctx context.Context, position string) (binlogplayer.StreamEventStream, error) {
|
||||
return nil, fmt.Errorf("Should never be called")
|
||||
}
|
||||
|
||||
type testStreamEventAdapter struct {
|
||||
c chan *binlogdatapb.BinlogTransaction
|
||||
ctx context.Context
|
||||
|
@ -164,15 +159,6 @@ func (ftc *fakeTabletConn) BeginExecuteBatch(ctx context.Context, target *queryp
|
|||
return nil, 0, fmt.Errorf("not implemented in this test")
|
||||
}
|
||||
|
||||
// Close is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) Close() {
|
||||
}
|
||||
|
||||
// Tablet is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) Tablet() *topodatapb.Tablet {
|
||||
return ftc.tablet
|
||||
}
|
||||
|
||||
// SplitQuery is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) SplitQuery(ctx context.Context, target *querypb.Target, query querytypes.BoundQuery, splitColumn string, splitCount int64) ([]querytypes.QuerySplit, error) {
|
||||
return nil, fmt.Errorf("not implemented in this test")
|
||||
|
@ -234,6 +220,20 @@ func (ftc *fakeTabletConn) StreamHealth(ctx context.Context) (tabletconn.StreamH
|
|||
}, nil
|
||||
}
|
||||
|
||||
// UpdateStream is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
return nil, fmt.Errorf("not implemented in this test")
|
||||
}
|
||||
|
||||
// Close is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) Close() {
|
||||
}
|
||||
|
||||
// Tablet is part of the TabletConn interface
|
||||
func (ftc *fakeTabletConn) Tablet() *topodatapb.Tablet {
|
||||
return ftc.tablet
|
||||
}
|
||||
|
||||
// createSourceTablet is a helper method to create the source tablet
|
||||
// in the given keyspace/shard.
|
||||
func createSourceTablet(t *testing.T, name string, ts topo.Server, keyspace, shard string) {
|
||||
|
|
|
@ -244,6 +244,23 @@ func (q *query) StreamHealth(request *querypb.StreamHealthRequest, stream querys
|
|||
return q.server.StreamHealthUnregister(id)
|
||||
}
|
||||
|
||||
// UpdateStream is part of the queryservice.QueryServer interface
|
||||
func (q *query) UpdateStream(request *querypb.UpdateStreamRequest, stream queryservicepb.Query_UpdateStreamServer) (err error) {
|
||||
defer q.server.HandlePanic(&err)
|
||||
ctx := callerid.NewContext(callinfo.GRPCCallInfo(stream.Context()),
|
||||
request.EffectiveCallerId,
|
||||
request.ImmediateCallerId,
|
||||
)
|
||||
if err := q.server.UpdateStream(ctx, request.Target, request.Position, request.Timestamp, func(reply *querypb.StreamEvent) error {
|
||||
return stream.Send(&querypb.UpdateStreamResponse{
|
||||
Event: reply,
|
||||
})
|
||||
}); err != nil {
|
||||
return vterrors.ToGRPCError(err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Register registers the implementation on the provide gRPC Server.
|
||||
func Register(s *grpc.Server, server queryservice.QueryService) {
|
||||
queryservicepb.RegisterQueryServer(s, &query{server})
|
||||
|
|
|
@ -463,6 +463,44 @@ func (conn *gRPCQueryClient) StreamHealth(ctx context.Context) (tabletconn.Strea
|
|||
return conn.c.StreamHealth(ctx, &querypb.StreamHealthRequest{})
|
||||
}
|
||||
|
||||
type updateStreamAdapter struct {
|
||||
stream queryservicepb.Query_UpdateStreamClient
|
||||
}
|
||||
|
||||
func (a *updateStreamAdapter) Recv() (*querypb.StreamEvent, error) {
|
||||
r, err := a.stream.Recv()
|
||||
switch err {
|
||||
case nil:
|
||||
return r.Event, nil
|
||||
case io.EOF:
|
||||
return nil, err
|
||||
default:
|
||||
return nil, tabletconn.TabletErrorFromGRPC(err)
|
||||
}
|
||||
}
|
||||
|
||||
// UpdateStream starts a streaming query to VTTablet.
|
||||
func (conn *gRPCQueryClient) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
conn.mu.RLock()
|
||||
defer conn.mu.RUnlock()
|
||||
if conn.cc == nil {
|
||||
return nil, tabletconn.ConnClosed
|
||||
}
|
||||
|
||||
req := &querypb.UpdateStreamRequest{
|
||||
Target: target,
|
||||
EffectiveCallerId: callerid.EffectiveCallerIDFromContext(ctx),
|
||||
ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx),
|
||||
Position: position,
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
stream, err := conn.c.UpdateStream(ctx, req)
|
||||
if err != nil {
|
||||
return nil, tabletconn.TabletErrorFromGRPC(err)
|
||||
}
|
||||
return &updateStreamAdapter{stream: stream}, err
|
||||
}
|
||||
|
||||
// Close closes underlying gRPC channel.
|
||||
func (conn *gRPCQueryClient) Close() {
|
||||
conn.mu.Lock()
|
||||
|
|
|
@ -89,6 +89,11 @@ func (e *ErrorQueryService) StreamHealthUnregister(int) error {
|
|||
return fmt.Errorf("ErrorQueryService does not implement any method")
|
||||
}
|
||||
|
||||
// UpdateStream is part of QueryService interface
|
||||
func (e *ErrorQueryService) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, sendReply func(*querypb.StreamEvent) error) error {
|
||||
return fmt.Errorf("ErrorQueryService does not implement any method")
|
||||
}
|
||||
|
||||
// HandlePanic is part of QueryService interface
|
||||
func (e *ErrorQueryService) HandlePanic(*error) {
|
||||
}
|
||||
|
|
|
@ -64,6 +64,9 @@ type QueryService interface {
|
|||
// StreamHealthUnregister unregisters a listener for StreamHealth
|
||||
StreamHealthUnregister(int) error
|
||||
|
||||
// UpdateStream streams updates from the provided position or timestamp.
|
||||
UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, sendReply func(*querypb.StreamEvent) error) error
|
||||
|
||||
// Helper for RPC panic handling: call this in a defer statement
|
||||
// at the beginning of each RPC handling method.
|
||||
HandlePanic(*error)
|
||||
|
|
|
@ -162,6 +162,16 @@ func (_mr *_MockQueryServiceRecorder) StreamHealthUnregister(arg0 interface{}) *
|
|||
return _mr.mock.ctrl.RecordCall(_mr.mock, "StreamHealthUnregister", arg0)
|
||||
}
|
||||
|
||||
func (_m *MockQueryService) UpdateStream(ctx context.Context, target *query.Target, position string, timestamp int64, sendReply func(*query.StreamEvent) error) error {
|
||||
ret := _m.ctrl.Call(_m, "UpdateStream", ctx, target, position, timestamp, sendReply)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
func (_mr *_MockQueryServiceRecorder) UpdateStream(arg0, arg1, arg2, arg3, arg4 interface{}) *gomock.Call {
|
||||
return _mr.mock.ctrl.RecordCall(_mr.mock, "UpdateStream", arg0, arg1, arg2, arg3, arg4)
|
||||
}
|
||||
|
||||
func (_m *MockQueryService) HandlePanic(_param0 *error) {
|
||||
_m.ctrl.Call(_m, "HandlePanic", _param0)
|
||||
}
|
||||
|
|
|
@ -265,6 +265,12 @@ func (sbc *SandboxConn) StreamHealth(ctx context.Context) (tabletconn.StreamHeal
|
|||
return nil, fmt.Errorf("Not implemented in test")
|
||||
}
|
||||
|
||||
// UpdateStream is part of the TabletConn interface.
|
||||
func (sbc *SandboxConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
// FIXME(alainjobart) implement, use in vtgate tests.
|
||||
return nil, fmt.Errorf("Not implemented in test")
|
||||
}
|
||||
|
||||
// Close does not change ExecCount
|
||||
func (sbc *SandboxConn) Close() {
|
||||
}
|
||||
|
|
|
@ -48,12 +48,20 @@ type OperationalError string
|
|||
|
||||
func (e OperationalError) Error() string { return string(e) }
|
||||
|
||||
// StreamHealthReader defines the interface for a reader to read StreamHealth messages.
|
||||
// StreamHealthReader defines the interface for a reader to read
|
||||
// StreamHealth messages.
|
||||
type StreamHealthReader interface {
|
||||
// Recv reads one StreamHealthResponse.
|
||||
Recv() (*querypb.StreamHealthResponse, error)
|
||||
}
|
||||
|
||||
// StreamEventReader defines the interface for a reader to read
|
||||
// StreamEvent messages.
|
||||
type StreamEventReader interface {
|
||||
// Recv reads one StreamEvent.
|
||||
Recv() (*querypb.StreamEvent, error)
|
||||
}
|
||||
|
||||
// In all the following calls, context is an opaque structure that may
|
||||
// carry data related to the call. For instance, if an incoming RPC
|
||||
// call is responsible for these outgoing calls, and the incoming
|
||||
|
@ -129,6 +137,13 @@ type TabletConn interface {
|
|||
|
||||
// StreamHealth starts a streaming RPC for VTTablet health status updates.
|
||||
StreamHealth(ctx context.Context) (StreamHealthReader, error)
|
||||
|
||||
// UpdateStream asks for a stream of updates from a server.
|
||||
// It returns a StreamEventReader to get results from. If
|
||||
// error is non-nil, it means that the UpdateStream failed to
|
||||
// send the request. Otherwise, you can pull values from the
|
||||
// StreamEventReader until io.EOF, or any other error.
|
||||
UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (StreamEventReader, error)
|
||||
}
|
||||
|
||||
var dialers = make(map[string]TabletDialer)
|
||||
|
|
|
@ -33,6 +33,7 @@ type FakeQueryService struct {
|
|||
// these fields are used to simulate and synchronize on panics
|
||||
Panics bool
|
||||
StreamExecutePanicsEarly bool
|
||||
UpdateStreamPanicsEarly bool
|
||||
PanicWait chan struct{}
|
||||
|
||||
// ExpectedTransactionID is what transactionID to expect for Execute
|
||||
|
@ -515,6 +516,71 @@ func (f *FakeQueryService) StreamHealthUnregister(int) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
const UpdateStreamPosition = "update stream position"
|
||||
|
||||
const UpdateStreamTimestamp = 123654
|
||||
|
||||
var UpdateStreamStreamEvent1 = querypb.StreamEvent{
|
||||
Statements: []*querypb.StreamEvent_Statement{
|
||||
{
|
||||
Category: querypb.StreamEvent_Statement_DML,
|
||||
TableName: "table1",
|
||||
},
|
||||
},
|
||||
EventToken: &querypb.EventToken{
|
||||
Timestamp: 789654,
|
||||
Shard: "shard1",
|
||||
Position: "streaming position 1",
|
||||
},
|
||||
}
|
||||
|
||||
var UpdateStreamStreamEvent2 = querypb.StreamEvent{
|
||||
Statements: []*querypb.StreamEvent_Statement{
|
||||
{
|
||||
Category: querypb.StreamEvent_Statement_DML,
|
||||
TableName: "table2",
|
||||
},
|
||||
},
|
||||
EventToken: &querypb.EventToken{
|
||||
Timestamp: 789655,
|
||||
Shard: "shard1",
|
||||
Position: "streaming position 2",
|
||||
},
|
||||
}
|
||||
|
||||
// UpdateStream is part of the queryservice.QueryService interface
|
||||
func (f *FakeQueryService) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, sendReply func(*querypb.StreamEvent) error) error {
|
||||
if f.Panics && f.UpdateStreamPanicsEarly {
|
||||
panic(fmt.Errorf("test-triggered panic early"))
|
||||
}
|
||||
if position != UpdateStreamPosition {
|
||||
f.t.Errorf("invalid UpdateStream.position: got %v expected %v", position, UpdateStreamPosition)
|
||||
}
|
||||
if timestamp != UpdateStreamTimestamp {
|
||||
f.t.Errorf("invalid UpdateStream.timestamp: got %v expected %v", timestamp, UpdateStreamTimestamp)
|
||||
}
|
||||
f.checkTargetCallerID(ctx, "UpdateStream", target)
|
||||
if err := sendReply(&UpdateStreamStreamEvent1); err != nil {
|
||||
f.t.Errorf("sendReply1 failed: %v", err)
|
||||
}
|
||||
if f.Panics && !f.UpdateStreamPanicsEarly {
|
||||
// wait until the client gets the response, then panics
|
||||
<-f.PanicWait
|
||||
panic(fmt.Errorf("test-triggered panic late"))
|
||||
}
|
||||
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
|
||||
return f.TabletError
|
||||
}
|
||||
if err := sendReply(&UpdateStreamStreamEvent2); err != nil {
|
||||
f.t.Errorf("sendReply2 failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// CreateFakeServer returns the fake server for the tests
|
||||
func CreateFakeServer(t *testing.T) *FakeQueryService {
|
||||
return &FakeQueryService{
|
||||
|
|
|
@ -547,6 +547,101 @@ func testStreamHealthPanics(t *testing.T, conn tabletconn.TabletConn, f *FakeQue
|
|||
})
|
||||
}
|
||||
|
||||
func testUpdateStream(t *testing.T, conn tabletconn.TabletConn, f *FakeQueryService) {
|
||||
t.Log("testUpdateStream")
|
||||
ctx := context.Background()
|
||||
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
|
||||
stream, err := conn.UpdateStream(ctx, TestTarget, UpdateStreamPosition, UpdateStreamTimestamp)
|
||||
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, UpdateStreamStreamEvent1) {
|
||||
t.Errorf("Unexpected result1 from UpdateStream: got %v wanted %v", qr, UpdateStreamStreamEvent1)
|
||||
}
|
||||
qr, err = stream.Recv()
|
||||
if err != nil {
|
||||
t.Fatalf("UpdateStream failed: cannot read result2: %v", err)
|
||||
}
|
||||
if !reflect.DeepEqual(*qr, UpdateStreamStreamEvent2) {
|
||||
t.Errorf("Unexpected result2 from UpdateStream: got %v wanted %v", qr, UpdateStreamStreamEvent2)
|
||||
}
|
||||
qr, err = stream.Recv()
|
||||
if err != io.EOF {
|
||||
t.Fatalf("UpdateStream errFunc failed: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func testUpdateStreamError(t *testing.T, conn tabletconn.TabletConn, f *FakeQueryService) {
|
||||
t.Log("testUpdateStreamError")
|
||||
f.HasError = true
|
||||
testErrorHelper(t, f, "UpdateStream", func(ctx context.Context) error {
|
||||
f.ErrorWait = make(chan struct{})
|
||||
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
|
||||
stream, err := conn.UpdateStream(ctx, TestTarget, UpdateStreamPosition, UpdateStreamTimestamp)
|
||||
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, UpdateStreamStreamEvent1) {
|
||||
t.Errorf("Unexpected result1 from UpdateStream: got %v wanted %v", qr, UpdateStreamStreamEvent1)
|
||||
}
|
||||
// signal to the server that the first result has been received
|
||||
close(f.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")
|
||||
}
|
||||
return err
|
||||
})
|
||||
f.HasError = false
|
||||
}
|
||||
|
||||
func testUpdateStreamPanics(t *testing.T, conn tabletconn.TabletConn, f *FakeQueryService) {
|
||||
t.Log("testUpdateStreamPanics")
|
||||
// early panic is before sending the Fields, that is returned
|
||||
// by the UpdateStream call itself, or as the first error
|
||||
// by ErrFunc
|
||||
f.UpdateStreamPanicsEarly = true
|
||||
testPanicHelper(t, f, "UpdateStream.Early", func(ctx context.Context) error {
|
||||
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
|
||||
stream, err := conn.UpdateStream(ctx, TestTarget, UpdateStreamPosition, UpdateStreamTimestamp)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, err = stream.Recv()
|
||||
return err
|
||||
})
|
||||
|
||||
// late panic is after sending Fields
|
||||
f.UpdateStreamPanicsEarly = false
|
||||
testPanicHelper(t, f, "UpdateStream.Late", func(ctx context.Context) error {
|
||||
f.PanicWait = make(chan struct{})
|
||||
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
|
||||
stream, err := conn.UpdateStream(ctx, TestTarget, UpdateStreamPosition, UpdateStreamTimestamp)
|
||||
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, UpdateStreamStreamEvent1) {
|
||||
t.Errorf("Unexpected result1 from UpdateStream: got %v wanted %v", qr, UpdateStreamStreamEvent1)
|
||||
}
|
||||
close(f.PanicWait)
|
||||
_, err = stream.Recv()
|
||||
return err
|
||||
})
|
||||
}
|
||||
|
||||
// TestSuite runs all the tests.
|
||||
// If fake.TestingGateway is set, we only test the calls that can go through
|
||||
// a gateway.
|
||||
|
@ -563,6 +658,7 @@ func TestSuite(t *testing.T, protocol string, tablet *topodatapb.Tablet, fake *F
|
|||
testBeginExecuteBatch,
|
||||
testSplitQuery,
|
||||
testSplitQueryV2,
|
||||
testUpdateStream,
|
||||
|
||||
// error test cases
|
||||
testBeginError,
|
||||
|
@ -576,6 +672,7 @@ func TestSuite(t *testing.T, protocol string, tablet *topodatapb.Tablet, fake *F
|
|||
testBeginExecuteBatchErrorInBegin,
|
||||
testBeginExecuteBatchErrorInExecuteBatch,
|
||||
testSplitQueryError,
|
||||
testUpdateStreamError,
|
||||
|
||||
// panic test cases
|
||||
testBeginPanics,
|
||||
|
@ -587,6 +684,7 @@ func TestSuite(t *testing.T, protocol string, tablet *topodatapb.Tablet, fake *F
|
|||
testExecuteBatchPanics,
|
||||
testBeginExecuteBatchPanics,
|
||||
testSplitQueryPanics,
|
||||
testUpdateStreamPanics,
|
||||
}
|
||||
|
||||
if !fake.TestingGateway {
|
||||
|
|
|
@ -22,9 +22,11 @@ import (
|
|||
"github.com/youtube/vitess/go/stats"
|
||||
"github.com/youtube/vitess/go/sync2"
|
||||
"github.com/youtube/vitess/go/tb"
|
||||
"github.com/youtube/vitess/go/vt/binlog"
|
||||
"github.com/youtube/vitess/go/vt/dbconfigs"
|
||||
"github.com/youtube/vitess/go/vt/dbconnpool"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/schema"
|
||||
"github.com/youtube/vitess/go/vt/sqlparser"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver/queryservice"
|
||||
|
@ -95,15 +97,14 @@ type TabletServer struct {
|
|||
begins sync.WaitGroup
|
||||
|
||||
// The following variables should be initialized only once
|
||||
// before starting the tabletserver. For backward compatibility,
|
||||
// we temporarily allow them to be changed until the migration
|
||||
// to the new API is complete.
|
||||
// before starting the tabletserver.
|
||||
dbconfigs dbconfigs.DBConfigs
|
||||
mysqld mysqlctl.MysqlDaemon
|
||||
|
||||
// The following variables should only be accessed within
|
||||
// the context of a startRequest-endRequest.
|
||||
qe *QueryEngine
|
||||
qe *QueryEngine
|
||||
updateStreamList *binlog.StreamList
|
||||
|
||||
// checkMySQLThrottler is used to throttle the number of
|
||||
// requests sent to CheckMySQL.
|
||||
|
@ -147,6 +148,7 @@ func NewTabletServer(config Config) *TabletServer {
|
|||
history: history.New(10),
|
||||
}
|
||||
tsv.qe = NewQueryEngine(tsv, config)
|
||||
tsv.updateStreamList = &binlog.StreamList{}
|
||||
if config.EnablePublishStats {
|
||||
stats.Publish(config.StatsPrefix+"TabletState", stats.IntFunc(func() int64 {
|
||||
tsv.mu.Lock()
|
||||
|
@ -349,6 +351,7 @@ func (tsv *TabletServer) fullStart() (err error) {
|
|||
if x := recover(); x != nil {
|
||||
log.Errorf("Could not start tabletserver: %v", x)
|
||||
tsv.qe.Close()
|
||||
tsv.updateStreamList.Stop()
|
||||
tsv.transition(StateNotConnected)
|
||||
err = x.(error)
|
||||
}
|
||||
|
@ -361,6 +364,7 @@ func (tsv *TabletServer) fullStart() (err error) {
|
|||
c.Close()
|
||||
|
||||
tsv.qe.Open(tsv.dbconfigs)
|
||||
tsv.updateStreamList.Init()
|
||||
return tsv.serveNewType()
|
||||
}
|
||||
|
||||
|
@ -369,6 +373,7 @@ func (tsv *TabletServer) serveNewType() (err error) {
|
|||
if x := recover(); x != nil {
|
||||
log.Errorf("Could not start tabletserver: %v", x)
|
||||
tsv.qe.Close()
|
||||
tsv.updateStreamList.Stop()
|
||||
tsv.transition(StateNotConnected)
|
||||
err = x.(error)
|
||||
}
|
||||
|
@ -414,6 +419,7 @@ func (tsv *TabletServer) waitForShutdown() {
|
|||
tsv.begins.Wait()
|
||||
tsv.qe.WaitForTxEmpty()
|
||||
tsv.qe.streamQList.TerminateAll()
|
||||
tsv.updateStreamList.Stop()
|
||||
tsv.requests.Wait()
|
||||
}
|
||||
|
||||
|
@ -1104,6 +1110,47 @@ func (tsv *TabletServer) StreamHealthUnregister(id int) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// UpdateStream streams binlog events.
|
||||
func (tsv *TabletServer) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, sendReply func(*querypb.StreamEvent) error) error {
|
||||
// Parse the position if needed.
|
||||
var p replication.Position
|
||||
var err error
|
||||
if timestamp == 0 {
|
||||
p, err = replication.DecodePosition(position)
|
||||
if err != nil {
|
||||
return NewTabletError(vtrpcpb.ErrorCode_BAD_INPUT, "cannot parse position: %v", err)
|
||||
}
|
||||
} else if position != "" {
|
||||
return NewTabletError(vtrpcpb.ErrorCode_BAD_INPUT, "only one of position and timestamp should be specified")
|
||||
}
|
||||
|
||||
// Validate proper target is used.
|
||||
if err = tsv.startRequest(target, false, false); err != nil {
|
||||
return err
|
||||
}
|
||||
defer tsv.endRequest(false)
|
||||
|
||||
s := binlog.NewEventStreamer(tsv.dbconfigs.App.DbName, tsv.mysqld, p, timestamp, func(event *querypb.StreamEvent) error {
|
||||
return sendReply(event)
|
||||
})
|
||||
|
||||
// Create a cancelable wrapping context.
|
||||
streamCtx, streamCancel := context.WithCancel(ctx)
|
||||
i := tsv.updateStreamList.Add(streamCancel)
|
||||
defer tsv.updateStreamList.Delete(i)
|
||||
|
||||
// And stream with it.
|
||||
err = s.Stream(streamCtx)
|
||||
switch err {
|
||||
case mysqlctl.ErrBinlogUnavailable:
|
||||
return NewTabletError(vtrpcpb.ErrorCode_QUERY_NOT_SERVED, "%v", err)
|
||||
case nil:
|
||||
return nil
|
||||
default:
|
||||
return NewTabletError(vtrpcpb.ErrorCode_INTERNAL_ERROR, "%v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// HandlePanic is part of the queryservice.QueryService interface
|
||||
func (tsv *TabletServer) HandlePanic(err *error) {
|
||||
if x := recover(); x != nil {
|
||||
|
|
|
@ -245,6 +245,21 @@ func (dg *discoveryGateway) SplitQueryV2(
|
|||
return
|
||||
}
|
||||
|
||||
// UpdateStream starts an update stream for the specified keyspace,
|
||||
// shard, and tablet type.
|
||||
func (dg *discoveryGateway) UpdateStream(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
var stream tabletconn.StreamEventReader
|
||||
err := dg.withRetry(ctx, keyspace, shard, tabletType, func(conn tabletconn.TabletConn, target *querypb.Target) error {
|
||||
var err error
|
||||
stream, err = conn.UpdateStream(ctx, target, position, timestamp)
|
||||
return err
|
||||
}, 0, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return stream, nil
|
||||
}
|
||||
|
||||
// Close shuts down underlying connections.
|
||||
func (dg *discoveryGateway) Close(ctx context.Context) error {
|
||||
for _, ctw := range dg.tabletsWatchers {
|
||||
|
|
|
@ -16,6 +16,7 @@ import (
|
|||
"github.com/youtube/vitess/go/sqltypes"
|
||||
"github.com/youtube/vitess/go/vt/discovery"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver/querytypes"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
|
||||
querypb "github.com/youtube/vitess/go/vt/proto/query"
|
||||
|
@ -77,6 +78,10 @@ type Gateway interface {
|
|||
numRowsPerQueryPart int64,
|
||||
algorithm querypb.SplitQueryRequest_Algorithm) ([]querytypes.QuerySplit, error)
|
||||
|
||||
// UpdateStream asks for an update stream query for the
|
||||
// specified keyspace, shard, and tablet type.
|
||||
UpdateStream(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, position string, timestamp int64) (tabletconn.StreamEventReader, error)
|
||||
|
||||
// WaitForTablets asks the gateway to wait for the provided
|
||||
// tablets types to be available. It the context is canceled
|
||||
// before the end, it should return ctx.Err().
|
||||
|
|
|
@ -278,6 +278,20 @@ func (lg *l2VTGateGateway) SplitQueryV2(
|
|||
return
|
||||
}
|
||||
|
||||
// UpdateStream request an update stream for the specified keyspace, shard, and tablet type.
|
||||
func (lg *l2VTGateGateway) UpdateStream(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
var stream tabletconn.StreamEventReader
|
||||
err := lg.withRetry(ctx, keyspace, shard, tabletType, func(conn *l2VTGateConn, target *querypb.Target) error {
|
||||
var err error
|
||||
stream, err = conn.conn.UpdateStream(ctx, target, position, timestamp)
|
||||
return err
|
||||
}, 0, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return stream, nil
|
||||
}
|
||||
|
||||
// Close shuts down underlying connections.
|
||||
func (lg *l2VTGateGateway) Close(ctx context.Context) error {
|
||||
lg.mu.Lock()
|
||||
|
|
|
@ -84,6 +84,10 @@ func (ga *gatewayAdapter) StreamHealth(ctx context.Context) (tabletconn.StreamHe
|
|||
return nil, fmt.Errorf("Not Implemented")
|
||||
}
|
||||
|
||||
func (ga *gatewayAdapter) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64) (tabletconn.StreamEventReader, error) {
|
||||
return ga.g.UpdateStream(ctx, target.Keyspace, target.Shard, target.TabletType, position, timestamp)
|
||||
}
|
||||
|
||||
// CreateFakeServers returns the servers to use for these tests
|
||||
func CreateFakeServers(t *testing.T) (*tabletconntest.FakeQueryService, topo.Server, string) {
|
||||
cell := "local"
|
||||
|
|
|
@ -140,6 +140,26 @@ func (l *L2VTGate) StreamHealthUnregister(int) error {
|
|||
return fmt.Errorf("L2VTGate does not provide health status")
|
||||
}
|
||||
|
||||
// UpdateStream is part of the queryservice.QueryService interface
|
||||
func (l *L2VTGate) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, sendReply func(*querypb.StreamEvent) error) error {
|
||||
stream, err := l.gateway.UpdateStream(ctx, target.Keyspace, target.Shard, target.TabletType, position, timestamp)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for {
|
||||
r, err := stream.Recv()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
if err := sendReply(r); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// HandlePanic is part of the queryservice.QueryService interface
|
||||
func (l *L2VTGate) HandlePanic(err *error) {
|
||||
if x := recover(); x != nil {
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
// This file contains all the types and servers necessary to make
|
||||
// RPC calls to VtTablet for the binlog protocol.
|
||||
// RPC calls to VtTablet for the binlog protocol, used by filtered
|
||||
// replication only.
|
||||
|
||||
syntax = "proto3";
|
||||
|
||||
|
@ -58,52 +59,6 @@ message BinlogTransaction {
|
|||
query.EventToken event_token = 4;
|
||||
}
|
||||
|
||||
// StreamEvent describes an update stream event inside the binlogs.
|
||||
// It is streamed by vttablet for the Update Stream feature.
|
||||
// For each BinlogTransaction (transactional unit), we will send any
|
||||
// number of SE_ERR, SE_DML and SE_DDL statements, followed by one SE_POS.
|
||||
message StreamEvent {
|
||||
// the category of this event
|
||||
enum Category {
|
||||
SE_ERR = 0;
|
||||
SE_DML = 1;
|
||||
SE_DDL = 2;
|
||||
SE_POS = 3;
|
||||
}
|
||||
Category category = 1;
|
||||
|
||||
// table_name, primary_key_fields and primary_key_values are set for SE_DML.
|
||||
string table_name = 2;
|
||||
repeated query.Field primary_key_fields = 3;
|
||||
repeated query.Row primary_key_values = 4;
|
||||
|
||||
// sql is set for SE_DDL or SE_ERR.
|
||||
bytes sql = 5;
|
||||
|
||||
// DEPRECATED (replaced by event_token): timestamp is the timestamp of the statements (set for all categories).
|
||||
// int64 timestamp = 6;
|
||||
reserved 6;
|
||||
|
||||
// DEPRECATED (replaced by event_token): Transaction ID after this
|
||||
// statement was applied (set for all categories).
|
||||
// string transaction_id = 7;
|
||||
reserved 7;
|
||||
|
||||
// The Event Token for this event. Only set for SE_POS.
|
||||
query.EventToken event_token = 8;
|
||||
}
|
||||
|
||||
// StreamUpdateRequest is the payload to StreamUpdate
|
||||
message StreamUpdateRequest{
|
||||
// where to start
|
||||
string position = 1;
|
||||
}
|
||||
|
||||
// StreamUpdateResponse is the response from StreamUpdate
|
||||
message StreamUpdateResponse{
|
||||
StreamEvent stream_event = 1;
|
||||
}
|
||||
|
||||
// StreamKeyRangeRequest is the payload to StreamKeyRange
|
||||
message StreamKeyRangeRequest {
|
||||
// where to start
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
// This file contains all the types and servers necessary to make
|
||||
// RPC calls to VtTablet for the binlog protocol.
|
||||
// This file contains the UpdateStream service definition, necessary
|
||||
// to make RPC calls to VtTablet for the binlog protocol, used by
|
||||
// filtered replication only.
|
||||
|
||||
syntax = "proto3";
|
||||
|
||||
|
@ -9,10 +10,6 @@ import "binlogdata.proto";
|
|||
|
||||
// UpdateStream is the RPC version of binlog.UpdateStream.
|
||||
service UpdateStream {
|
||||
|
||||
// StreamUpdate streams the binlog events, to know which objects have changed.
|
||||
rpc StreamUpdate(binlogdata.StreamUpdateRequest) returns (stream binlogdata.StreamUpdateResponse) {};
|
||||
|
||||
// StreamKeyRange returns the binlog transactions related to
|
||||
// the specified Keyrange.
|
||||
rpc StreamKeyRange(binlogdata.StreamKeyRangeRequest) returns (stream binlogdata.StreamKeyRangeResponse) {};
|
||||
|
|
|
@ -210,6 +210,37 @@ message QueryResult {
|
|||
repeated Row rows = 4;
|
||||
}
|
||||
|
||||
// StreamEvent describes a set of transformations that happened as a
|
||||
// single transactional unit on a server. It is streamed back by the
|
||||
// Update Stream calls.
|
||||
message StreamEvent {
|
||||
// One individual Statement in a transaction.
|
||||
message Statement {
|
||||
// The category of one statement.
|
||||
enum Category {
|
||||
Error = 0;
|
||||
DML = 1;
|
||||
DDL = 2;
|
||||
}
|
||||
Category category = 1;
|
||||
|
||||
// table_name, primary_key_fields and primary_key_values are set for DML.
|
||||
string table_name = 2;
|
||||
repeated Field primary_key_fields = 3;
|
||||
repeated Row primary_key_values = 4;
|
||||
|
||||
// sql is set for all queries.
|
||||
// FIXME(alainjobart) we may not need it for DMLs.
|
||||
bytes sql = 5;
|
||||
}
|
||||
|
||||
// The statements in this transaction.
|
||||
repeated Statement statements = 1;
|
||||
|
||||
// The Event Token for this event.
|
||||
EventToken event_token = 2;
|
||||
}
|
||||
|
||||
// ExecuteRequest is the payload to Execute
|
||||
message ExecuteRequest {
|
||||
vtrpc.CallerID effective_caller_id = 1;
|
||||
|
@ -436,3 +467,24 @@ message StreamHealthResponse {
|
|||
// realtime_stats contains information about the tablet status
|
||||
RealtimeStats realtime_stats = 4;
|
||||
}
|
||||
|
||||
// UpdateStreamRequest is the payload for UpdateStream. Only one of
|
||||
// position and timestamp can be set.
|
||||
message UpdateStreamRequest {
|
||||
vtrpc.CallerID effective_caller_id = 1;
|
||||
VTGateCallerID immediate_caller_id = 2;
|
||||
Target target = 3;
|
||||
|
||||
// If position is set, we will start the streaming from that replication
|
||||
// position. Incompatible with timestamp.
|
||||
string position = 4;
|
||||
|
||||
// If timestamp is set, we will start the streaming from the first
|
||||
// event in the binlogs that have that timestamp. Incompatible with position.
|
||||
int64 timestamp = 5;
|
||||
}
|
||||
|
||||
// UpdateStreamResponse is returned by UpdateStream
|
||||
message UpdateStreamResponse {
|
||||
StreamEvent event = 1;
|
||||
}
|
||||
|
|
|
@ -44,4 +44,7 @@ service Query {
|
|||
// StreamHealth runs a streaming RPC to the tablet, that returns the
|
||||
// current health of the tablet on a regular basis.
|
||||
rpc StreamHealth(query.StreamHealthRequest) returns (stream query.StreamHealthResponse) {};
|
||||
|
||||
// UpdateStream asks the server to return a stream of the updates that have been applied to its database.
|
||||
rpc UpdateStream(query.UpdateStreamRequest) returns (stream query.UpdateStreamResponse) {};
|
||||
}
|
||||
|
|
|
@ -9,11 +9,10 @@ import google.protobuf # pylint: disable=unused-import
|
|||
from grpc.beta import implementations
|
||||
from grpc.framework.interfaces.face import face
|
||||
|
||||
from vtproto import binlogdata_pb2
|
||||
from vtproto import binlogservice_pb2
|
||||
from vtproto import query_pb2
|
||||
from vtproto import queryservice_pb2
|
||||
|
||||
from vtdb import dbexceptions
|
||||
from vtdb import proto3_encoding
|
||||
from vtdb import update_stream
|
||||
|
||||
|
||||
|
@ -34,7 +33,7 @@ class GRPCUpdateStreamConnection(update_stream.UpdateStreamConnection):
|
|||
def dial(self):
|
||||
p = urlparse('http://' + self.addr)
|
||||
channel = implementations.insecure_channel(p.hostname, p.port)
|
||||
self.stub = binlogservice_pb2.beta_create_UpdateStream_stub(channel)
|
||||
self.stub = queryservice_pb2.beta_create_Query_stub(channel)
|
||||
|
||||
def close(self):
|
||||
self.stub = None
|
||||
|
@ -42,33 +41,19 @@ class GRPCUpdateStreamConnection(update_stream.UpdateStreamConnection):
|
|||
def is_closed(self):
|
||||
return self.stub is None
|
||||
|
||||
def stream_update(self, position, timeout=3600.0):
|
||||
def stream_update(self, keyspace, shard, tablet_type,
|
||||
position='', timestamp=0,
|
||||
timeout=3600.0):
|
||||
try:
|
||||
req = binlogdata_pb2.StreamUpdateRequest(position=position)
|
||||
target = query_pb2.Target(keyspace=keyspace,
|
||||
shard=shard,
|
||||
tablet_type=tablet_type)
|
||||
req = query_pb2.UpdateStreamRequest(target=target,
|
||||
position=position,
|
||||
timestamp=timestamp)
|
||||
|
||||
it = self.stub.StreamUpdate(req, timeout)
|
||||
for response in it:
|
||||
stream_event = response.stream_event
|
||||
fields = []
|
||||
rows = []
|
||||
if stream_event.primary_key_fields:
|
||||
conversions = []
|
||||
for field in stream_event.primary_key_fields:
|
||||
fields.append(field.name)
|
||||
conversions.append(proto3_encoding.conversions.get(field.type))
|
||||
|
||||
for r in stream_event.primary_key_values:
|
||||
row = tuple(proto3_encoding.make_row(r, conversions))
|
||||
rows.append(row)
|
||||
|
||||
yield update_stream.StreamEvent(
|
||||
category=int(stream_event.category),
|
||||
table_name=stream_event.table_name,
|
||||
fields=fields,
|
||||
rows=rows,
|
||||
sql=stream_event.sql,
|
||||
timestamp=stream_event.event_token.timestamp,
|
||||
position=stream_event.event_token.position)
|
||||
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
|
||||
|
|
|
@ -1,6 +1,10 @@
|
|||
# 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.
|
||||
|
@ -38,28 +42,6 @@ def connect(protocol, *pargs, **kargs):
|
|||
return conn
|
||||
|
||||
|
||||
class StreamEvent(object):
|
||||
"""StreamEvent describes a single event in the update stream.
|
||||
|
||||
Eventually we will use the proto3 definition object.
|
||||
"""
|
||||
|
||||
ERR = 0
|
||||
DML = 1
|
||||
DDL = 2
|
||||
POS = 3
|
||||
|
||||
def __init__(self, category, table_name, fields, rows, sql, timestamp,
|
||||
position):
|
||||
self.category = category
|
||||
self.table_name = table_name
|
||||
self.fields = fields
|
||||
self.rows = rows
|
||||
self.sql = sql
|
||||
self.timestamp = timestamp
|
||||
self.position = position
|
||||
|
||||
|
||||
class UpdateStreamConnection(object):
|
||||
"""The interface for the update stream client implementations.
|
||||
|
||||
|
@ -94,14 +76,45 @@ class UpdateStreamConnection(object):
|
|||
"""
|
||||
pass
|
||||
|
||||
def stream_update(self, position, timeout=3600.0):
|
||||
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 StreamEvent objects.
|
||||
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
|
||||
|
|
|
@ -21,7 +21,7 @@ DESCRIPTOR = _descriptor.FileDescriptor(
|
|||
name='binlogdata.proto',
|
||||
package='binlogdata',
|
||||
syntax='proto3',
|
||||
serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xfc\x02\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xf5\x01\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"q\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\n\n\x06\x42L_DML\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06J\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"\xa4\x02\n\x0bStreamEvent\x12\x32\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32 .binlogdata.StreamEvent.Category\x12\x12\n\ntable_name\x18\x02 \x01(\t\x12(\n\x12primary_key_fields\x18\x03 \x03(\x0b\x32\x0c.query.Field\x12&\n\x12primary_key_values\x18\x04 \x03(\x0b\x32\n.query.Row\x12\x0b\n\x03sql\x18\x05 \x01(\x0c\x12&\n\x0b\x65vent_token\x18\x08 \x01(\x0b\x32\x11.query.EventToken\":\n\x08\x43\x61tegory\x12\n\n\x06SE_ERR\x10\x00\x12\n\n\x06SE_DML\x10\x01\x12\n\n\x06SE_DDL\x10\x02\x12\n\n\x06SE_POS\x10\x03J\x04\x08\x06\x10\x07J\x04\x08\x07\x10\x08\"\'\n\x13StreamUpdateRequest\x12\x10\n\x08position\x18\x01 \x01(\t\"E\n\x14StreamUpdateResponse\x12-\n\x0cstream_event\x18\x01 \x01(\x0b\x32\x17.binlogdata.StreamEvent\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransactionb\x06proto3')
|
||||
serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xfc\x02\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xf5\x01\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"q\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\n\n\x06\x42L_DML\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06J\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransactionb\x06proto3')
|
||||
,
|
||||
dependencies=[query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,])
|
||||
_sym_db.RegisterFileDescriptor(DESCRIPTOR)
|
||||
|
@ -70,36 +70,6 @@ _BINLOGTRANSACTION_STATEMENT_CATEGORY = _descriptor.EnumDescriptor(
|
|||
)
|
||||
_sym_db.RegisterEnumDescriptor(_BINLOGTRANSACTION_STATEMENT_CATEGORY)
|
||||
|
||||
_STREAMEVENT_CATEGORY = _descriptor.EnumDescriptor(
|
||||
name='Category',
|
||||
full_name='binlogdata.StreamEvent.Category',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
values=[
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='SE_ERR', index=0, number=0,
|
||||
options=None,
|
||||
type=None),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='SE_DML', index=1, number=1,
|
||||
options=None,
|
||||
type=None),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='SE_DDL', index=2, number=2,
|
||||
options=None,
|
||||
type=None),
|
||||
_descriptor.EnumValueDescriptor(
|
||||
name='SE_POS', index=3, number=3,
|
||||
options=None,
|
||||
type=None),
|
||||
],
|
||||
containing_type=None,
|
||||
options=None,
|
||||
serialized_start=724,
|
||||
serialized_end=782,
|
||||
)
|
||||
_sym_db.RegisterEnumDescriptor(_STREAMEVENT_CATEGORY)
|
||||
|
||||
|
||||
_CHARSET = _descriptor.Descriptor(
|
||||
name='Charset',
|
||||
|
@ -229,135 +199,6 @@ _BINLOGTRANSACTION = _descriptor.Descriptor(
|
|||
)
|
||||
|
||||
|
||||
_STREAMEVENT = _descriptor.Descriptor(
|
||||
name='StreamEvent',
|
||||
full_name='binlogdata.StreamEvent',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='category', full_name='binlogdata.StreamEvent.category', index=0,
|
||||
number=1, type=14, cpp_type=8, label=1,
|
||||
has_default_value=False, default_value=0,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='table_name', full_name='binlogdata.StreamEvent.table_name', index=1,
|
||||
number=2, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=_b("").decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='primary_key_fields', full_name='binlogdata.StreamEvent.primary_key_fields', index=2,
|
||||
number=3, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='primary_key_values', full_name='binlogdata.StreamEvent.primary_key_values', index=3,
|
||||
number=4, type=11, cpp_type=10, label=3,
|
||||
has_default_value=False, default_value=[],
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='sql', full_name='binlogdata.StreamEvent.sql', index=4,
|
||||
number=5, type=12, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=_b(""),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
_descriptor.FieldDescriptor(
|
||||
name='event_token', full_name='binlogdata.StreamEvent.event_token', index=5,
|
||||
number=8, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
_STREAMEVENT_CATEGORY,
|
||||
],
|
||||
options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=502,
|
||||
serialized_end=794,
|
||||
)
|
||||
|
||||
|
||||
_STREAMUPDATEREQUEST = _descriptor.Descriptor(
|
||||
name='StreamUpdateRequest',
|
||||
full_name='binlogdata.StreamUpdateRequest',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='position', full_name='binlogdata.StreamUpdateRequest.position', index=0,
|
||||
number=1, type=9, cpp_type=9, label=1,
|
||||
has_default_value=False, default_value=_b("").decode('utf-8'),
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=796,
|
||||
serialized_end=835,
|
||||
)
|
||||
|
||||
|
||||
_STREAMUPDATERESPONSE = _descriptor.Descriptor(
|
||||
name='StreamUpdateResponse',
|
||||
full_name='binlogdata.StreamUpdateResponse',
|
||||
filename=None,
|
||||
file=DESCRIPTOR,
|
||||
containing_type=None,
|
||||
fields=[
|
||||
_descriptor.FieldDescriptor(
|
||||
name='stream_event', full_name='binlogdata.StreamUpdateResponse.stream_event', index=0,
|
||||
number=1, type=11, cpp_type=10, label=1,
|
||||
has_default_value=False, default_value=None,
|
||||
message_type=None, enum_type=None, containing_type=None,
|
||||
is_extension=False, extension_scope=None,
|
||||
options=None),
|
||||
],
|
||||
extensions=[
|
||||
],
|
||||
nested_types=[],
|
||||
enum_types=[
|
||||
],
|
||||
options=None,
|
||||
is_extendable=False,
|
||||
syntax='proto3',
|
||||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=837,
|
||||
serialized_end=906,
|
||||
)
|
||||
|
||||
|
||||
_STREAMKEYRANGEREQUEST = _descriptor.Descriptor(
|
||||
name='StreamKeyRangeRequest',
|
||||
full_name='binlogdata.StreamKeyRangeRequest',
|
||||
|
@ -398,8 +239,8 @@ _STREAMKEYRANGEREQUEST = _descriptor.Descriptor(
|
|||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=908,
|
||||
serialized_end=1026,
|
||||
serialized_start=501,
|
||||
serialized_end=619,
|
||||
)
|
||||
|
||||
|
||||
|
@ -429,8 +270,8 @@ _STREAMKEYRANGERESPONSE = _descriptor.Descriptor(
|
|||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=1028,
|
||||
serialized_end=1111,
|
||||
serialized_start=621,
|
||||
serialized_end=704,
|
||||
)
|
||||
|
||||
|
||||
|
@ -474,8 +315,8 @@ _STREAMTABLESREQUEST = _descriptor.Descriptor(
|
|||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=1113,
|
||||
serialized_end=1206,
|
||||
serialized_start=706,
|
||||
serialized_end=799,
|
||||
)
|
||||
|
||||
|
||||
|
@ -505,8 +346,8 @@ _STREAMTABLESRESPONSE = _descriptor.Descriptor(
|
|||
extension_ranges=[],
|
||||
oneofs=[
|
||||
],
|
||||
serialized_start=1208,
|
||||
serialized_end=1289,
|
||||
serialized_start=801,
|
||||
serialized_end=882,
|
||||
)
|
||||
|
||||
_BINLOGTRANSACTION_STATEMENT.fields_by_name['category'].enum_type = _BINLOGTRANSACTION_STATEMENT_CATEGORY
|
||||
|
@ -515,12 +356,6 @@ _BINLOGTRANSACTION_STATEMENT.containing_type = _BINLOGTRANSACTION
|
|||
_BINLOGTRANSACTION_STATEMENT_CATEGORY.containing_type = _BINLOGTRANSACTION_STATEMENT
|
||||
_BINLOGTRANSACTION.fields_by_name['statements'].message_type = _BINLOGTRANSACTION_STATEMENT
|
||||
_BINLOGTRANSACTION.fields_by_name['event_token'].message_type = query__pb2._EVENTTOKEN
|
||||
_STREAMEVENT.fields_by_name['category'].enum_type = _STREAMEVENT_CATEGORY
|
||||
_STREAMEVENT.fields_by_name['primary_key_fields'].message_type = query__pb2._FIELD
|
||||
_STREAMEVENT.fields_by_name['primary_key_values'].message_type = query__pb2._ROW
|
||||
_STREAMEVENT.fields_by_name['event_token'].message_type = query__pb2._EVENTTOKEN
|
||||
_STREAMEVENT_CATEGORY.containing_type = _STREAMEVENT
|
||||
_STREAMUPDATERESPONSE.fields_by_name['stream_event'].message_type = _STREAMEVENT
|
||||
_STREAMKEYRANGEREQUEST.fields_by_name['key_range'].message_type = topodata__pb2._KEYRANGE
|
||||
_STREAMKEYRANGEREQUEST.fields_by_name['charset'].message_type = _CHARSET
|
||||
_STREAMKEYRANGERESPONSE.fields_by_name['binlog_transaction'].message_type = _BINLOGTRANSACTION
|
||||
|
@ -528,9 +363,6 @@ _STREAMTABLESREQUEST.fields_by_name['charset'].message_type = _CHARSET
|
|||
_STREAMTABLESRESPONSE.fields_by_name['binlog_transaction'].message_type = _BINLOGTRANSACTION
|
||||
DESCRIPTOR.message_types_by_name['Charset'] = _CHARSET
|
||||
DESCRIPTOR.message_types_by_name['BinlogTransaction'] = _BINLOGTRANSACTION
|
||||
DESCRIPTOR.message_types_by_name['StreamEvent'] = _STREAMEVENT
|
||||
DESCRIPTOR.message_types_by_name['StreamUpdateRequest'] = _STREAMUPDATEREQUEST
|
||||
DESCRIPTOR.message_types_by_name['StreamUpdateResponse'] = _STREAMUPDATERESPONSE
|
||||
DESCRIPTOR.message_types_by_name['StreamKeyRangeRequest'] = _STREAMKEYRANGEREQUEST
|
||||
DESCRIPTOR.message_types_by_name['StreamKeyRangeResponse'] = _STREAMKEYRANGERESPONSE
|
||||
DESCRIPTOR.message_types_by_name['StreamTablesRequest'] = _STREAMTABLESREQUEST
|
||||
|
@ -558,27 +390,6 @@ BinlogTransaction = _reflection.GeneratedProtocolMessageType('BinlogTransaction'
|
|||
_sym_db.RegisterMessage(BinlogTransaction)
|
||||
_sym_db.RegisterMessage(BinlogTransaction.Statement)
|
||||
|
||||
StreamEvent = _reflection.GeneratedProtocolMessageType('StreamEvent', (_message.Message,), dict(
|
||||
DESCRIPTOR = _STREAMEVENT,
|
||||
__module__ = 'binlogdata_pb2'
|
||||
# @@protoc_insertion_point(class_scope:binlogdata.StreamEvent)
|
||||
))
|
||||
_sym_db.RegisterMessage(StreamEvent)
|
||||
|
||||
StreamUpdateRequest = _reflection.GeneratedProtocolMessageType('StreamUpdateRequest', (_message.Message,), dict(
|
||||
DESCRIPTOR = _STREAMUPDATEREQUEST,
|
||||
__module__ = 'binlogdata_pb2'
|
||||
# @@protoc_insertion_point(class_scope:binlogdata.StreamUpdateRequest)
|
||||
))
|
||||
_sym_db.RegisterMessage(StreamUpdateRequest)
|
||||
|
||||
StreamUpdateResponse = _reflection.GeneratedProtocolMessageType('StreamUpdateResponse', (_message.Message,), dict(
|
||||
DESCRIPTOR = _STREAMUPDATERESPONSE,
|
||||
__module__ = 'binlogdata_pb2'
|
||||
# @@protoc_insertion_point(class_scope:binlogdata.StreamUpdateResponse)
|
||||
))
|
||||
_sym_db.RegisterMessage(StreamUpdateResponse)
|
||||
|
||||
StreamKeyRangeRequest = _reflection.GeneratedProtocolMessageType('StreamKeyRangeRequest', (_message.Message,), dict(
|
||||
DESCRIPTOR = _STREAMKEYRANGEREQUEST,
|
||||
__module__ = 'binlogdata_pb2'
|
||||
|
|
|
@ -20,7 +20,7 @@ DESCRIPTOR = _descriptor.FileDescriptor(
|
|||
name='binlogservice.proto',
|
||||
package='binlogservice',
|
||||
syntax='proto3',
|
||||
serialized_pb=_b('\n\x13\x62inlogservice.proto\x12\rbinlogservice\x1a\x10\x62inlogdata.proto2\x99\x02\n\x0cUpdateStream\x12U\n\x0cStreamUpdate\x12\x1f.binlogdata.StreamUpdateRequest\x1a .binlogdata.StreamUpdateResponse\"\x00\x30\x01\x12[\n\x0eStreamKeyRange\x12!.binlogdata.StreamKeyRangeRequest\x1a\".binlogdata.StreamKeyRangeResponse\"\x00\x30\x01\x12U\n\x0cStreamTables\x12\x1f.binlogdata.StreamTablesRequest\x1a .binlogdata.StreamTablesResponse\"\x00\x30\x01\x62\x06proto3')
|
||||
serialized_pb=_b('\n\x13\x62inlogservice.proto\x12\rbinlogservice\x1a\x10\x62inlogdata.proto2\xc2\x01\n\x0cUpdateStream\x12[\n\x0eStreamKeyRange\x12!.binlogdata.StreamKeyRangeRequest\x1a\".binlogdata.StreamKeyRangeResponse\"\x00\x30\x01\x12U\n\x0cStreamTables\x12\x1f.binlogdata.StreamTablesRequest\x1a .binlogdata.StreamTablesResponse\"\x00\x30\x01\x62\x06proto3')
|
||||
,
|
||||
dependencies=[binlogdata__pb2.DESCRIPTOR,])
|
||||
_sym_db.RegisterFileDescriptor(DESCRIPTOR)
|
||||
|
@ -38,9 +38,6 @@ class BetaUpdateStreamServicer(object):
|
|||
"""<fill me in later!>"""
|
||||
__metaclass__ = abc.ABCMeta
|
||||
@abc.abstractmethod
|
||||
def StreamUpdate(self, request, context):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
def StreamKeyRange(self, request, context):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
|
@ -51,9 +48,6 @@ class BetaUpdateStreamStub(object):
|
|||
"""The interface to which stubs will conform."""
|
||||
__metaclass__ = abc.ABCMeta
|
||||
@abc.abstractmethod
|
||||
def StreamUpdate(self, request, timeout):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
def StreamKeyRange(self, request, timeout):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
|
@ -65,22 +59,17 @@ def beta_create_UpdateStream_server(servicer, pool=None, pool_size=None, default
|
|||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
request_deserializers = {
|
||||
('binlogservice.UpdateStream', 'StreamKeyRange'): binlogdata_pb2.StreamKeyRangeRequest.FromString,
|
||||
('binlogservice.UpdateStream', 'StreamTables'): binlogdata_pb2.StreamTablesRequest.FromString,
|
||||
('binlogservice.UpdateStream', 'StreamUpdate'): binlogdata_pb2.StreamUpdateRequest.FromString,
|
||||
}
|
||||
response_serializers = {
|
||||
('binlogservice.UpdateStream', 'StreamKeyRange'): binlogdata_pb2.StreamKeyRangeResponse.SerializeToString,
|
||||
('binlogservice.UpdateStream', 'StreamTables'): binlogdata_pb2.StreamTablesResponse.SerializeToString,
|
||||
('binlogservice.UpdateStream', 'StreamUpdate'): binlogdata_pb2.StreamUpdateResponse.SerializeToString,
|
||||
}
|
||||
method_implementations = {
|
||||
('binlogservice.UpdateStream', 'StreamKeyRange'): face_utilities.unary_stream_inline(servicer.StreamKeyRange),
|
||||
('binlogservice.UpdateStream', 'StreamTables'): face_utilities.unary_stream_inline(servicer.StreamTables),
|
||||
('binlogservice.UpdateStream', 'StreamUpdate'): face_utilities.unary_stream_inline(servicer.StreamUpdate),
|
||||
}
|
||||
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)
|
||||
|
@ -90,22 +79,17 @@ def beta_create_UpdateStream_stub(channel, host=None, metadata_transformer=None,
|
|||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
import binlogdata_pb2
|
||||
request_serializers = {
|
||||
('binlogservice.UpdateStream', 'StreamKeyRange'): binlogdata_pb2.StreamKeyRangeRequest.SerializeToString,
|
||||
('binlogservice.UpdateStream', 'StreamTables'): binlogdata_pb2.StreamTablesRequest.SerializeToString,
|
||||
('binlogservice.UpdateStream', 'StreamUpdate'): binlogdata_pb2.StreamUpdateRequest.SerializeToString,
|
||||
}
|
||||
response_deserializers = {
|
||||
('binlogservice.UpdateStream', 'StreamKeyRange'): binlogdata_pb2.StreamKeyRangeResponse.FromString,
|
||||
('binlogservice.UpdateStream', 'StreamTables'): binlogdata_pb2.StreamTablesResponse.FromString,
|
||||
('binlogservice.UpdateStream', 'StreamUpdate'): binlogdata_pb2.StreamUpdateResponse.FromString,
|
||||
}
|
||||
cardinalities = {
|
||||
'StreamKeyRange': cardinality.Cardinality.UNARY_STREAM,
|
||||
'StreamTables': cardinality.Cardinality.UNARY_STREAM,
|
||||
'StreamUpdate': 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, 'binlogservice.UpdateStream', cardinalities, options=stub_options)
|
||||
|
|
Различия файлов скрыты, потому что одна или несколько строк слишком длинны
|
@ -20,7 +20,7 @@ DESCRIPTOR = _descriptor.FileDescriptor(
|
|||
name='queryservice.proto',
|
||||
package='queryservice',
|
||||
syntax='proto3',
|
||||
serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto2\xc3\x05\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x62\x06proto3')
|
||||
serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto2\x90\x06\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x12K\n\x0cUpdateStream\x12\x1a.query.UpdateStreamRequest\x1a\x1b.query.UpdateStreamResponse\"\x00\x30\x01\x62\x06proto3')
|
||||
,
|
||||
dependencies=[query__pb2.DESCRIPTOR,])
|
||||
_sym_db.RegisterFileDescriptor(DESCRIPTOR)
|
||||
|
@ -67,6 +67,9 @@ class BetaQueryServicer(object):
|
|||
@abc.abstractmethod
|
||||
def StreamHealth(self, request, context):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
def UpdateStream(self, request, context):
|
||||
raise NotImplementedError()
|
||||
|
||||
class BetaQueryStub(object):
|
||||
"""The interface to which stubs will conform."""
|
||||
|
@ -109,6 +112,9 @@ class BetaQueryStub(object):
|
|||
@abc.abstractmethod
|
||||
def StreamHealth(self, request, timeout):
|
||||
raise NotImplementedError()
|
||||
@abc.abstractmethod
|
||||
def UpdateStream(self, request, timeout):
|
||||
raise NotImplementedError()
|
||||
|
||||
def beta_create_Query_server(servicer, pool=None, pool_size=None, default_timeout=None, maximum_timeout=None):
|
||||
import query_pb2
|
||||
|
@ -131,6 +137,8 @@ def beta_create_Query_server(servicer, pool=None, pool_size=None, default_timeou
|
|||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
request_deserializers = {
|
||||
('queryservice.Query', 'Begin'): query_pb2.BeginRequest.FromString,
|
||||
('queryservice.Query', 'BeginExecute'): query_pb2.BeginExecuteRequest.FromString,
|
||||
|
@ -142,6 +150,7 @@ def beta_create_Query_server(servicer, pool=None, pool_size=None, default_timeou
|
|||
('queryservice.Query', 'SplitQuery'): query_pb2.SplitQueryRequest.FromString,
|
||||
('queryservice.Query', 'StreamExecute'): query_pb2.StreamExecuteRequest.FromString,
|
||||
('queryservice.Query', 'StreamHealth'): query_pb2.StreamHealthRequest.FromString,
|
||||
('queryservice.Query', 'UpdateStream'): query_pb2.UpdateStreamRequest.FromString,
|
||||
}
|
||||
response_serializers = {
|
||||
('queryservice.Query', 'Begin'): query_pb2.BeginResponse.SerializeToString,
|
||||
|
@ -154,6 +163,7 @@ def beta_create_Query_server(servicer, pool=None, pool_size=None, default_timeou
|
|||
('queryservice.Query', 'SplitQuery'): query_pb2.SplitQueryResponse.SerializeToString,
|
||||
('queryservice.Query', 'StreamExecute'): query_pb2.StreamExecuteResponse.SerializeToString,
|
||||
('queryservice.Query', 'StreamHealth'): query_pb2.StreamHealthResponse.SerializeToString,
|
||||
('queryservice.Query', 'UpdateStream'): query_pb2.UpdateStreamResponse.SerializeToString,
|
||||
}
|
||||
method_implementations = {
|
||||
('queryservice.Query', 'Begin'): face_utilities.unary_unary_inline(servicer.Begin),
|
||||
|
@ -166,6 +176,7 @@ def beta_create_Query_server(servicer, pool=None, pool_size=None, default_timeou
|
|||
('queryservice.Query', 'SplitQuery'): face_utilities.unary_unary_inline(servicer.SplitQuery),
|
||||
('queryservice.Query', 'StreamExecute'): face_utilities.unary_stream_inline(servicer.StreamExecute),
|
||||
('queryservice.Query', 'StreamHealth'): face_utilities.unary_stream_inline(servicer.StreamHealth),
|
||||
('queryservice.Query', '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)
|
||||
|
@ -191,6 +202,8 @@ def beta_create_Query_stub(channel, host=None, metadata_transformer=None, pool=N
|
|||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
import query_pb2
|
||||
request_serializers = {
|
||||
('queryservice.Query', 'Begin'): query_pb2.BeginRequest.SerializeToString,
|
||||
('queryservice.Query', 'BeginExecute'): query_pb2.BeginExecuteRequest.SerializeToString,
|
||||
|
@ -202,6 +215,7 @@ def beta_create_Query_stub(channel, host=None, metadata_transformer=None, pool=N
|
|||
('queryservice.Query', 'SplitQuery'): query_pb2.SplitQueryRequest.SerializeToString,
|
||||
('queryservice.Query', 'StreamExecute'): query_pb2.StreamExecuteRequest.SerializeToString,
|
||||
('queryservice.Query', 'StreamHealth'): query_pb2.StreamHealthRequest.SerializeToString,
|
||||
('queryservice.Query', 'UpdateStream'): query_pb2.UpdateStreamRequest.SerializeToString,
|
||||
}
|
||||
response_deserializers = {
|
||||
('queryservice.Query', 'Begin'): query_pb2.BeginResponse.FromString,
|
||||
|
@ -214,6 +228,7 @@ def beta_create_Query_stub(channel, host=None, metadata_transformer=None, pool=N
|
|||
('queryservice.Query', 'SplitQuery'): query_pb2.SplitQueryResponse.FromString,
|
||||
('queryservice.Query', 'StreamExecute'): query_pb2.StreamExecuteResponse.FromString,
|
||||
('queryservice.Query', 'StreamHealth'): query_pb2.StreamHealthResponse.FromString,
|
||||
('queryservice.Query', 'UpdateStream'): query_pb2.UpdateStreamResponse.FromString,
|
||||
}
|
||||
cardinalities = {
|
||||
'Begin': cardinality.Cardinality.UNARY_UNARY,
|
||||
|
@ -226,6 +241,7 @@ def beta_create_Query_stub(channel, host=None, metadata_transformer=None, pool=N
|
|||
'SplitQuery': cardinality.Cardinality.UNARY_UNARY,
|
||||
'StreamExecute': cardinality.Cardinality.UNARY_STREAM,
|
||||
'StreamHealth': 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, 'queryservice.Query', cardinalities, options=stub_options)
|
||||
|
|
|
@ -14,6 +14,7 @@ import unittest
|
|||
|
||||
from vtdb import keyrange_constants
|
||||
from vtdb import update_stream
|
||||
from vtproto import topodata_pb2
|
||||
|
||||
import environment
|
||||
import tablet
|
||||
|
@ -158,8 +159,9 @@ class TestBinlog(unittest.TestCase):
|
|||
|
||||
# Wait for it to replicate.
|
||||
stream = _get_update_stream(dst_replica)
|
||||
for stream_event in stream.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
for event in stream.stream_update('test_keyspace', '-',
|
||||
topodata_pb2.REPLICA, start_position):
|
||||
if event.event_token.position:
|
||||
break
|
||||
stream.close()
|
||||
|
||||
|
@ -194,12 +196,12 @@ class TestBinlog(unittest.TestCase):
|
|||
# dst_replica, which now has binlog_checksum enabled.
|
||||
stream = _get_update_stream(dst_replica)
|
||||
found = False
|
||||
for stream_event in stream.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
break
|
||||
if stream_event.sql == sql:
|
||||
found = True
|
||||
break
|
||||
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')
|
||||
|
||||
|
@ -224,12 +226,12 @@ class TestBinlog(unittest.TestCase):
|
|||
# dst_replica, which now has binlog_checksum disabled.
|
||||
stream = _get_update_stream(dst_replica)
|
||||
found = False
|
||||
for stream_event in stream.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
break
|
||||
if stream_event.sql == sql:
|
||||
found = True
|
||||
break
|
||||
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')
|
||||
|
||||
|
|
|
@ -1,8 +1,7 @@
|
|||
#!/usr/bin/env python
|
||||
|
||||
import logging
|
||||
import traceback
|
||||
import threading
|
||||
import time
|
||||
import unittest
|
||||
|
||||
import environment
|
||||
|
@ -11,6 +10,7 @@ import utils
|
|||
from vtdb import dbexceptions
|
||||
from vtdb import update_stream
|
||||
from vtdb import vtgate_client
|
||||
from vtproto import topodata_pb2
|
||||
from mysql_flavor import mysql_flavor
|
||||
from protocols_flavor import protocols_flavor
|
||||
|
||||
|
@ -167,100 +167,6 @@ class TestUpdateStream(unittest.TestCase):
|
|||
protocol, endpoint = replica_tablet.update_stream_python_endpoint()
|
||||
return update_stream.connect(protocol, endpoint, 30)
|
||||
|
||||
def _test_service_disabled(self):
|
||||
# it looks like update stream would be re-enabled automatically
|
||||
# because of vttablet health check
|
||||
return
|
||||
start_position = _get_repl_current_position()
|
||||
logging.debug('_test_service_disabled starting @ %s', start_position)
|
||||
self._exec_vt_txn(self._populate_vt_insert_test)
|
||||
self._exec_vt_txn(['delete from vt_insert_test'])
|
||||
utils.run_vtctl(['ChangeSlaveType', replica_tablet.tablet_alias, 'spare'])
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'spare')
|
||||
logging.debug('dialing replica update stream service')
|
||||
replica_conn = self._get_replica_stream_conn()
|
||||
try:
|
||||
for _ in replica_conn.stream_update(start_position):
|
||||
break
|
||||
except dbexceptions.DatabaseError as e:
|
||||
self.assertIn('update stream service is not enabled', str(e))
|
||||
replica_conn.close()
|
||||
|
||||
v = utils.get_vars(replica_tablet.port)
|
||||
if v['UpdateStreamState'] != 'Disabled':
|
||||
self.fail("Update stream service should be 'Disabled' but is '%s'" %
|
||||
v['UpdateStreamState'])
|
||||
|
||||
def perform_writes(self, count):
|
||||
for _ in xrange(count):
|
||||
self._exec_vt_txn(self._populate_vt_insert_test)
|
||||
self._exec_vt_txn(['delete from vt_insert_test'])
|
||||
|
||||
def _test_service_enabled(self):
|
||||
# it looks like update stream would be re-enabled automatically
|
||||
# because of vttablet health check
|
||||
return
|
||||
start_position = _get_repl_current_position()
|
||||
logging.debug('_test_service_enabled starting @ %s', start_position)
|
||||
utils.run_vtctl(
|
||||
['ChangeSlaveType', replica_tablet.tablet_alias, 'replica'])
|
||||
logging.debug('sleeping a bit for the replica action to complete')
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'replica', 30)
|
||||
thd = threading.Thread(target=self.perform_writes, name='write_thd',
|
||||
args=(100,))
|
||||
thd.daemon = True
|
||||
thd.start()
|
||||
replica_conn = self._get_replica_stream_conn()
|
||||
|
||||
try:
|
||||
for stream_event in replica_conn.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.DML:
|
||||
logging.debug('Test Service Enabled: Pass')
|
||||
break
|
||||
except dbexceptions.DatabaseError as e:
|
||||
self.fail('Exception in getting stream from replica: %s\n Traceback %s' %
|
||||
(str(e), traceback.format_exc()))
|
||||
thd.join(timeout=30)
|
||||
replica_conn.close()
|
||||
|
||||
v = utils.get_vars(replica_tablet.port)
|
||||
if v['UpdateStreamState'] != 'Enabled':
|
||||
self.fail("Update stream service should be 'Enabled' but is '%s'" %
|
||||
v['UpdateStreamState'])
|
||||
self.assertIn('SE_DML', v['UpdateStreamEvents'])
|
||||
self.assertIn('SE_POS', v['UpdateStreamEvents'])
|
||||
|
||||
logging.debug('Testing enable -> disable switch starting @ %s',
|
||||
start_position)
|
||||
replica_conn = self._get_replica_stream_conn()
|
||||
first = True
|
||||
txn_count = 0
|
||||
try:
|
||||
for stream_event in replica_conn.stream_update(start_position):
|
||||
if first:
|
||||
utils.run_vtctl(
|
||||
['ChangeSlaveType', replica_tablet.tablet_alias, 'spare'])
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'spare', 30)
|
||||
first = False
|
||||
else:
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
txn_count += 1
|
||||
# FIXME(alainjobart) gasp, the test fails but we don't assert?
|
||||
logging.debug('Test Service Switch: FAIL')
|
||||
replica_conn.close()
|
||||
return
|
||||
except dbexceptions.DatabaseError as e:
|
||||
self.assertEqual(
|
||||
'Fatal Service Error: Disconnecting because the Update Stream '
|
||||
'service has been disabled',
|
||||
str(e))
|
||||
except Exception as e:
|
||||
logging.error('Exception: %s', str(e))
|
||||
logging.error('Traceback: %s', traceback.format_exc())
|
||||
self.fail("Update stream returned error '%s'" % str(e))
|
||||
logging.debug('Streamed %d transactions before exiting', txn_count)
|
||||
replica_conn.close()
|
||||
|
||||
def _exec_vt_txn(self, query_list):
|
||||
protocol, addr = utils.vtgate.rpc_endpoint(python=True)
|
||||
vtgate_conn = vtgate_client.connect(protocol, addr, 30.0)
|
||||
|
@ -297,50 +203,62 @@ class TestUpdateStream(unittest.TestCase):
|
|||
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_events = []
|
||||
for stream_event in master_conn.stream_update(master_start_position):
|
||||
master_events.append(stream_event)
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
for event in master_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.MASTER,
|
||||
position=master_start_position):
|
||||
master_events.append(event)
|
||||
if event.event_token.position:
|
||||
break
|
||||
master_conn.close()
|
||||
|
||||
# get replica events
|
||||
replica_events = []
|
||||
replica_conn = self._get_replica_stream_conn()
|
||||
for stream_event in replica_conn.stream_update(replica_start_position):
|
||||
replica_events.append(stream_event)
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
for event in replica_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.REPLICA,
|
||||
position=replica_start_position):
|
||||
replica_events.append(event)
|
||||
if event.event_token.position:
|
||||
break
|
||||
replica_conn.close()
|
||||
|
||||
# and compare
|
||||
if len(master_events) != len(replica_events):
|
||||
logging.debug(
|
||||
'Test Failed - # of records mismatch, master %s replica %s',
|
||||
master_events, replica_events)
|
||||
for master_val, replica_val in zip(master_events, replica_events):
|
||||
master_data = master_val.__dict__
|
||||
replica_data = replica_val.__dict__
|
||||
# the timestamp is from when the event was written to the binlogs.
|
||||
for master_event, replica_event in zip(master_events, replica_events):
|
||||
# The timestamp is from when the event was written to the binlogs.
|
||||
# the master uses the timestamp of when it wrote it originally,
|
||||
# the slave of when it applied the logs. These can differ and make this
|
||||
# test flaky. So we just blank them out, easier. We really want to
|
||||
# compare the replication positions.
|
||||
master_data['timestamp'] = 'XXX'
|
||||
replica_data['timestamp'] = 'XXX'
|
||||
master_event.event_token.timestamp = 123
|
||||
replica_event.event_token.timestamp = 123
|
||||
self.assertEqual(
|
||||
master_data, replica_data,
|
||||
"Test failed, data mismatch - master '%s' and replica position '%s'" %
|
||||
(master_data, replica_data))
|
||||
master_conn.close()
|
||||
replica_conn.close()
|
||||
master_event, replica_event,
|
||||
"Test failed, data mismatch - master '%s' and replica '%s'" %
|
||||
(master_event, replica_event))
|
||||
logging.debug('Test Writes: PASS')
|
||||
|
||||
def test_ddl(self):
|
||||
start_position = master_start_position
|
||||
logging.debug('test_ddl: starting @ %s', start_position)
|
||||
master_conn = self._get_master_stream_conn()
|
||||
for stream_event in master_conn.stream_update(start_position):
|
||||
self.assertEqual(stream_event.sql, _create_vt_insert_test,
|
||||
"DDL didn't match original")
|
||||
master_conn.close()
|
||||
return
|
||||
self.fail("didn't get right sql")
|
||||
found = False
|
||||
for event in master_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.MASTER,
|
||||
position=start_position):
|
||||
for statement in event.statements:
|
||||
if statement.sql == _create_vt_insert_test:
|
||||
found = True
|
||||
break
|
||||
master_conn.close()
|
||||
self.assertTrue(found, "didn't get right sql")
|
||||
|
||||
def test_set_insert_id(self):
|
||||
start_position = _get_master_current_position()
|
||||
|
@ -349,12 +267,15 @@ class TestUpdateStream(unittest.TestCase):
|
|||
logging.debug('test_set_insert_id: starting @ %s', start_position)
|
||||
master_conn = self._get_master_stream_conn()
|
||||
expected_id = 1000000
|
||||
for stream_event in master_conn.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
break
|
||||
self.assertEqual(stream_event.fields[0], 'id')
|
||||
self.assertEqual(stream_event.rows[0][0], expected_id)
|
||||
expected_id += 1
|
||||
for event in master_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.MASTER,
|
||||
position=start_position):
|
||||
for statement in event.statements:
|
||||
fields, rows = update_stream.convert_statement(statement)
|
||||
self.assertEqual(fields[0], 'id')
|
||||
self.assertEqual(rows[0][0], expected_id)
|
||||
expected_id += 1
|
||||
break
|
||||
if expected_id != 1000004:
|
||||
self.fail('did not get my four values!')
|
||||
master_conn.close()
|
||||
|
@ -365,25 +286,94 @@ class TestUpdateStream(unittest.TestCase):
|
|||
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 stream_event in master_conn.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
break
|
||||
self.assertNotEqual(
|
||||
stream_event.category, update_stream.StreamEvent.DDL,
|
||||
"query using other_database wasn't filted out")
|
||||
for event in master_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.MASTER,
|
||||
position=start_position):
|
||||
for statement in event.statements:
|
||||
self.assertNotEqual(statement.category, 2, # query_pb2.StreamEvent.DDL
|
||||
"query using other_database wasn't filtered out")
|
||||
break
|
||||
master_conn.close()
|
||||
|
||||
def test_service_switch(self):
|
||||
"""tests the service switch from disable -> enable -> disable."""
|
||||
self._test_service_disabled()
|
||||
self._test_service_enabled()
|
||||
# The above tests leaves the service in disabled state, hence enabling it.
|
||||
# make the replica spare
|
||||
utils.run_vtctl(['ChangeSlaveType', replica_tablet.tablet_alias, 'spare'])
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'spare')
|
||||
|
||||
# Check UpdateStreamState is disabled.
|
||||
v = utils.get_vars(replica_tablet.port)
|
||||
if v['UpdateStreamState'] != 'Disabled':
|
||||
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()
|
||||
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')
|
||||
except dbexceptions.DatabaseError as e:
|
||||
self.assertIn('operation not allowed in state NOT_SERVING', str(e))
|
||||
|
||||
# Go back to replica.
|
||||
utils.run_vtctl(
|
||||
['ChangeSlaveType', replica_tablet.tablet_alias, 'replica'])
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'replica')
|
||||
|
||||
# Check UpdateStreamState is enabled.
|
||||
v = utils.get_vars(replica_tablet.port)
|
||||
if v['UpdateStreamState'] != 'Enabled':
|
||||
self.fail("Update stream service should be 'Enabled' but is '%s'" %
|
||||
v['UpdateStreamState'])
|
||||
|
||||
def test_update_stream_interrupt(self):
|
||||
"""Checks that a running query is terminated on going non-serving."""
|
||||
# Make sure the replica is replica type.
|
||||
utils.run_vtctl(
|
||||
['ChangeSlaveType', replica_tablet.tablet_alias, 'replica'])
|
||||
logging.debug('sleeping a bit for the replica action to complete')
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'replica', 30)
|
||||
|
||||
# Save current position, insert some data.
|
||||
start_position = _get_repl_current_position()
|
||||
logging.debug('test_update_stream_interrupt starting @ %s', start_position)
|
||||
self._exec_vt_txn(self._populate_vt_a(1))
|
||||
self._exec_vt_txn(['delete from vt_a'])
|
||||
|
||||
# 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()
|
||||
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)
|
||||
if first:
|
||||
utils.run_vtctl(
|
||||
['ChangeSlaveType', replica_tablet.tablet_alias, 'spare'])
|
||||
utils.wait_for_tablet_type(replica_tablet.tablet_alias, 'spare', 30)
|
||||
first = False
|
||||
else:
|
||||
if event.event_token.position:
|
||||
txn_count += 1
|
||||
|
||||
self.assertFail('stream_update terminated with no exception')
|
||||
except dbexceptions.DatabaseError as e:
|
||||
self.assertIn('context canceled', str(e))
|
||||
self.assertFalse(first)
|
||||
|
||||
logging.debug('Streamed %d transactions before exiting', txn_count)
|
||||
replica_conn.close()
|
||||
|
||||
def test_log_rotation(self):
|
||||
start_position = _get_master_current_position()
|
||||
logging.debug('test_log_rotation: starting @ %s', start_position)
|
||||
position = start_position
|
||||
master_tablet.mquery('vt_test_keyspace', 'flush logs')
|
||||
self._exec_vt_txn(self._populate_vt_a(15))
|
||||
|
@ -391,10 +381,12 @@ class TestUpdateStream(unittest.TestCase):
|
|||
master_conn = self._get_master_stream_conn()
|
||||
master_txn_count = 0
|
||||
logs_correct = False
|
||||
for stream_event in master_conn.stream_update(start_position):
|
||||
if stream_event.category == update_stream.StreamEvent.POS:
|
||||
for event in master_conn.stream_update('test_keyspace', '0',
|
||||
topodata_pb2.MASTER,
|
||||
position=start_position):
|
||||
if event.event_token.position:
|
||||
master_txn_count += 1
|
||||
position = stream_event.position
|
||||
position = event.event_token.position
|
||||
if mysql_flavor().position_after(position, start_position):
|
||||
logs_correct = True
|
||||
logging.debug('Log rotation correctly interpreted')
|
||||
|
@ -405,5 +397,116 @@ class TestUpdateStream(unittest.TestCase):
|
|||
self.fail("Flush logs didn't get properly interpreted")
|
||||
master_conn.close()
|
||||
|
||||
def test_timestamp_start_current_log(self):
|
||||
"""Test we can start binlog streaming from the current binlog.
|
||||
|
||||
Order of operation:
|
||||
- Insert something in the binlogs for tablet vt_a then delete it.
|
||||
- Get the current timestamp.
|
||||
- Wait for 4 seconds for the timestamp to change for sure.
|
||||
- Insert something else in vt_b and delete it.
|
||||
- Then we stream events starting at the original timestamp + 2, we
|
||||
should get only the vt_b events.
|
||||
"""
|
||||
self._test_timestamp_start(rotate_before_sleep=False,
|
||||
rotate_after_sleep=False)
|
||||
|
||||
def test_timestamp_start_rotated_log_before_sleep(self):
|
||||
"""Test we can start binlog streaming from the current rotated binlog.
|
||||
|
||||
Order of operation:
|
||||
- Insert something in the binlogs for tablet vt_a then delete it.
|
||||
- Rotate the logs.
|
||||
- Get the current timestamp.
|
||||
- Wait for 4 seconds for the timestamp to change for sure.
|
||||
- Insert something else in vt_b and delete it.
|
||||
- Then we stream events starting at the original timestamp + 2, we
|
||||
should get only the vt_b events.
|
||||
|
||||
In this test case, the current binlogs have a starting time stamp
|
||||
that is smaller than what we ask for, so it should just stay on it.
|
||||
"""
|
||||
self._test_timestamp_start(rotate_before_sleep=True,
|
||||
rotate_after_sleep=False)
|
||||
|
||||
def test_timestamp_start_rotated_log_after_sleep(self):
|
||||
"""Test we can start binlog streaming from the previous binlog.
|
||||
|
||||
Order of operation:
|
||||
- Insert something in the binlogs for tablet vt_a then delete it.
|
||||
- Get the current timestamp.
|
||||
- Wait for 4 seconds for the timestamp to change for sure.
|
||||
- Rotate the logs.
|
||||
- Insert something else in vt_b and delete it.
|
||||
- Then we stream events starting at the original timestamp + 2, we
|
||||
should get only the vt_b events.
|
||||
|
||||
In this test case, the current binlogs have a starting time stamp
|
||||
that is 2s higher than what we ask for, so it should go back to
|
||||
the previous binlog.
|
||||
"""
|
||||
self._test_timestamp_start(rotate_before_sleep=False,
|
||||
rotate_after_sleep=True)
|
||||
|
||||
def _test_timestamp_start(self,
|
||||
rotate_before_sleep=False,
|
||||
rotate_after_sleep=False):
|
||||
"""Common function for timestamp tests."""
|
||||
# Insert something in the binlogs for tablet vt_a then delete it.
|
||||
self._exec_vt_txn(self._populate_vt_a(1))
|
||||
self._exec_vt_txn(['delete from vt_a'])
|
||||
|
||||
# (optional) Rotate the logs
|
||||
if rotate_before_sleep:
|
||||
master_tablet.mquery('vt_test_keyspace', 'flush logs')
|
||||
|
||||
# Get the current timestamp.
|
||||
starting_timestamp = long(time.time())
|
||||
logging.debug('test_timestamp_start_current_log: starting @ %d',
|
||||
starting_timestamp)
|
||||
|
||||
# Wait for 4 seconds for the timestamp to change for sure.
|
||||
time.sleep(4)
|
||||
|
||||
# (optional) Rotate the logs
|
||||
if rotate_after_sleep:
|
||||
master_tablet.mquery('vt_test_keyspace', 'flush logs')
|
||||
|
||||
# Insert something else in vt_b and delete it.
|
||||
self._exec_vt_txn(self._populate_vt_b(1))
|
||||
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()
|
||||
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))
|
||||
# we might get a couple extra events from the rotation, ignore these.
|
||||
if event.statements[0].category == 0: # Statement.Category.Error
|
||||
continue
|
||||
self.assertEqual(event.statements[0].table_name, 'vt_b',
|
||||
'got wrong event: %s' % str(event))
|
||||
count += 1
|
||||
if count == 2:
|
||||
break
|
||||
master_conn.close()
|
||||
|
||||
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()
|
||||
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:
|
||||
self.assertIn('retry: cannot find relevant binlogs on this server',
|
||||
str(e))
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
utils.main()
|
||||
|
|
Загрузка…
Ссылка в новой задаче