Now using strings instead of ReplicationPosition structures in tabletmanager.

That limits the number of encoding / decoding, and is all around easier.
We now only use the real structures when talking to mysqlctl module.
This commit is contained in:
Alain Jobart 2015-11-10 09:03:29 -08:00
Родитель a49338fbcb
Коммит 96f6b35711
14 изменённых файлов: 542 добавлений и 563 удалений

Просмотреть файл

@ -404,11 +404,11 @@ func CreateBlpCheckpoint() []string {
// PopulateBlpCheckpoint returns a statement to populate the first value into
// the _vt.blp_checkpoint table.
func PopulateBlpCheckpoint(index uint32, pos myproto.ReplicationPosition, timeUpdated int64, flags string) string {
func PopulateBlpCheckpoint(index uint32, position string, timeUpdated int64, flags string) string {
return fmt.Sprintf("INSERT INTO _vt.blp_checkpoint "+
"(source_shard_uid, pos, time_updated, transaction_timestamp, flags) "+
"VALUES (%v, '%v', %v, 0, '%v')",
index, myproto.EncodeReplicationPosition(pos), timeUpdated, flags)
index, position, timeUpdated, flags)
}
// UpdateBlpCheckpoint returns a statement to update a value in the

Просмотреть файл

@ -11,12 +11,11 @@ import (
)
func TestPopulateBlpCheckpoint(t *testing.T) {
gtid := myproto.MustParseGTID("MariaDB", "0-1-1083")
want := "INSERT INTO _vt.blp_checkpoint " +
"(source_shard_uid, pos, time_updated, transaction_timestamp, flags) " +
"VALUES (18372, 'MariaDB/0-1-1083', 481823, 0, 'myflags')"
got := PopulateBlpCheckpoint(18372, myproto.ReplicationPosition{GTIDSet: gtid.GTIDSet()}, 481823, "myflags")
got := PopulateBlpCheckpoint(18372, "MariaDB/0-1-1083", 481823, "myflags")
if got != want {
t.Errorf("PopulateBlpCheckpoint() = %#v, want %#v", got, want)
}

Просмотреть файл

@ -7,13 +7,13 @@ package proto
import (
"fmt"
pb "github.com/youtube/vitess/go/vt/proto/replicationdata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
)
// ReplicationStatusToProto translates a ReplicationStatus to
// proto, or panics
func ReplicationStatusToProto(r ReplicationStatus) *pb.Status {
return &pb.Status{
func ReplicationStatusToProto(r ReplicationStatus) *replicationdatapb.Status {
return &replicationdatapb.Status{
Position: EncodeReplicationPosition(r.Position),
SlaveIoRunning: r.SlaveIORunning,
SlaveSqlRunning: r.SlaveSQLRunning,
@ -25,7 +25,7 @@ func ReplicationStatusToProto(r ReplicationStatus) *pb.Status {
}
// ProtoToReplicationStatus translates a proto ReplicationStatus, or panics
func ProtoToReplicationStatus(r *pb.Status) ReplicationStatus {
func ProtoToReplicationStatus(r *replicationdatapb.Status) ReplicationStatus {
pos, err := DecodeReplicationPosition(r.Position)
if err != nil {
panic(fmt.Errorf("cannot decode Position: %v", err))

Просмотреть файл

@ -19,8 +19,9 @@ import (
"github.com/youtube/vitess/go/vt/topotools"
"golang.org/x/net/context"
pbt "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// This file contains the actions that exist as RPC only on the ActionAgent.
@ -35,15 +36,15 @@ type RPCAgent interface {
Ping(ctx context.Context, args string) string
GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*pbt.SchemaDefinition, error)
GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error)
GetPermissions(ctx context.Context) (*pbt.Permissions, error)
GetPermissions(ctx context.Context) (*tabletmanagerdatapb.Permissions, error)
// Various read-write methods
SetReadOnly(ctx context.Context, rdonly bool) error
ChangeType(ctx context.Context, tabletType pb.TabletType) error
ChangeType(ctx context.Context, tabletType topodatapb.TabletType) error
Sleep(ctx context.Context, duration time.Duration)
@ -51,7 +52,7 @@ type RPCAgent interface {
RefreshState(ctx context.Context)
RunHealthCheck(ctx context.Context, targetTabletType pb.TabletType)
RunHealthCheck(ctx context.Context, targetTabletType topodatapb.TabletType)
ReloadSchema(ctx context.Context)
@ -65,13 +66,13 @@ type RPCAgent interface {
// Replication related methods
SlaveStatus(ctx context.Context) (myproto.ReplicationStatus, error)
SlaveStatus(ctx context.Context) (*replicationdatapb.Status, error)
MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error)
MasterPosition(ctx context.Context) (string, error)
StopSlave(ctx context.Context) error
StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error)
StopSlaveMinimum(ctx context.Context, position string, waitTime time.Duration) (string, error)
StartSlave(ctx context.Context) error
@ -79,37 +80,37 @@ type RPCAgent interface {
GetSlaves(ctx context.Context) ([]string, error)
WaitBlpPosition(ctx context.Context, blpPosition *pbt.BlpPosition, waitTime time.Duration) error
WaitBlpPosition(ctx context.Context, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error
StopBlp(ctx context.Context) ([]*pbt.BlpPosition, error)
StopBlp(ctx context.Context) ([]*tabletmanagerdatapb.BlpPosition, error)
StartBlp(ctx context.Context) error
RunBlpUntil(ctx context.Context, bpl []*pbt.BlpPosition, waitTime time.Duration) (*myproto.ReplicationPosition, error)
RunBlpUntil(ctx context.Context, bpl []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error)
// Reparenting related functions
ResetReplication(ctx context.Context) error
InitMaster(ctx context.Context) (myproto.ReplicationPosition, error)
InitMaster(ctx context.Context) (string, error)
PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *pb.TabletAlias, pos myproto.ReplicationPosition) error
PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, pos string) error
InitSlave(ctx context.Context, parent *pb.TabletAlias, replicationPosition myproto.ReplicationPosition, timeCreatedNS int64) error
InitSlave(ctx context.Context, parent *topodatapb.TabletAlias, replicationPosition string, timeCreatedNS int64) error
DemoteMaster(ctx context.Context) (myproto.ReplicationPosition, error)
DemoteMaster(ctx context.Context) (string, error)
PromoteSlaveWhenCaughtUp(ctx context.Context, replicationPosition myproto.ReplicationPosition) (myproto.ReplicationPosition, error)
PromoteSlaveWhenCaughtUp(ctx context.Context, replicationPosition string) (string, error)
SlaveWasPromoted(ctx context.Context) error
SetMaster(ctx context.Context, parent *pb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
SetMaster(ctx context.Context, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error
StopReplicationAndGetStatus(ctx context.Context) (myproto.ReplicationStatus, error)
StopReplicationAndGetStatus(ctx context.Context) (*replicationdatapb.Status, error)
PromoteSlave(ctx context.Context) (myproto.ReplicationPosition, error)
PromoteSlave(ctx context.Context) (string, error)
// Backup / restore related methods
@ -136,13 +137,13 @@ func (agent *ActionAgent) Ping(ctx context.Context, args string) string {
// GetSchema returns the schema.
// Should be called under RPCWrap.
func (agent *ActionAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*pbt.SchemaDefinition, error) {
func (agent *ActionAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) {
return agent.MysqlDaemon.GetSchema(agent.Tablet().DbName(), tables, excludeTables, includeViews)
}
// GetPermissions returns the db permissions.
// Should be called under RPCWrap.
func (agent *ActionAgent) GetPermissions(ctx context.Context) (*pbt.Permissions, error) {
func (agent *ActionAgent) GetPermissions(ctx context.Context) (*tabletmanagerdatapb.Permissions, error) {
return mysqlctl.GetPermissions(agent.MysqlDaemon)
}
@ -154,7 +155,7 @@ func (agent *ActionAgent) SetReadOnly(ctx context.Context, rdonly bool) error {
// ChangeType changes the tablet type
// Should be called under RPCWrapLockAction.
func (agent *ActionAgent) ChangeType(ctx context.Context, tabletType pb.TabletType) error {
func (agent *ActionAgent) ChangeType(ctx context.Context, tabletType topodatapb.TabletType) error {
return topotools.ChangeType(ctx, agent.TopoServer, agent.TabletAlias, tabletType, nil)
}
@ -178,7 +179,7 @@ func (agent *ActionAgent) RefreshState(ctx context.Context) {
// RunHealthCheck will manually run the health check on the tablet
// Should be called under RPCWrap.
func (agent *ActionAgent) RunHealthCheck(ctx context.Context, targetTabletType pb.TabletType) {
func (agent *ActionAgent) RunHealthCheck(ctx context.Context, targetTabletType topodatapb.TabletType) {
agent.runHealthCheck(targetTabletType)
}
@ -280,14 +281,22 @@ func (agent *ActionAgent) ExecuteFetchAsApp(ctx context.Context, query string, m
// SlaveStatus returns the replication status
// Should be called under RPCWrap.
func (agent *ActionAgent) SlaveStatus(ctx context.Context) (myproto.ReplicationStatus, error) {
return agent.MysqlDaemon.SlaveStatus()
func (agent *ActionAgent) SlaveStatus(ctx context.Context) (*replicationdatapb.Status, error) {
status, err := agent.MysqlDaemon.SlaveStatus()
if err != nil {
return nil, err
}
return myproto.ReplicationStatusToProto(status), nil
}
// MasterPosition returns the master position
// Should be called under RPCWrap.
func (agent *ActionAgent) MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error) {
return agent.MysqlDaemon.MasterPosition()
func (agent *ActionAgent) MasterPosition(ctx context.Context) (string, error) {
pos, err := agent.MysqlDaemon.MasterPosition()
if err != nil {
return "", err
}
return myproto.EncodeReplicationPosition(pos), nil
}
// StopSlave will stop the replication. Works both when Vitess manages
@ -300,14 +309,22 @@ func (agent *ActionAgent) StopSlave(ctx context.Context) error {
// StopSlaveMinimum will stop the slave after it reaches at least the
// provided position. Works both when Vitess manages
// replication or not (using hook if not).
func (agent *ActionAgent) StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
if err := agent.MysqlDaemon.WaitMasterPos(position, waitTime); err != nil {
return myproto.ReplicationPosition{}, err
func (agent *ActionAgent) StopSlaveMinimum(ctx context.Context, position string, waitTime time.Duration) (string, error) {
pos, err := myproto.DecodeReplicationPosition(position)
if err != nil {
return "", err
}
if err := agent.MysqlDaemon.WaitMasterPos(pos, waitTime); err != nil {
return "", err
}
if err := mysqlctl.StopSlave(agent.MysqlDaemon, agent.hookExtraEnv()); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
return agent.MysqlDaemon.MasterPosition()
pos, err = agent.MysqlDaemon.MasterPosition()
if err != nil {
return "", err
}
return myproto.EncodeReplicationPosition(pos), nil
}
// StartSlave will start the replication. Works both when Vitess manages
@ -326,13 +343,13 @@ func (agent *ActionAgent) GetSlaves(ctx context.Context) ([]string, error) {
// WaitBlpPosition waits until a specific filtered replication position is
// reached.
// Should be called under RPCWrapLock.
func (agent *ActionAgent) WaitBlpPosition(ctx context.Context, blpPosition *pbt.BlpPosition, waitTime time.Duration) error {
func (agent *ActionAgent) WaitBlpPosition(ctx context.Context, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error {
return mysqlctl.WaitBlpPosition(agent.MysqlDaemon, blpPosition.Uid, blpPosition.Position, waitTime)
}
// StopBlp stops the binlog players, and return their positions.
// Should be called under RPCWrapLockAction.
func (agent *ActionAgent) StopBlp(ctx context.Context) ([]*pbt.BlpPosition, error) {
func (agent *ActionAgent) StopBlp(ctx context.Context) ([]*tabletmanagerdatapb.BlpPosition, error) {
if agent.BinlogPlayerMap == nil {
return nil, fmt.Errorf("No BinlogPlayerMap configured")
}
@ -352,15 +369,18 @@ func (agent *ActionAgent) StartBlp(ctx context.Context) error {
// RunBlpUntil runs the binlog player server until the position is reached,
// and returns the current mysql master replication position.
func (agent *ActionAgent) RunBlpUntil(ctx context.Context, bpl []*pbt.BlpPosition, waitTime time.Duration) (*myproto.ReplicationPosition, error) {
func (agent *ActionAgent) RunBlpUntil(ctx context.Context, bpl []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error) {
if agent.BinlogPlayerMap == nil {
return nil, fmt.Errorf("No BinlogPlayerMap configured")
return "", fmt.Errorf("No BinlogPlayerMap configured")
}
if err := agent.BinlogPlayerMap.RunUntil(ctx, bpl, waitTime); err != nil {
return nil, err
return "", err
}
rp, err := agent.MysqlDaemon.MasterPosition()
return &rp, err
pos, err := agent.MysqlDaemon.MasterPosition()
if err != nil {
return "", err
}
return myproto.EncodeReplicationPosition(pos), nil
}
//
@ -381,18 +401,18 @@ func (agent *ActionAgent) ResetReplication(ctx context.Context) error {
// InitMaster breaks slaves replication, get the current MySQL replication
// position, insert a row in the reparent_journal table, and returns
// the replication position
func (agent *ActionAgent) InitMaster(ctx context.Context) (myproto.ReplicationPosition, error) {
func (agent *ActionAgent) InitMaster(ctx context.Context) (string, error) {
// we need to insert something in the binlogs, so we can get the
// current position. Let's just use the mysqlctl.CreateReparentJournal commands.
cmds := mysqlctl.CreateReparentJournal()
if err := agent.MysqlDaemon.ExecuteSuperQueryList(cmds); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// get the current replication position
rp, err := agent.MysqlDaemon.MasterPosition()
pos, err := agent.MysqlDaemon.MasterPosition()
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// Set the server read-write, from now on we can accept real
@ -400,24 +420,28 @@ func (agent *ActionAgent) InitMaster(ctx context.Context) (myproto.ReplicationPo
// we'll still need some slaves to be able to commit
// transactions.
if err := agent.MysqlDaemon.SetReadOnly(false); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// Change our type to master if not already
if err := agent.TopoServer.UpdateTabletFields(ctx, agent.TabletAlias, func(tablet *pb.Tablet) error {
tablet.Type = pb.TabletType_MASTER
if err := agent.TopoServer.UpdateTabletFields(ctx, agent.TabletAlias, func(tablet *topodatapb.Tablet) error {
tablet.Type = topodatapb.TabletType_MASTER
tablet.HealthMap = nil
return nil
}); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
agent.initReplication = true
return rp, nil
return myproto.EncodeReplicationPosition(pos), nil
}
// PopulateReparentJournal adds an entry into the reparent_journal table.
func (agent *ActionAgent) PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *pb.TabletAlias, pos myproto.ReplicationPosition) error {
func (agent *ActionAgent) PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, position string) error {
pos, err := myproto.DecodeReplicationPosition(position)
if err != nil {
return err
}
cmds := mysqlctl.CreateReparentJournal()
cmds = append(cmds, mysqlctl.PopulateReparentJournal(timeCreatedNS, actionName, topoproto.TabletAliasString(masterAlias), pos))
@ -426,13 +450,17 @@ func (agent *ActionAgent) PopulateReparentJournal(ctx context.Context, timeCreat
// InitSlave sets replication master and position, and waits for the
// reparent_journal table entry up to context timeout
func (agent *ActionAgent) InitSlave(ctx context.Context, parent *pb.TabletAlias, replicationPosition myproto.ReplicationPosition, timeCreatedNS int64) error {
func (agent *ActionAgent) InitSlave(ctx context.Context, parent *topodatapb.TabletAlias, position string, timeCreatedNS int64) error {
pos, err := myproto.DecodeReplicationPosition(position)
if err != nil {
return err
}
ti, err := agent.TopoServer.GetTablet(ctx, parent)
if err != nil {
return err
}
cmds, err := agent.MysqlDaemon.SetSlavePositionCommands(replicationPosition)
cmds, err := agent.MysqlDaemon.SetSlavePositionCommands(pos)
if err != nil {
return err
}
@ -455,11 +483,11 @@ func (agent *ActionAgent) InitSlave(ctx context.Context, parent *pb.TabletAlias,
// DemoteMaster marks the server read-only, wait until it is done with
// its current transactions, and returns its master position.
// Should be called under RPCWrapLockAction.
func (agent *ActionAgent) DemoteMaster(ctx context.Context) (myproto.ReplicationPosition, error) {
func (agent *ActionAgent) DemoteMaster(ctx context.Context) (string, error) {
// Set the server read-only. Note all active connections are not
// affected.
if err := agent.MysqlDaemon.SetReadOnly(true); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// Now stop the query service, to make sure nobody is writing to the
@ -468,7 +496,11 @@ func (agent *ActionAgent) DemoteMaster(ctx context.Context) (myproto.Replication
tablet := agent.Tablet()
agent.disallowQueries(tablet.Tablet.Type, "DemoteMaster marks server rdonly")
return agent.MysqlDaemon.DemoteMaster()
pos, err := agent.MysqlDaemon.DemoteMaster()
if err != nil {
return "", err
}
return myproto.EncodeReplicationPosition(pos), nil
// There is no serving graph update - the master tablet will
// be replaced. Even though writes may fail, reads will
// succeed. It will be less noisy to simply leave the entry
@ -478,10 +510,14 @@ func (agent *ActionAgent) DemoteMaster(ctx context.Context) (myproto.Replication
// PromoteSlaveWhenCaughtUp waits for this slave to be caught up on
// replication up to the provided point, and then makes the slave the
// shard master.
func (agent *ActionAgent) PromoteSlaveWhenCaughtUp(ctx context.Context, pos myproto.ReplicationPosition) (myproto.ReplicationPosition, error) {
func (agent *ActionAgent) PromoteSlaveWhenCaughtUp(ctx context.Context, position string) (string, error) {
pos, err := myproto.DecodeReplicationPosition(position)
if err != nil {
return "", err
}
tablet, err := agent.TopoServer.GetTablet(ctx, agent.TabletAlias)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// TODO(alainjobart) change the flavor API to take the context directly
@ -494,19 +530,19 @@ func (agent *ActionAgent) PromoteSlaveWhenCaughtUp(ctx context.Context, pos mypr
}
}
if err := agent.MysqlDaemon.WaitMasterPos(pos, waitTimeout); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
rp, err := agent.MysqlDaemon.PromoteSlave(agent.hookExtraEnv())
pos, err = agent.MysqlDaemon.PromoteSlave(agent.hookExtraEnv())
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
if err := agent.MysqlDaemon.SetReadOnly(false); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
return rp, agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
return myproto.EncodeReplicationPosition(pos), agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
}
// SlaveWasPromoted promotes a slave to master, no questions asked.
@ -522,7 +558,7 @@ func (agent *ActionAgent) SlaveWasPromoted(ctx context.Context) error {
// SetMaster sets replication master, and waits for the
// reparent_journal table entry up to context timeout
func (agent *ActionAgent) SetMaster(ctx context.Context, parent *pb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
func (agent *ActionAgent) SetMaster(ctx context.Context, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
ti, err := agent.TopoServer.GetTablet(ctx, parent)
if err != nil {
return err
@ -562,8 +598,8 @@ func (agent *ActionAgent) SetMaster(ctx context.Context, parent *pb.TabletAlias,
if err != nil {
return err
}
if tablet.Type == pb.TabletType_MASTER {
tablet.Type = pb.TabletType_SPARE
if tablet.Type == topodatapb.TabletType_MASTER {
tablet.Type = topodatapb.TabletType_SPARE
tablet.HealthMap = nil
if err := agent.TopoServer.UpdateTablet(ctx, tablet); err != nil {
return err
@ -587,8 +623,8 @@ func (agent *ActionAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnod
}
// Once this action completes, update authoritative tablet node first.
if tablet.Type == pb.TabletType_MASTER {
tablet.Type = pb.TabletType_SPARE
if tablet.Type == topodatapb.TabletType_MASTER {
tablet.Type = topodatapb.TabletType_SPARE
}
err = agent.TopoServer.UpdateTablet(ctx, tablet)
if err != nil {
@ -607,52 +643,52 @@ func (agent *ActionAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnod
// StopReplicationAndGetStatus stops MySQL replication, and returns the
// current status
func (agent *ActionAgent) StopReplicationAndGetStatus(ctx context.Context) (myproto.ReplicationStatus, error) {
func (agent *ActionAgent) StopReplicationAndGetStatus(ctx context.Context) (*replicationdatapb.Status, error) {
// get the status before we stop replication
rs, err := agent.MysqlDaemon.SlaveStatus()
if err != nil {
return myproto.ReplicationStatus{}, fmt.Errorf("before status failed: %v", err)
return nil, fmt.Errorf("before status failed: %v", err)
}
if !rs.SlaveIORunning && !rs.SlaveSQLRunning {
// no replication is running, just return what we got
return rs, nil
return myproto.ReplicationStatusToProto(rs), nil
}
if err := mysqlctl.StopSlave(agent.MysqlDaemon, agent.hookExtraEnv()); err != nil {
return myproto.ReplicationStatus{}, fmt.Errorf("stop slave failed: %v", err)
return nil, fmt.Errorf("stop slave failed: %v", err)
}
// now patch in the current position
rs.Position, err = agent.MysqlDaemon.MasterPosition()
if err != nil {
return myproto.ReplicationStatus{}, fmt.Errorf("after position failed: %v", err)
return nil, fmt.Errorf("after position failed: %v", err)
}
return rs, nil
return myproto.ReplicationStatusToProto(rs), nil
}
// PromoteSlave makes the current tablet the master
func (agent *ActionAgent) PromoteSlave(ctx context.Context) (myproto.ReplicationPosition, error) {
func (agent *ActionAgent) PromoteSlave(ctx context.Context) (string, error) {
tablet, err := agent.TopoServer.GetTablet(ctx, agent.TabletAlias)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
rp, err := agent.MysqlDaemon.PromoteSlave(agent.hookExtraEnv())
pos, err := agent.MysqlDaemon.PromoteSlave(agent.hookExtraEnv())
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
// Set the server read-write
if err := agent.MysqlDaemon.SetReadOnly(false); err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
return rp, agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
return myproto.EncodeReplicationPosition(pos), agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
}
// updateReplicationGraphForPromotedSlave makes sure the newly promoted slave
// is correctly represented in the replication graph
func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(ctx context.Context, tablet *topo.TabletInfo) error {
// Update tablet regardless - trend towards consistency.
tablet.Type = pb.TabletType_MASTER
tablet.Type = topodatapb.TabletType_MASTER
tablet.HealthMap = nil
err := agent.TopoServer.UpdateTablet(ctx, tablet)
if err != nil {
@ -685,11 +721,11 @@ func (agent *ActionAgent) Backup(ctx context.Context, concurrency int, logger lo
if err != nil {
return err
}
if tablet.Type == pb.TabletType_MASTER {
if tablet.Type == topodatapb.TabletType_MASTER {
return fmt.Errorf("type MASTER cannot take backup, if you really need to do this, restart vttablet in replica mode")
}
originalType := tablet.Type
if err := topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, pb.TabletType_BACKUP, make(map[string]string)); err != nil {
if err := topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, topodatapb.TabletType_BACKUP, make(map[string]string)); err != nil {
return err
}
@ -710,7 +746,7 @@ func (agent *ActionAgent) Backup(ctx context.Context, concurrency int, logger lo
// - if healthcheck is enabled, go to spare
// - if not, go back to original type
if agent.IsRunningHealthCheck() {
originalType = pb.TabletType_SPARE
originalType = topodatapb.TabletType_SPARE
}
err = topotools.ChangeType(ctx, agent.TopoServer, tablet.Alias, originalType, nil)
if err != nil {

Просмотреть файл

@ -23,9 +23,10 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/proto/query"
pbt "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
querypb "github.com/youtube/vitess/go/vt/proto/query"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// fakeRPCAgent implements tabletmanager.RPCAgent and fills in all
@ -141,9 +142,9 @@ func agentRPCTestPingPanic(ctx context.Context, t *testing.T, client tmclient.Ta
var testGetSchemaTables = []string{"table1", "table2"}
var testGetSchemaExcludeTables = []string{"etable1", "etable2", "etable3"}
var testGetSchemaReply = &pbt.SchemaDefinition{
var testGetSchemaReply = &tabletmanagerdatapb.SchemaDefinition{
DatabaseSchema: "CREATE DATABASE {{.DatabaseName}}",
TableDefinitions: []*pbt.TableDefinition{
TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
{
Name: "table_name",
Schema: "create table_name",
@ -166,7 +167,7 @@ var testGetSchemaReply = &pbt.SchemaDefinition{
Version: "xxx",
}
func (fra *fakeRPCAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*pbt.SchemaDefinition, error) {
func (fra *fakeRPCAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -186,8 +187,8 @@ func agentRPCTestGetSchemaPanic(ctx context.Context, t *testing.T, client tmclie
expectRPCWrapPanic(t, err)
}
var testGetPermissionsReply = &pbt.Permissions{
UserPermissions: []*pbt.UserPermission{
var testGetPermissionsReply = &tabletmanagerdatapb.Permissions{
UserPermissions: []*tabletmanagerdatapb.UserPermission{
{
Host: "host1",
User: "user1",
@ -198,7 +199,7 @@ var testGetPermissionsReply = &pbt.Permissions{
},
},
},
DbPermissions: []*pbt.DbPermission{
DbPermissions: []*tabletmanagerdatapb.DbPermission{
{
Host: "host2",
Db: "db1",
@ -209,7 +210,7 @@ var testGetPermissionsReply = &pbt.Permissions{
},
},
},
HostPermissions: []*pbt.HostPermission{
HostPermissions: []*tabletmanagerdatapb.HostPermission{
{
Host: "host3",
Db: "db2",
@ -221,7 +222,7 @@ var testGetPermissionsReply = &pbt.Permissions{
},
}
func (fra *fakeRPCAgent) GetPermissions(ctx context.Context) (*pbt.Permissions, error) {
func (fra *fakeRPCAgent) GetPermissions(ctx context.Context) (*tabletmanagerdatapb.Permissions, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -272,9 +273,9 @@ func agentRPCTestSetReadOnlyPanic(ctx context.Context, t *testing.T, client tmcl
expectRPCWrapLockActionPanic(t, err)
}
var testChangeTypeValue = pb.TabletType_REPLICA
var testChangeTypeValue = topodatapb.TabletType_REPLICA
func (fra *fakeRPCAgent) ChangeType(ctx context.Context, tabletType pb.TabletType) error {
func (fra *fakeRPCAgent) ChangeType(ctx context.Context, tabletType topodatapb.TabletType) error {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -374,9 +375,9 @@ func agentRPCTestRefreshStatePanic(ctx context.Context, t *testing.T, client tmc
expectRPCWrapLockActionPanic(t, err)
}
var testRunHealthCheckValue = pb.TabletType_RDONLY
var testRunHealthCheckValue = topodatapb.TabletType_RDONLY
func (fra *fakeRPCAgent) RunHealthCheck(ctx context.Context, targetTabletType pb.TabletType) {
func (fra *fakeRPCAgent) RunHealthCheck(ctx context.Context, targetTabletType topodatapb.TabletType) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -475,12 +476,12 @@ func agentRPCTestApplySchemaPanic(ctx context.Context, t *testing.T, client tmcl
var testExecuteFetchQuery = "fetch this"
var testExecuteFetchMaxRows = 100
var testExecuteFetchResult = &mproto.QueryResult{
Fields: []*query.Field{
&query.Field{
Fields: []*querypb.Field{
&querypb.Field{
Name: "column1",
Type: sqltypes.Blob,
},
&query.Field{
&querypb.Field{
Name: "column2",
Type: sqltypes.Datetime,
},
@ -536,23 +537,17 @@ func agentRPCTestExecuteFetchPanic(ctx context.Context, t *testing.T, client tmc
// Replication related methods
//
var testReplicationStatus = myproto.ReplicationStatus{
Position: myproto.ReplicationPosition{
GTIDSet: myproto.MariadbGTID{
Domain: 1,
Server: 345,
Sequence: 789,
},
},
SlaveIORunning: true,
SlaveSQLRunning: true,
var testReplicationStatus = &replicationdatapb.Status{
Position: "MariaDB/1-345-789",
SlaveIoRunning: true,
SlaveSqlRunning: true,
SecondsBehindMaster: 654,
MasterHost: "master.host",
MasterPort: 3366,
MasterConnectRetry: 12,
}
func (fra *fakeRPCAgent) SlaveStatus(ctx context.Context) (myproto.ReplicationStatus, error) {
func (fra *fakeRPCAgent) SlaveStatus(ctx context.Context) (*replicationdatapb.Status, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -569,15 +564,9 @@ func agentRPCTestSlaveStatusPanic(ctx context.Context, t *testing.T, client tmcl
expectRPCWrapPanic(t, err)
}
var testReplicationPosition = myproto.ReplicationPosition{
GTIDSet: myproto.MariadbGTID{
Domain: 5,
Server: 456,
Sequence: 890,
},
}
var testReplicationPosition = "MariaDB/5-456-890"
func (fra *fakeRPCAgent) MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) MasterPosition(ctx context.Context) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -616,11 +605,11 @@ func agentRPCTestStopSlavePanic(ctx context.Context, t *testing.T, client tmclie
var testStopSlaveMinimumWaitTime = time.Hour
func (fra *fakeRPCAgent) StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) StopSlaveMinimum(ctx context.Context, position string, waitTime time.Duration) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
compare(fra.t, "StopSlaveMinimum position", position.GTIDSet, testReplicationPosition.GTIDSet)
compare(fra.t, "StopSlaveMinimum position", position, testReplicationPosition)
compare(fra.t, "StopSlaveMinimum waitTime", waitTime, testStopSlaveMinimumWaitTime)
return testReplicationPositionReturned, nil
}
@ -694,14 +683,14 @@ func agentRPCTestGetSlavesPanic(ctx context.Context, t *testing.T, client tmclie
expectRPCWrapPanic(t, err)
}
var testBlpPosition = &pbt.BlpPosition{
var testBlpPosition = &tabletmanagerdatapb.BlpPosition{
Uid: 73,
Position: "testReplicationPosition",
}
var testWaitBlpPositionWaitTime = time.Hour
var testWaitBlpPositionCalled = false
func (fra *fakeRPCAgent) WaitBlpPosition(ctx context.Context, blpPosition *pbt.BlpPosition, waitTime time.Duration) error {
func (fra *fakeRPCAgent) WaitBlpPosition(ctx context.Context, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -721,14 +710,14 @@ func agentRPCTestWaitBlpPositionPanic(ctx context.Context, t *testing.T, client
expectRPCWrapLockPanic(t, err)
}
var testBlpPositionList = []*pbt.BlpPosition{
var testBlpPositionList = []*tabletmanagerdatapb.BlpPosition{
{
Uid: 12,
Position: "testBlpPosition",
},
}
func (fra *fakeRPCAgent) StopBlp(ctx context.Context) ([]*pbt.BlpPosition, error) {
func (fra *fakeRPCAgent) StopBlp(ctx context.Context) ([]*tabletmanagerdatapb.BlpPosition, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -767,13 +756,13 @@ func agentRPCTestStartBlpPanic(ctx context.Context, t *testing.T, client tmclien
var testRunBlpUntilWaitTime = 3 * time.Minute
func (fra *fakeRPCAgent) RunBlpUntil(ctx context.Context, bpl []*pbt.BlpPosition, waitTime time.Duration) (*myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) RunBlpUntil(ctx context.Context, bpl []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
compare(fra.t, "RunBlpUntil bpl", bpl, testBlpPositionList)
compare(fra.t, "RunBlpUntil waitTime", waitTime, testRunBlpUntilWaitTime)
return &testReplicationPosition, nil
return testReplicationPosition, nil
}
func agentRPCTestRunBlpUntil(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
@ -810,7 +799,7 @@ func agentRPCTestResetReplicationPanic(ctx context.Context, t *testing.T, client
expectRPCWrapLockActionPanic(t, err)
}
func (fra *fakeRPCAgent) InitMaster(ctx context.Context) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) InitMaster(ctx context.Context) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -830,19 +819,19 @@ func agentRPCTestInitMasterPanic(ctx context.Context, t *testing.T, client tmcli
var testPopulateReparentJournalCalled = false
var testTimeCreatedNS int64 = 4569900
var testActionName = "TestActionName"
var testMasterAlias = &pb.TabletAlias{
var testMasterAlias = &topodatapb.TabletAlias{
Cell: "ce",
Uid: 372,
}
func (fra *fakeRPCAgent) PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *pb.TabletAlias, pos myproto.ReplicationPosition) error {
func (fra *fakeRPCAgent) PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, position string) error {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
compare(fra.t, "PopulateReparentJournal timeCreatedNS", timeCreatedNS, testTimeCreatedNS)
compare(fra.t, "PopulateReparentJournal actionName", actionName, testActionName)
compare(fra.t, "PopulateReparentJournal masterAlias", masterAlias, testMasterAlias)
compare(fra.t, "PopulateReparentJournal pos", pos, testReplicationPosition)
compare(fra.t, "PopulateReparentJournal pos", position, testReplicationPosition)
testPopulateReparentJournalCalled = true
return nil
}
@ -859,12 +848,12 @@ func agentRPCTestPopulateReparentJournalPanic(ctx context.Context, t *testing.T,
var testInitSlaveCalled = false
func (fra *fakeRPCAgent) InitSlave(ctx context.Context, parent *pb.TabletAlias, pos myproto.ReplicationPosition, timeCreatedNS int64) error {
func (fra *fakeRPCAgent) InitSlave(ctx context.Context, parent *topodatapb.TabletAlias, position string, timeCreatedNS int64) error {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
compare(fra.t, "InitSlave parent", parent, testMasterAlias)
compare(fra.t, "InitSlave pos", pos, testReplicationPosition)
compare(fra.t, "InitSlave pos", position, testReplicationPosition)
compare(fra.t, "InitSlave timeCreatedNS", timeCreatedNS, testTimeCreatedNS)
testInitSlaveCalled = true
return nil
@ -880,7 +869,7 @@ func agentRPCTestInitSlavePanic(ctx context.Context, t *testing.T, client tmclie
expectRPCWrapLockActionPanic(t, err)
}
func (fra *fakeRPCAgent) DemoteMaster(ctx context.Context) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) DemoteMaster(ctx context.Context) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -897,19 +886,13 @@ func agentRPCTestDemoteMasterPanic(ctx context.Context, t *testing.T, client tmc
expectRPCWrapLockActionPanic(t, err)
}
var testReplicationPositionReturned = myproto.ReplicationPosition{
GTIDSet: myproto.MariadbGTID{
Domain: 5,
Server: 567,
Sequence: 3456,
},
}
var testReplicationPositionReturned = "MariaDB/5-567-3456"
func (fra *fakeRPCAgent) PromoteSlaveWhenCaughtUp(ctx context.Context, pos myproto.ReplicationPosition) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) PromoteSlaveWhenCaughtUp(ctx context.Context, position string) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
compare(fra.t, "PromoteSlaveWhenCaughtUp pos", pos, testReplicationPosition)
compare(fra.t, "PromoteSlaveWhenCaughtUp pos", position, testReplicationPosition)
return testReplicationPositionReturned, nil
}
@ -946,7 +929,7 @@ func agentRPCTestSlaveWasPromotedPanic(ctx context.Context, t *testing.T, client
var testSetMasterCalled = false
var testForceStartSlave = true
func (fra *fakeRPCAgent) SetMaster(ctx context.Context, parent *pb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
func (fra *fakeRPCAgent) SetMaster(ctx context.Context, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -968,7 +951,7 @@ func agentRPCTestSetMasterPanic(ctx context.Context, t *testing.T, client tmclie
}
var testSlaveWasRestartedArgs = &actionnode.SlaveWasRestartedArgs{
Parent: &pb.TabletAlias{
Parent: &topodatapb.TabletAlias{
Cell: "prison",
Uid: 42,
},
@ -994,7 +977,7 @@ func agentRPCTestSlaveWasRestartedPanic(ctx context.Context, t *testing.T, clien
expectRPCWrapLockActionPanic(t, err)
}
func (fra *fakeRPCAgent) StopReplicationAndGetStatus(ctx context.Context) (myproto.ReplicationStatus, error) {
func (fra *fakeRPCAgent) StopReplicationAndGetStatus(ctx context.Context) (*replicationdatapb.Status, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}
@ -1011,7 +994,7 @@ func agentRPCTestStopReplicationAndGetStatusPanic(ctx context.Context, t *testin
expectRPCWrapLockActionPanic(t, err)
}
func (fra *fakeRPCAgent) PromoteSlave(ctx context.Context) (myproto.ReplicationPosition, error) {
func (fra *fakeRPCAgent) PromoteSlave(ctx context.Context) (string, error) {
if fra.panics {
panic(fmt.Errorf("test-triggered panic"))
}

Просмотреть файл

@ -22,8 +22,9 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
pbt "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
type timeoutError error
@ -72,15 +73,13 @@ func (client *FakeTabletManagerClient) ExecuteHook(ctx context.Context, tablet *
}
// GetSchema is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*pbt.SchemaDefinition, error) {
func (client *FakeTabletManagerClient) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) {
return client.tmc.GetSchema(ctx, tablet, tables, excludeTables, includeViews)
// var sd myproto.SchemaDefinition
// return &sd, nil
}
// GetPermissions is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*pbt.Permissions, error) {
return &pbt.Permissions{}, nil
func (client *FakeTabletManagerClient) GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*tabletmanagerdatapb.Permissions, error) {
return &tabletmanagerdatapb.Permissions{}, nil
}
//
@ -98,7 +97,7 @@ func (client *FakeTabletManagerClient) SetReadWrite(ctx context.Context, tablet
}
// ChangeType is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType pb.TabletType) error {
func (client *FakeTabletManagerClient) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topodatapb.TabletType) error {
return nil
}
@ -108,7 +107,7 @@ func (client *FakeTabletManagerClient) RefreshState(ctx context.Context, tablet
}
// RunHealthCheck is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType pb.TabletType) error {
func (client *FakeTabletManagerClient) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topodatapb.TabletType) error {
return nil
}
@ -146,15 +145,13 @@ func (client *FakeTabletManagerClient) ExecuteFetchAsApp(ctx context.Context, ta
//
// SlaveStatus is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error) {
var status myproto.ReplicationStatus
return status, nil
func (client *FakeTabletManagerClient) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error) {
return &replicationdatapb.Status{}, nil
}
// MasterPosition is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
var rp myproto.ReplicationPosition
return rp, nil
func (client *FakeTabletManagerClient) MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
return "", nil
}
// StopSlave is part of the tmclient.TabletManagerClient interface
@ -163,9 +160,8 @@ func (client *FakeTabletManagerClient) StopSlave(ctx context.Context, tablet *to
}
// StopSlaveMinimum is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, minPos myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
var pos myproto.ReplicationPosition
return pos, nil
func (client *FakeTabletManagerClient) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, minPos string, waitTime time.Duration) (string, error) {
return "", nil
}
// StartSlave is part of the tmclient.TabletManagerClient interface
@ -184,16 +180,16 @@ func (client *FakeTabletManagerClient) GetSlaves(ctx context.Context, tablet *to
}
// WaitBlpPosition is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *pbt.BlpPosition, waitTime time.Duration) error {
func (client *FakeTabletManagerClient) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error {
return nil
}
// StopBlp is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*pbt.BlpPosition, error) {
func (client *FakeTabletManagerClient) StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*tabletmanagerdatapb.BlpPosition, error) {
// TODO(aaijazi): this works because all tests so far only need to rely on Uid 0.
// Ideally, this should turn into a full mock, where the caller can configure the exact
// return value.
bpl := []*pbt.BlpPosition{
bpl := []*tabletmanagerdatapb.BlpPosition{
{
Uid: uint32(0),
},
@ -207,9 +203,8 @@ func (client *FakeTabletManagerClient) StartBlp(ctx context.Context, tablet *top
}
// RunBlpUntil is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*pbt.BlpPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
var pos myproto.ReplicationPosition
return pos, nil
func (client *FakeTabletManagerClient) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error) {
return "", nil
}
//
@ -222,30 +217,28 @@ func (client *FakeTabletManagerClient) ResetReplication(ctx context.Context, tab
}
// InitMaster is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) InitMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
return myproto.ReplicationPosition{}, nil
func (client *FakeTabletManagerClient) InitMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
return "", nil
}
// PopulateReparentJournal is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *pb.TabletAlias, pos myproto.ReplicationPosition) error {
func (client *FakeTabletManagerClient) PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, position string) error {
return nil
}
// InitSlave is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *pb.TabletAlias, replicationPosition myproto.ReplicationPosition, timeCreatedNS int64) error {
func (client *FakeTabletManagerClient) InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, position string, timeCreatedNS int64) error {
return nil
}
// DemoteMaster is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
var rp myproto.ReplicationPosition
return rp, nil
func (client *FakeTabletManagerClient) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
return "", nil
}
// PromoteSlaveWhenCaughtUp is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, pos myproto.ReplicationPosition) (myproto.ReplicationPosition, error) {
var rp myproto.ReplicationPosition
return rp, nil
func (client *FakeTabletManagerClient) PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, position string) (string, error) {
return "", nil
}
// SlaveWasPromoted is part of the tmclient.TabletManagerClient interface
@ -254,7 +247,7 @@ func (client *FakeTabletManagerClient) SlaveWasPromoted(ctx context.Context, tab
}
// SetMaster is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *pb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
func (client *FakeTabletManagerClient) SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
return nil
}
@ -264,15 +257,13 @@ func (client *FakeTabletManagerClient) SlaveWasRestarted(ctx context.Context, ta
}
// StopReplicationAndGetStatus is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error) {
var rp myproto.ReplicationStatus
return rp, nil
func (client *FakeTabletManagerClient) StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error) {
return &replicationdatapb.Status{}, nil
}
// PromoteSlave is part of the tmclient.TabletManagerClient interface
func (client *FakeTabletManagerClient) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
var rp myproto.ReplicationPosition
return rp, nil
func (client *FakeTabletManagerClient) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
return "", nil
}
//

Просмотреть файл

@ -21,9 +21,10 @@ import (
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
pb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pbs "github.com/youtube/vitess/go/vt/proto/tabletmanagerservice"
pbt "github.com/youtube/vitess/go/vt/proto/topodata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
tabletmanagerservicepb "github.com/youtube/vitess/go/vt/proto/tabletmanagerservice"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
type timeoutError struct {
@ -40,7 +41,7 @@ func init() {
type Client struct{}
// dial returns a client to use
func (client *Client) dial(ctx context.Context, tablet *topo.TabletInfo) (*grpc.ClientConn, pbs.TabletManagerClient, error) {
func (client *Client) dial(ctx context.Context, tablet *topo.TabletInfo) (*grpc.ClientConn, tabletmanagerservicepb.TabletManagerClient, error) {
// create the RPC client, using ctx.Deadline if set, or no timeout.
var connectTimeout time.Duration
deadline, ok := ctx.Deadline()
@ -62,7 +63,7 @@ func (client *Client) dial(ctx context.Context, tablet *topo.TabletInfo) (*grpc.
if err != nil {
return nil, nil, err
}
return cc, pbs.NewTabletManagerClient(cc), nil
return cc, tabletmanagerservicepb.NewTabletManagerClient(cc), nil
}
//
@ -76,7 +77,7 @@ func (client *Client) Ping(ctx context.Context, tablet *topo.TabletInfo) error {
return err
}
defer cc.Close()
result, err := c.Ping(ctx, &pb.PingRequest{
result, err := c.Ping(ctx, &tabletmanagerdatapb.PingRequest{
Payload: "payload",
})
if err != nil {
@ -95,7 +96,7 @@ func (client *Client) Sleep(ctx context.Context, tablet *topo.TabletInfo, durati
return err
}
defer cc.Close()
_, err = c.Sleep(ctx, &pb.SleepRequest{
_, err = c.Sleep(ctx, &tabletmanagerdatapb.SleepRequest{
Duration: int64(duration),
})
return err
@ -108,7 +109,7 @@ func (client *Client) ExecuteHook(ctx context.Context, tablet *topo.TabletInfo,
return nil, err
}
defer cc.Close()
hr, err := c.ExecuteHook(ctx, &pb.ExecuteHookRequest{
hr, err := c.ExecuteHook(ctx, &tabletmanagerdatapb.ExecuteHookRequest{
Name: hk.Name,
Parameters: hk.Parameters,
ExtraEnv: hk.ExtraEnv,
@ -124,13 +125,13 @@ func (client *Client) ExecuteHook(ctx context.Context, tablet *topo.TabletInfo,
}
// GetSchema is part of the tmclient.TabletManagerClient interface
func (client *Client) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*pb.SchemaDefinition, error) {
func (client *Client) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return nil, err
}
defer cc.Close()
response, err := c.GetSchema(ctx, &pb.GetSchemaRequest{
response, err := c.GetSchema(ctx, &tabletmanagerdatapb.GetSchemaRequest{
Tables: tables,
ExcludeTables: excludeTables,
IncludeViews: includeViews,
@ -142,13 +143,13 @@ func (client *Client) GetSchema(ctx context.Context, tablet *topo.TabletInfo, ta
}
// GetPermissions is part of the tmclient.TabletManagerClient interface
func (client *Client) GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*pb.Permissions, error) {
func (client *Client) GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*tabletmanagerdatapb.Permissions, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return nil, err
}
defer cc.Close()
response, err := c.GetPermissions(ctx, &pb.GetPermissionsRequest{})
response, err := c.GetPermissions(ctx, &tabletmanagerdatapb.GetPermissionsRequest{})
if err != nil {
return nil, err
}
@ -166,7 +167,7 @@ func (client *Client) SetReadOnly(ctx context.Context, tablet *topo.TabletInfo)
return err
}
defer cc.Close()
_, err = c.SetReadOnly(ctx, &pb.SetReadOnlyRequest{})
_, err = c.SetReadOnly(ctx, &tabletmanagerdatapb.SetReadOnlyRequest{})
return err
}
@ -177,18 +178,18 @@ func (client *Client) SetReadWrite(ctx context.Context, tablet *topo.TabletInfo)
return err
}
defer cc.Close()
_, err = c.SetReadWrite(ctx, &pb.SetReadWriteRequest{})
_, err = c.SetReadWrite(ctx, &tabletmanagerdatapb.SetReadWriteRequest{})
return err
}
// ChangeType is part of the tmclient.TabletManagerClient interface
func (client *Client) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType pbt.TabletType) error {
func (client *Client) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topodatapb.TabletType) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.ChangeType(ctx, &pb.ChangeTypeRequest{
_, err = c.ChangeType(ctx, &tabletmanagerdatapb.ChangeTypeRequest{
TabletType: dbType,
})
return err
@ -201,18 +202,18 @@ func (client *Client) RefreshState(ctx context.Context, tablet *topo.TabletInfo)
return err
}
defer cc.Close()
_, err = c.RefreshState(ctx, &pb.RefreshStateRequest{})
_, err = c.RefreshState(ctx, &tabletmanagerdatapb.RefreshStateRequest{})
return err
}
// RunHealthCheck is part of the tmclient.TabletManagerClient interface
func (client *Client) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType pbt.TabletType) error {
func (client *Client) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topodatapb.TabletType) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.RunHealthCheck(ctx, &pb.RunHealthCheckRequest{
_, err = c.RunHealthCheck(ctx, &tabletmanagerdatapb.RunHealthCheckRequest{
TabletType: targetTabletType,
})
return err
@ -225,7 +226,7 @@ func (client *Client) ReloadSchema(ctx context.Context, tablet *topo.TabletInfo)
return err
}
defer cc.Close()
_, err = c.ReloadSchema(ctx, &pb.ReloadSchemaRequest{})
_, err = c.ReloadSchema(ctx, &tabletmanagerdatapb.ReloadSchemaRequest{})
return err
}
@ -236,7 +237,7 @@ func (client *Client) PreflightSchema(ctx context.Context, tablet *topo.TabletIn
return nil, err
}
defer cc.Close()
response, err := c.PreflightSchema(ctx, &pb.PreflightSchemaRequest{
response, err := c.PreflightSchema(ctx, &tabletmanagerdatapb.PreflightSchemaRequest{
Change: change,
})
if err != nil {
@ -255,7 +256,7 @@ func (client *Client) ApplySchema(ctx context.Context, tablet *topo.TabletInfo,
return nil, err
}
defer cc.Close()
response, err := c.ApplySchema(ctx, &pb.ApplySchemaRequest{
response, err := c.ApplySchema(ctx, &tabletmanagerdatapb.ApplySchemaRequest{
Sql: change.Sql,
Force: change.Force,
AllowReplication: change.AllowReplication,
@ -278,7 +279,7 @@ func (client *Client) ExecuteFetchAsDba(ctx context.Context, tablet *topo.Tablet
return nil, err
}
defer cc.Close()
response, err := c.ExecuteFetchAsDba(ctx, &pb.ExecuteFetchAsDbaRequest{
response, err := c.ExecuteFetchAsDba(ctx, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{
Query: query,
DbName: tablet.DbName(),
MaxRows: uint64(maxRows),
@ -299,7 +300,7 @@ func (client *Client) ExecuteFetchAsApp(ctx context.Context, tablet *topo.Tablet
return nil, err
}
defer cc.Close()
response, err := c.ExecuteFetchAsApp(ctx, &pb.ExecuteFetchAsAppRequest{
response, err := c.ExecuteFetchAsApp(ctx, &tabletmanagerdatapb.ExecuteFetchAsAppRequest{
Query: query,
MaxRows: uint64(maxRows),
WantFields: wantFields,
@ -315,35 +316,31 @@ func (client *Client) ExecuteFetchAsApp(ctx context.Context, tablet *topo.Tablet
//
// SlaveStatus is part of the tmclient.TabletManagerClient interface
func (client *Client) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error) {
func (client *Client) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationStatus{}, err
return nil, err
}
defer cc.Close()
response, err := c.SlaveStatus(ctx, &pb.SlaveStatusRequest{})
response, err := c.SlaveStatus(ctx, &tabletmanagerdatapb.SlaveStatusRequest{})
if err != nil {
return myproto.ReplicationStatus{}, err
return nil, err
}
return myproto.ProtoToReplicationStatus(response.Status), nil
return response.Status, nil
}
// MasterPosition is part of the tmclient.TabletManagerClient interface
func (client *Client) MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
func (client *Client) MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.MasterPosition(ctx, &pb.MasterPositionRequest{})
response, err := c.MasterPosition(ctx, &tabletmanagerdatapb.MasterPositionRequest{})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
// StopSlave is part of the tmclient.TabletManagerClient interface
@ -353,29 +350,25 @@ func (client *Client) StopSlave(ctx context.Context, tablet *topo.TabletInfo) er
return err
}
defer cc.Close()
_, err = c.StopSlave(ctx, &pb.StopSlaveRequest{})
_, err = c.StopSlave(ctx, &tabletmanagerdatapb.StopSlaveRequest{})
return err
}
// StopSlaveMinimum is part of the tmclient.TabletManagerClient interface
func (client *Client) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, minPos myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
func (client *Client) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, minPos string, waitTime time.Duration) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.StopSlaveMinimum(ctx, &pb.StopSlaveMinimumRequest{
Position: myproto.EncodeReplicationPosition(minPos),
response, err := c.StopSlaveMinimum(ctx, &tabletmanagerdatapb.StopSlaveMinimumRequest{
Position: minPos,
WaitTimeout: int64(waitTime),
})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
// StartSlave is part of the tmclient.TabletManagerClient interface
@ -385,7 +378,7 @@ func (client *Client) StartSlave(ctx context.Context, tablet *topo.TabletInfo) e
return err
}
defer cc.Close()
_, err = c.StartSlave(ctx, &pb.StartSlaveRequest{})
_, err = c.StartSlave(ctx, &tabletmanagerdatapb.StartSlaveRequest{})
return err
}
@ -396,7 +389,7 @@ func (client *Client) TabletExternallyReparented(ctx context.Context, tablet *to
return err
}
defer cc.Close()
_, err = c.TabletExternallyReparented(ctx, &pb.TabletExternallyReparentedRequest{
_, err = c.TabletExternallyReparented(ctx, &tabletmanagerdatapb.TabletExternallyReparentedRequest{
ExternalId: externalID,
})
return err
@ -409,7 +402,7 @@ func (client *Client) GetSlaves(ctx context.Context, tablet *topo.TabletInfo) ([
return nil, err
}
defer cc.Close()
response, err := c.GetSlaves(ctx, &pb.GetSlavesRequest{})
response, err := c.GetSlaves(ctx, &tabletmanagerdatapb.GetSlavesRequest{})
if err != nil {
return nil, err
}
@ -417,13 +410,13 @@ func (client *Client) GetSlaves(ctx context.Context, tablet *topo.TabletInfo) ([
}
// WaitBlpPosition is part of the tmclient.TabletManagerClient interface
func (client *Client) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *pb.BlpPosition, waitTime time.Duration) error {
func (client *Client) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.WaitBlpPosition(ctx, &pb.WaitBlpPositionRequest{
_, err = c.WaitBlpPosition(ctx, &tabletmanagerdatapb.WaitBlpPositionRequest{
BlpPosition: blpPosition,
WaitTimeout: int64(waitTime),
})
@ -431,13 +424,13 @@ func (client *Client) WaitBlpPosition(ctx context.Context, tablet *topo.TabletIn
}
// StopBlp is part of the tmclient.TabletManagerClient interface
func (client *Client) StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*pb.BlpPosition, error) {
func (client *Client) StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*tabletmanagerdatapb.BlpPosition, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return nil, err
}
defer cc.Close()
response, err := c.StopBlp(ctx, &pb.StopBlpRequest{})
response, err := c.StopBlp(ctx, &tabletmanagerdatapb.StopBlpRequest{})
if err != nil {
return nil, err
}
@ -451,29 +444,25 @@ func (client *Client) StartBlp(ctx context.Context, tablet *topo.TabletInfo) err
return err
}
defer cc.Close()
_, err = c.StartBlp(ctx, &pb.StartBlpRequest{})
_, err = c.StartBlp(ctx, &tabletmanagerdatapb.StartBlpRequest{})
return err
}
// RunBlpUntil is part of the tmclient.TabletManagerClient interface
func (client *Client) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*pb.BlpPosition, waitTime time.Duration) (myproto.ReplicationPosition, error) {
func (client *Client) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.RunBlpUntil(ctx, &pb.RunBlpUntilRequest{
response, err := c.RunBlpUntil(ctx, &tabletmanagerdatapb.RunBlpUntilRequest{
BlpPositions: positions,
WaitTimeout: int64(waitTime),
})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
//
@ -487,95 +476,83 @@ func (client *Client) ResetReplication(ctx context.Context, tablet *topo.TabletI
return err
}
defer cc.Close()
_, err = c.ResetReplication(ctx, &pb.ResetReplicationRequest{})
_, err = c.ResetReplication(ctx, &tabletmanagerdatapb.ResetReplicationRequest{})
return err
}
// InitMaster is part of the tmclient.TabletManagerClient interface
func (client *Client) InitMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
func (client *Client) InitMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.InitMaster(ctx, &pb.InitMasterRequest{})
response, err := c.InitMaster(ctx, &tabletmanagerdatapb.InitMasterRequest{})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
// PopulateReparentJournal is part of the tmclient.TabletManagerClient interface
func (client *Client) PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *pbt.TabletAlias, pos myproto.ReplicationPosition) error {
func (client *Client) PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, pos string) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.PopulateReparentJournal(ctx, &pb.PopulateReparentJournalRequest{
_, err = c.PopulateReparentJournal(ctx, &tabletmanagerdatapb.PopulateReparentJournalRequest{
TimeCreatedNs: timeCreatedNS,
ActionName: actionName,
MasterAlias: masterAlias,
ReplicationPosition: myproto.EncodeReplicationPosition(pos),
ReplicationPosition: pos,
})
return err
}
// InitSlave is part of the tmclient.TabletManagerClient interface
func (client *Client) InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *pbt.TabletAlias, replicationPosition myproto.ReplicationPosition, timeCreatedNS int64) error {
func (client *Client) InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, replicationPosition string, timeCreatedNS int64) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.InitSlave(ctx, &pb.InitSlaveRequest{
_, err = c.InitSlave(ctx, &tabletmanagerdatapb.InitSlaveRequest{
Parent: parent,
ReplicationPosition: myproto.EncodeReplicationPosition(replicationPosition),
ReplicationPosition: replicationPosition,
TimeCreatedNs: timeCreatedNS,
})
return err
}
// DemoteMaster is part of the tmclient.TabletManagerClient interface
func (client *Client) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
func (client *Client) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.DemoteMaster(ctx, &pb.DemoteMasterRequest{})
response, err := c.DemoteMaster(ctx, &tabletmanagerdatapb.DemoteMasterRequest{})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
// PromoteSlaveWhenCaughtUp is part of the tmclient.TabletManagerClient interface
func (client *Client) PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, pos myproto.ReplicationPosition) (myproto.ReplicationPosition, error) {
func (client *Client) PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, pos string) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.PromoteSlaveWhenCaughtUp(ctx, &pb.PromoteSlaveWhenCaughtUpRequest{
Position: myproto.EncodeReplicationPosition(pos),
response, err := c.PromoteSlaveWhenCaughtUp(ctx, &tabletmanagerdatapb.PromoteSlaveWhenCaughtUpRequest{
Position: pos,
})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
// SlaveWasPromoted is part of the tmclient.TabletManagerClient interface
@ -585,18 +562,18 @@ func (client *Client) SlaveWasPromoted(ctx context.Context, tablet *topo.TabletI
return err
}
defer cc.Close()
_, err = c.SlaveWasPromoted(ctx, &pb.SlaveWasPromotedRequest{})
_, err = c.SlaveWasPromoted(ctx, &tabletmanagerdatapb.SlaveWasPromotedRequest{})
return err
}
// SetMaster is part of the tmclient.TabletManagerClient interface
func (client *Client) SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *pbt.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
func (client *Client) SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return err
}
defer cc.Close()
_, err = c.SetMaster(ctx, &pb.SetMasterRequest{
_, err = c.SetMaster(ctx, &tabletmanagerdatapb.SetMasterRequest{
Parent: parent,
TimeCreatedNs: timeCreatedNS,
ForceStartSlave: forceStartSlave,
@ -611,42 +588,38 @@ func (client *Client) SlaveWasRestarted(ctx context.Context, tablet *topo.Tablet
return err
}
defer cc.Close()
_, err = c.SlaveWasRestarted(ctx, &pb.SlaveWasRestartedRequest{
_, err = c.SlaveWasRestarted(ctx, &tabletmanagerdatapb.SlaveWasRestartedRequest{
Parent: args.Parent,
})
return err
}
// StopReplicationAndGetStatus is part of the tmclient.TabletManagerClient interface
func (client *Client) StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error) {
func (client *Client) StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationStatus{}, err
return nil, err
}
defer cc.Close()
response, err := c.StopReplicationAndGetStatus(ctx, &pb.StopReplicationAndGetStatusRequest{})
response, err := c.StopReplicationAndGetStatus(ctx, &tabletmanagerdatapb.StopReplicationAndGetStatusRequest{})
if err != nil {
return myproto.ReplicationStatus{}, err
return nil, err
}
return myproto.ProtoToReplicationStatus(response.Status), nil
return response.Status, nil
}
// PromoteSlave is part of the tmclient.TabletManagerClient interface
func (client *Client) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error) {
func (client *Client) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (string, error) {
cc, c, err := client.dial(ctx, tablet)
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
defer cc.Close()
response, err := c.PromoteSlave(ctx, &pb.PromoteSlaveRequest{})
response, err := c.PromoteSlave(ctx, &tabletmanagerdatapb.PromoteSlaveRequest{})
if err != nil {
return myproto.ReplicationPosition{}, err
return "", err
}
position, err := myproto.DecodeReplicationPosition(response.Position)
if err != nil {
return myproto.ReplicationPosition{}, err
}
return position, err
return response.Position, nil
}
//
@ -661,7 +634,7 @@ func (client *Client) Backup(ctx context.Context, tablet *topo.TabletInfo, concu
}
logstream := make(chan *logutil.LoggerEvent, 10)
stream, err := c.Backup(ctx, &pb.BackupRequest{
stream, err := c.Backup(ctx, &tabletmanagerdatapb.BackupRequest{
Concurrency: int64(concurrency),
})
if err != nil {

Просмотреть файл

@ -21,8 +21,8 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
pb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pbs "github.com/youtube/vitess/go/vt/proto/tabletmanagerservice"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
tabletmanagerservicepb "github.com/youtube/vitess/go/vt/proto/tabletmanagerservice"
)
// server is the gRPC implementation of the RPC server
@ -31,27 +31,27 @@ type server struct {
agent tabletmanager.RPCAgent
}
func (s *server) Ping(ctx context.Context, request *pb.PingRequest) (*pb.PingResponse, error) {
func (s *server) Ping(ctx context.Context, request *tabletmanagerdatapb.PingRequest) (*tabletmanagerdatapb.PingResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.PingResponse{}
response := &tabletmanagerdatapb.PingResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionPing, request, response, func() error {
response.Payload = s.agent.Ping(ctx, request.Payload)
return nil
})
}
func (s *server) Sleep(ctx context.Context, request *pb.SleepRequest) (*pb.SleepResponse, error) {
func (s *server) Sleep(ctx context.Context, request *tabletmanagerdatapb.SleepRequest) (*tabletmanagerdatapb.SleepResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SleepResponse{}
response := &tabletmanagerdatapb.SleepResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSleep, request, response, true, func() error {
s.agent.Sleep(ctx, time.Duration(request.Duration))
return nil
})
}
func (s *server) ExecuteHook(ctx context.Context, request *pb.ExecuteHookRequest) (*pb.ExecuteHookResponse, error) {
func (s *server) ExecuteHook(ctx context.Context, request *tabletmanagerdatapb.ExecuteHookRequest) (*tabletmanagerdatapb.ExecuteHookResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ExecuteHookResponse{}
response := &tabletmanagerdatapb.ExecuteHookResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionExecuteHook, request, response, true, func() error {
hr := s.agent.ExecuteHook(ctx, &hook.Hook{
Name: request.Name,
@ -65,9 +65,9 @@ func (s *server) ExecuteHook(ctx context.Context, request *pb.ExecuteHookRequest
})
}
func (s *server) GetSchema(ctx context.Context, request *pb.GetSchemaRequest) (*pb.GetSchemaResponse, error) {
func (s *server) GetSchema(ctx context.Context, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.GetSchemaResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.GetSchemaResponse{}
response := &tabletmanagerdatapb.GetSchemaResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetSchema, request, response, func() error {
sd, err := s.agent.GetSchema(ctx, request.Tables, request.ExcludeTables, request.IncludeViews)
if err == nil {
@ -77,9 +77,9 @@ func (s *server) GetSchema(ctx context.Context, request *pb.GetSchemaRequest) (*
})
}
func (s *server) GetPermissions(ctx context.Context, request *pb.GetPermissionsRequest) (*pb.GetPermissionsResponse, error) {
func (s *server) GetPermissions(ctx context.Context, request *tabletmanagerdatapb.GetPermissionsRequest) (*tabletmanagerdatapb.GetPermissionsResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.GetPermissionsResponse{}
response := &tabletmanagerdatapb.GetPermissionsResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetPermissions, request, response, func() error {
p, err := s.agent.GetPermissions(ctx)
if err == nil {
@ -93,60 +93,60 @@ func (s *server) GetPermissions(ctx context.Context, request *pb.GetPermissionsR
// Various read-write methods
//
func (s *server) SetReadOnly(ctx context.Context, request *pb.SetReadOnlyRequest) (*pb.SetReadOnlyResponse, error) {
func (s *server) SetReadOnly(ctx context.Context, request *tabletmanagerdatapb.SetReadOnlyRequest) (*tabletmanagerdatapb.SetReadOnlyResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SetReadOnlyResponse{}
response := &tabletmanagerdatapb.SetReadOnlyResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetReadOnly, request, response, true, func() error {
return s.agent.SetReadOnly(ctx, true)
})
}
func (s *server) SetReadWrite(ctx context.Context, request *pb.SetReadWriteRequest) (*pb.SetReadWriteResponse, error) {
func (s *server) SetReadWrite(ctx context.Context, request *tabletmanagerdatapb.SetReadWriteRequest) (*tabletmanagerdatapb.SetReadWriteResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SetReadWriteResponse{}
response := &tabletmanagerdatapb.SetReadWriteResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetReadWrite, request, response, true, func() error {
return s.agent.SetReadOnly(ctx, false)
})
}
func (s *server) ChangeType(ctx context.Context, request *pb.ChangeTypeRequest) (*pb.ChangeTypeResponse, error) {
func (s *server) ChangeType(ctx context.Context, request *tabletmanagerdatapb.ChangeTypeRequest) (*tabletmanagerdatapb.ChangeTypeResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ChangeTypeResponse{}
response := &tabletmanagerdatapb.ChangeTypeResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionChangeType, request, response, true, func() error {
return s.agent.ChangeType(ctx, request.TabletType)
})
}
func (s *server) RefreshState(ctx context.Context, request *pb.RefreshStateRequest) (*pb.RefreshStateResponse, error) {
func (s *server) RefreshState(ctx context.Context, request *tabletmanagerdatapb.RefreshStateRequest) (*tabletmanagerdatapb.RefreshStateResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.RefreshStateResponse{}
response := &tabletmanagerdatapb.RefreshStateResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, request, response, true, func() error {
s.agent.RefreshState(ctx)
return nil
})
}
func (s *server) RunHealthCheck(ctx context.Context, request *pb.RunHealthCheckRequest) (*pb.RunHealthCheckResponse, error) {
func (s *server) RunHealthCheck(ctx context.Context, request *tabletmanagerdatapb.RunHealthCheckRequest) (*tabletmanagerdatapb.RunHealthCheckResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.RunHealthCheckResponse{}
response := &tabletmanagerdatapb.RunHealthCheckResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionRunHealthCheck, request, response, func() error {
s.agent.RunHealthCheck(ctx, request.TabletType)
return nil
})
}
func (s *server) ReloadSchema(ctx context.Context, request *pb.ReloadSchemaRequest) (*pb.ReloadSchemaResponse, error) {
func (s *server) ReloadSchema(ctx context.Context, request *tabletmanagerdatapb.ReloadSchemaRequest) (*tabletmanagerdatapb.ReloadSchemaResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ReloadSchemaResponse{}
response := &tabletmanagerdatapb.ReloadSchemaResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionReloadSchema, request, response, true, func() error {
s.agent.ReloadSchema(ctx)
return nil
})
}
func (s *server) PreflightSchema(ctx context.Context, request *pb.PreflightSchemaRequest) (*pb.PreflightSchemaResponse, error) {
func (s *server) PreflightSchema(ctx context.Context, request *tabletmanagerdatapb.PreflightSchemaRequest) (*tabletmanagerdatapb.PreflightSchemaResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.PreflightSchemaResponse{}
response := &tabletmanagerdatapb.PreflightSchemaResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPreflightSchema, request, response, true, func() error {
scr, err := s.agent.PreflightSchema(ctx, request.Change)
if err == nil {
@ -157,9 +157,9 @@ func (s *server) PreflightSchema(ctx context.Context, request *pb.PreflightSchem
})
}
func (s *server) ApplySchema(ctx context.Context, request *pb.ApplySchemaRequest) (*pb.ApplySchemaResponse, error) {
func (s *server) ApplySchema(ctx context.Context, request *tabletmanagerdatapb.ApplySchemaRequest) (*tabletmanagerdatapb.ApplySchemaResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ApplySchemaResponse{}
response := &tabletmanagerdatapb.ApplySchemaResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionApplySchema, request, response, true, func() error {
scr, err := s.agent.ApplySchema(ctx, &myproto.SchemaChange{
Sql: request.Sql,
@ -176,9 +176,9 @@ func (s *server) ApplySchema(ctx context.Context, request *pb.ApplySchemaRequest
})
}
func (s *server) ExecuteFetchAsDba(ctx context.Context, request *pb.ExecuteFetchAsDbaRequest) (*pb.ExecuteFetchAsDbaResponse, error) {
func (s *server) ExecuteFetchAsDba(ctx context.Context, request *tabletmanagerdatapb.ExecuteFetchAsDbaRequest) (*tabletmanagerdatapb.ExecuteFetchAsDbaResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ExecuteFetchAsDbaResponse{}
response := &tabletmanagerdatapb.ExecuteFetchAsDbaResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionExecuteFetchAsDba, request, response, func() error {
qr, err := s.agent.ExecuteFetchAsDba(ctx, request.Query, request.DbName, int(request.MaxRows), request.WantFields, request.DisableBinlogs, request.ReloadSchema)
if err != nil {
@ -189,9 +189,9 @@ func (s *server) ExecuteFetchAsDba(ctx context.Context, request *pb.ExecuteFetch
})
}
func (s *server) ExecuteFetchAsApp(ctx context.Context, request *pb.ExecuteFetchAsAppRequest) (*pb.ExecuteFetchAsAppResponse, error) {
func (s *server) ExecuteFetchAsApp(ctx context.Context, request *tabletmanagerdatapb.ExecuteFetchAsAppRequest) (*tabletmanagerdatapb.ExecuteFetchAsAppResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ExecuteFetchAsAppResponse{}
response := &tabletmanagerdatapb.ExecuteFetchAsAppResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionExecuteFetchAsApp, request, response, func() error {
qr, err := s.agent.ExecuteFetchAsApp(ctx, request.Query, int(request.MaxRows), request.WantFields)
if err != nil {
@ -206,77 +206,73 @@ func (s *server) ExecuteFetchAsApp(ctx context.Context, request *pb.ExecuteFetch
// Replication related methods
//
func (s *server) SlaveStatus(ctx context.Context, request *pb.SlaveStatusRequest) (*pb.SlaveStatusResponse, error) {
func (s *server) SlaveStatus(ctx context.Context, request *tabletmanagerdatapb.SlaveStatusRequest) (*tabletmanagerdatapb.SlaveStatusResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SlaveStatusResponse{}
response := &tabletmanagerdatapb.SlaveStatusResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionSlaveStatus, request, response, func() error {
status, err := s.agent.SlaveStatus(ctx)
if err == nil {
response.Status = myproto.ReplicationStatusToProto(status)
response.Status = status
}
return err
})
}
func (s *server) MasterPosition(ctx context.Context, request *pb.MasterPositionRequest) (*pb.MasterPositionResponse, error) {
func (s *server) MasterPosition(ctx context.Context, request *tabletmanagerdatapb.MasterPositionRequest) (*tabletmanagerdatapb.MasterPositionResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.MasterPositionResponse{}
response := &tabletmanagerdatapb.MasterPositionResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionMasterPosition, request, response, func() error {
position, err := s.agent.MasterPosition(ctx)
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) StopSlave(ctx context.Context, request *pb.StopSlaveRequest) (*pb.StopSlaveResponse, error) {
func (s *server) StopSlave(ctx context.Context, request *tabletmanagerdatapb.StopSlaveRequest) (*tabletmanagerdatapb.StopSlaveResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StopSlaveResponse{}
response := &tabletmanagerdatapb.StopSlaveResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopSlave, request, response, true, func() error {
return s.agent.StopSlave(ctx)
})
}
func (s *server) StopSlaveMinimum(ctx context.Context, request *pb.StopSlaveMinimumRequest) (*pb.StopSlaveMinimumResponse, error) {
func (s *server) StopSlaveMinimum(ctx context.Context, request *tabletmanagerdatapb.StopSlaveMinimumRequest) (*tabletmanagerdatapb.StopSlaveMinimumResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StopSlaveMinimumResponse{}
response := &tabletmanagerdatapb.StopSlaveMinimumResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopSlaveMinimum, request, response, true, func() error {
position, err := myproto.DecodeReplicationPosition(request.Position)
if err != nil {
return err
}
position, err = s.agent.StopSlaveMinimum(ctx, position, time.Duration(request.WaitTimeout))
position, err := s.agent.StopSlaveMinimum(ctx, request.Position, time.Duration(request.WaitTimeout))
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) StartSlave(ctx context.Context, request *pb.StartSlaveRequest) (*pb.StartSlaveResponse, error) {
func (s *server) StartSlave(ctx context.Context, request *tabletmanagerdatapb.StartSlaveRequest) (*tabletmanagerdatapb.StartSlaveResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StartSlaveResponse{}
response := &tabletmanagerdatapb.StartSlaveResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStartSlave, request, response, true, func() error {
return s.agent.StartSlave(ctx)
})
}
func (s *server) TabletExternallyReparented(ctx context.Context, request *pb.TabletExternallyReparentedRequest) (*pb.TabletExternallyReparentedResponse, error) {
func (s *server) TabletExternallyReparented(ctx context.Context, request *tabletmanagerdatapb.TabletExternallyReparentedRequest) (*tabletmanagerdatapb.TabletExternallyReparentedResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.TabletExternallyReparentedResponse{}
response := &tabletmanagerdatapb.TabletExternallyReparentedResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionExternallyReparented, request, response, false, func() error {
return s.agent.TabletExternallyReparented(ctx, request.ExternalId)
})
}
func (s *server) TabletExternallyElected(ctx context.Context, request *pb.TabletExternallyElectedRequest) (*pb.TabletExternallyElectedResponse, error) {
return &pb.TabletExternallyElectedResponse{}, nil
func (s *server) TabletExternallyElected(ctx context.Context, request *tabletmanagerdatapb.TabletExternallyElectedRequest) (*tabletmanagerdatapb.TabletExternallyElectedResponse, error) {
return &tabletmanagerdatapb.TabletExternallyElectedResponse{}, nil
}
func (s *server) GetSlaves(ctx context.Context, request *pb.GetSlavesRequest) (*pb.GetSlavesResponse, error) {
func (s *server) GetSlaves(ctx context.Context, request *tabletmanagerdatapb.GetSlavesRequest) (*tabletmanagerdatapb.GetSlavesResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.GetSlavesResponse{}
response := &tabletmanagerdatapb.GetSlavesResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetSlaves, request, response, func() error {
addrs, err := s.agent.GetSlaves(ctx)
if err == nil {
@ -286,17 +282,17 @@ func (s *server) GetSlaves(ctx context.Context, request *pb.GetSlavesRequest) (*
})
}
func (s *server) WaitBlpPosition(ctx context.Context, request *pb.WaitBlpPositionRequest) (*pb.WaitBlpPositionResponse, error) {
func (s *server) WaitBlpPosition(ctx context.Context, request *tabletmanagerdatapb.WaitBlpPositionRequest) (*tabletmanagerdatapb.WaitBlpPositionResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.WaitBlpPositionResponse{}
response := &tabletmanagerdatapb.WaitBlpPositionResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionWaitBLPPosition, request, response, true, func() error {
return s.agent.WaitBlpPosition(ctx, request.BlpPosition, time.Duration(request.WaitTimeout))
})
}
func (s *server) StopBlp(ctx context.Context, request *pb.StopBlpRequest) (*pb.StopBlpResponse, error) {
func (s *server) StopBlp(ctx context.Context, request *tabletmanagerdatapb.StopBlpRequest) (*tabletmanagerdatapb.StopBlpResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StopBlpResponse{}
response := &tabletmanagerdatapb.StopBlpResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopBLP, request, response, true, func() error {
positions, err := s.agent.StopBlp(ctx)
if err == nil {
@ -306,21 +302,21 @@ func (s *server) StopBlp(ctx context.Context, request *pb.StopBlpRequest) (*pb.S
})
}
func (s *server) StartBlp(ctx context.Context, request *pb.StartBlpRequest) (*pb.StartBlpResponse, error) {
func (s *server) StartBlp(ctx context.Context, request *tabletmanagerdatapb.StartBlpRequest) (*tabletmanagerdatapb.StartBlpResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StartBlpResponse{}
response := &tabletmanagerdatapb.StartBlpResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStartBLP, request, response, true, func() error {
return s.agent.StartBlp(ctx)
})
}
func (s *server) RunBlpUntil(ctx context.Context, request *pb.RunBlpUntilRequest) (*pb.RunBlpUntilResponse, error) {
func (s *server) RunBlpUntil(ctx context.Context, request *tabletmanagerdatapb.RunBlpUntilRequest) (*tabletmanagerdatapb.RunBlpUntilResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.RunBlpUntilResponse{}
response := &tabletmanagerdatapb.RunBlpUntilResponse{}
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionRunBLPUntil, request, response, true, func() error {
position, err := s.agent.RunBlpUntil(ctx, request.BlpPositions, time.Duration(request.WaitTimeout))
if err == nil {
response.Position = myproto.EncodeReplicationPosition(*position)
response.Position = position
}
return err
})
@ -330,97 +326,85 @@ func (s *server) RunBlpUntil(ctx context.Context, request *pb.RunBlpUntilRequest
// Reparenting related functions
//
func (s *server) ResetReplication(ctx context.Context, request *pb.ResetReplicationRequest) (*pb.ResetReplicationResponse, error) {
func (s *server) ResetReplication(ctx context.Context, request *tabletmanagerdatapb.ResetReplicationRequest) (*tabletmanagerdatapb.ResetReplicationResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.ResetReplicationResponse{}
response := &tabletmanagerdatapb.ResetReplicationResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionResetReplication, request, response, true, func() error {
return s.agent.ResetReplication(ctx)
})
}
func (s *server) InitMaster(ctx context.Context, request *pb.InitMasterRequest) (*pb.InitMasterResponse, error) {
func (s *server) InitMaster(ctx context.Context, request *tabletmanagerdatapb.InitMasterRequest) (*tabletmanagerdatapb.InitMasterResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.InitMasterResponse{}
response := &tabletmanagerdatapb.InitMasterResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionInitMaster, request, response, true, func() error {
position, err := s.agent.InitMaster(ctx)
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) PopulateReparentJournal(ctx context.Context, request *pb.PopulateReparentJournalRequest) (*pb.PopulateReparentJournalResponse, error) {
func (s *server) PopulateReparentJournal(ctx context.Context, request *tabletmanagerdatapb.PopulateReparentJournalRequest) (*tabletmanagerdatapb.PopulateReparentJournalResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.PopulateReparentJournalResponse{}
response := &tabletmanagerdatapb.PopulateReparentJournalResponse{}
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionPopulateReparentJournal, request, response, func() error {
position, err := myproto.DecodeReplicationPosition(request.ReplicationPosition)
if err != nil {
return err
}
return s.agent.PopulateReparentJournal(ctx, request.TimeCreatedNs, request.ActionName, request.MasterAlias, position)
return s.agent.PopulateReparentJournal(ctx, request.TimeCreatedNs, request.ActionName, request.MasterAlias, request.ReplicationPosition)
})
}
func (s *server) InitSlave(ctx context.Context, request *pb.InitSlaveRequest) (*pb.InitSlaveResponse, error) {
func (s *server) InitSlave(ctx context.Context, request *tabletmanagerdatapb.InitSlaveRequest) (*tabletmanagerdatapb.InitSlaveResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.InitSlaveResponse{}
response := &tabletmanagerdatapb.InitSlaveResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionInitSlave, request, response, true, func() error {
position, err := myproto.DecodeReplicationPosition(request.ReplicationPosition)
if err != nil {
return err
}
return s.agent.InitSlave(ctx, request.Parent, position, request.TimeCreatedNs)
return s.agent.InitSlave(ctx, request.Parent, request.ReplicationPosition, request.TimeCreatedNs)
})
}
func (s *server) DemoteMaster(ctx context.Context, request *pb.DemoteMasterRequest) (*pb.DemoteMasterResponse, error) {
func (s *server) DemoteMaster(ctx context.Context, request *tabletmanagerdatapb.DemoteMasterRequest) (*tabletmanagerdatapb.DemoteMasterResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.DemoteMasterResponse{}
response := &tabletmanagerdatapb.DemoteMasterResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionDemoteMaster, request, response, true, func() error {
position, err := s.agent.DemoteMaster(ctx)
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) PromoteSlaveWhenCaughtUp(ctx context.Context, request *pb.PromoteSlaveWhenCaughtUpRequest) (*pb.PromoteSlaveWhenCaughtUpResponse, error) {
func (s *server) PromoteSlaveWhenCaughtUp(ctx context.Context, request *tabletmanagerdatapb.PromoteSlaveWhenCaughtUpRequest) (*tabletmanagerdatapb.PromoteSlaveWhenCaughtUpResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.PromoteSlaveWhenCaughtUpResponse{}
response := &tabletmanagerdatapb.PromoteSlaveWhenCaughtUpResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPromoteSlaveWhenCaughtUp, request, response, true, func() error {
position, err := myproto.DecodeReplicationPosition(request.Position)
if err != nil {
return err
}
position, err = s.agent.PromoteSlaveWhenCaughtUp(ctx, position)
position, err := s.agent.PromoteSlaveWhenCaughtUp(ctx, request.Position)
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) SlaveWasPromoted(ctx context.Context, request *pb.SlaveWasPromotedRequest) (*pb.SlaveWasPromotedResponse, error) {
func (s *server) SlaveWasPromoted(ctx context.Context, request *tabletmanagerdatapb.SlaveWasPromotedRequest) (*tabletmanagerdatapb.SlaveWasPromotedResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SlaveWasPromotedResponse{}
response := &tabletmanagerdatapb.SlaveWasPromotedResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSlaveWasPromoted, request, response, true, func() error {
return s.agent.SlaveWasPromoted(ctx)
})
}
func (s *server) SetMaster(ctx context.Context, request *pb.SetMasterRequest) (*pb.SetMasterResponse, error) {
func (s *server) SetMaster(ctx context.Context, request *tabletmanagerdatapb.SetMasterRequest) (*tabletmanagerdatapb.SetMasterResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SetMasterResponse{}
response := &tabletmanagerdatapb.SetMasterResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetMaster, request, response, true, func() error {
return s.agent.SetMaster(ctx, request.Parent, request.TimeCreatedNs, request.ForceStartSlave)
})
}
func (s *server) SlaveWasRestarted(ctx context.Context, request *pb.SlaveWasRestartedRequest) (*pb.SlaveWasRestartedResponse, error) {
func (s *server) SlaveWasRestarted(ctx context.Context, request *tabletmanagerdatapb.SlaveWasRestartedRequest) (*tabletmanagerdatapb.SlaveWasRestartedResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.SlaveWasRestartedResponse{}
response := &tabletmanagerdatapb.SlaveWasRestartedResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSlaveWasRestarted, request, response, true, func() error {
return s.agent.SlaveWasRestarted(ctx, &actionnode.SlaveWasRestartedArgs{
Parent: request.Parent,
@ -428,31 +412,31 @@ func (s *server) SlaveWasRestarted(ctx context.Context, request *pb.SlaveWasRest
})
}
func (s *server) StopReplicationAndGetStatus(ctx context.Context, request *pb.StopReplicationAndGetStatusRequest) (*pb.StopReplicationAndGetStatusResponse, error) {
func (s *server) StopReplicationAndGetStatus(ctx context.Context, request *tabletmanagerdatapb.StopReplicationAndGetStatusRequest) (*tabletmanagerdatapb.StopReplicationAndGetStatusResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.StopReplicationAndGetStatusResponse{}
response := &tabletmanagerdatapb.StopReplicationAndGetStatusResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionStopReplicationAndGetStatus, request, response, true, func() error {
status, err := s.agent.StopReplicationAndGetStatus(ctx)
if err == nil {
response.Status = myproto.ReplicationStatusToProto(status)
response.Status = status
}
return err
})
}
func (s *server) PromoteSlave(ctx context.Context, request *pb.PromoteSlaveRequest) (*pb.PromoteSlaveResponse, error) {
func (s *server) PromoteSlave(ctx context.Context, request *tabletmanagerdatapb.PromoteSlaveRequest) (*tabletmanagerdatapb.PromoteSlaveResponse, error) {
ctx = callinfo.GRPCCallInfo(ctx)
response := &pb.PromoteSlaveResponse{}
response := &tabletmanagerdatapb.PromoteSlaveResponse{}
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPromoteSlave, request, response, true, func() error {
position, err := s.agent.PromoteSlave(ctx)
if err == nil {
response.Position = myproto.EncodeReplicationPosition(position)
response.Position = position
}
return err
})
}
func (s *server) Backup(request *pb.BackupRequest, stream pbs.TabletManager_BackupServer) error {
func (s *server) Backup(request *tabletmanagerdatapb.BackupRequest, stream tabletmanagerservicepb.TabletManager_BackupServer) error {
ctx := callinfo.GRPCCallInfo(stream.Context())
return s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionBackup, request, nil, true, func() error {
// create a logger, send the result back to the caller
@ -466,7 +450,7 @@ func (s *server) Backup(request *pb.BackupRequest, stream pbs.TabletManager_Back
// command, even if the channel to the client
// has been broken. We'll just keep trying
// to send.
stream.Send(&pb.BackupResponse{
stream.Send(&tabletmanagerdatapb.BackupResponse{
Event: logutil.LoggerEventToProto(&e),
})
@ -486,12 +470,12 @@ func (s *server) Backup(request *pb.BackupRequest, stream pbs.TabletManager_Back
func init() {
tabletmanager.RegisterQueryServices = append(tabletmanager.RegisterQueryServices, func(agent *tabletmanager.ActionAgent) {
if servenv.GRPCCheckServiceMap("tabletmanager") {
pbs.RegisterTabletManagerServer(servenv.GRPCServer, &server{agent})
tabletmanagerservicepb.RegisterTabletManagerServer(servenv.GRPCServer, &server{agent})
}
})
}
// RegisterForTest will register the RPC, to be used by test instances only
func RegisterForTest(s *grpc.Server, agent *tabletmanager.ActionAgent) {
pbs.RegisterTabletManagerServer(s, &server{agent})
tabletmanagerservicepb.RegisterTabletManagerServer(s, &server{agent})
}

Просмотреть файл

@ -17,8 +17,9 @@ import (
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
pbt "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// TabletManagerProtocol is the implementation to use for tablet
@ -38,10 +39,10 @@ type TabletManagerClient interface {
Ping(ctx context.Context, tablet *topo.TabletInfo) error
// GetSchema asks the remote tablet for its database schema
GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*pbt.SchemaDefinition, error)
GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error)
// GetPermissions asks the remote tablet for its permissions list
GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*pbt.Permissions, error)
GetPermissions(ctx context.Context, tablet *topo.TabletInfo) (*tabletmanagerdatapb.Permissions, error)
//
// Various read-write methods
@ -54,7 +55,7 @@ type TabletManagerClient interface {
SetReadWrite(ctx context.Context, tablet *topo.TabletInfo) error
// ChangeType asks the remote tablet to change its type
ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType pb.TabletType) error
ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topodatapb.TabletType) error
// Sleep will sleep for a duration (used for tests)
Sleep(ctx context.Context, tablet *topo.TabletInfo, duration time.Duration) error
@ -66,7 +67,7 @@ type TabletManagerClient interface {
RefreshState(ctx context.Context, tablet *topo.TabletInfo) error
// RunHealthCheck asks the remote tablet to run a health check cycle
RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType pb.TabletType) error
RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topodatapb.TabletType) error
// ReloadSchema asks the remote tablet to reload its schema
ReloadSchema(ctx context.Context, tablet *topo.TabletInfo) error
@ -88,17 +89,17 @@ type TabletManagerClient interface {
//
// SlaveStatus returns the tablet's mysql slave status.
SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error)
SlaveStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error)
// MasterPosition returns the tablet's master position
MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error)
MasterPosition(ctx context.Context, tablet *topo.TabletInfo) (string, error)
// StopSlave stops the mysql replication
StopSlave(ctx context.Context, tablet *topo.TabletInfo) error
// StopSlaveMinimum stops the mysql replication after it reaches
// the provided minimum point
StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, stopPos myproto.ReplicationPosition, waitTime time.Duration) (myproto.ReplicationPosition, error)
StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, stopPos string, waitTime time.Duration) (string, error)
// StartSlave starts the mysql replication
StartSlave(ctx context.Context, tablet *topo.TabletInfo) error
@ -116,18 +117,18 @@ type TabletManagerClient interface {
// WaitBlpPosition asks the tablet to wait until it reaches that
// position in replication
WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *pbt.BlpPosition, waitTime time.Duration) error
WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition *tabletmanagerdatapb.BlpPosition, waitTime time.Duration) error
// StopBlp asks the tablet to stop all its binlog players,
// and returns the current position for all of them
StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*pbt.BlpPosition, error)
StopBlp(ctx context.Context, tablet *topo.TabletInfo) ([]*tabletmanagerdatapb.BlpPosition, error)
// StartBlp asks the tablet to restart its binlog players
StartBlp(ctx context.Context, tablet *topo.TabletInfo) error
// RunBlpUntil asks the tablet to restart its binlog players until
// it reaches the given positions, if not there yet.
RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*pbt.BlpPosition, waitTime time.Duration) (myproto.ReplicationPosition, error)
RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions []*tabletmanagerdatapb.BlpPosition, waitTime time.Duration) (string, error)
//
// Reparenting related functions
@ -141,23 +142,23 @@ type TabletManagerClient interface {
// InitMaster tells a tablet to make itself the new master,
// and return the replication position the slaves should use to
// reparent to it.
InitMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error)
InitMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error)
// PopulateReparentJournal asks the master to insert a row in
// its reparent_journal table.
PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *pb.TabletAlias, pos myproto.ReplicationPosition) error
PopulateReparentJournal(ctx context.Context, tablet *topo.TabletInfo, timeCreatedNS int64, actionName string, masterAlias *topodatapb.TabletAlias, pos string) error
// InitSlave tells a tablet to make itself a slave to the
// passed in master tablet alias, and wait for the row in the
// reparent_journal table.
InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *pb.TabletAlias, replicationPosition myproto.ReplicationPosition, timeCreatedNS int64) error
InitSlave(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, replicationPosition string, timeCreatedNS int64) error
// DemoteMaster tells the soon-to-be-former master it's gonna change,
// and it should go read-only and return its current position.
DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error)
DemoteMaster(ctx context.Context, tablet *topo.TabletInfo) (string, error)
// PromoteSlaveWhenCaughtUp transforms the tablet from a slave to a master.
PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, pos myproto.ReplicationPosition) (myproto.ReplicationPosition, error)
PromoteSlaveWhenCaughtUp(ctx context.Context, tablet *topo.TabletInfo, pos string) (string, error)
// SlaveWasPromoted tells the remote tablet it is now the master
SlaveWasPromoted(ctx context.Context, tablet *topo.TabletInfo) error
@ -165,17 +166,17 @@ type TabletManagerClient interface {
// SetMaster tells a tablet to make itself a slave to the
// passed in master tablet alias, and wait for the row in the
// reparent_journal table (if timeCreatedNS is non-zero).
SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *pb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
SetMaster(ctx context.Context, tablet *topo.TabletInfo, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
// SlaveWasRestarted tells the remote tablet its master has changed
SlaveWasRestarted(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs) error
// StopReplicationAndGetStatus stops replication and returns the
// current position.
StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationStatus, error)
StopReplicationAndGetStatus(ctx context.Context, tablet *topo.TabletInfo) (*replicationdatapb.Status, error)
// PromoteSlave makes the tablet the new master
PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (myproto.ReplicationPosition, error)
PromoteSlave(ctx context.Context, tablet *topo.TabletInfo) (string, error)
//
// Backup / restore related methods

Просмотреть файл

@ -94,13 +94,15 @@ import (
"github.com/youtube/vitess/go/vt/key"
"github.com/youtube/vitess/go/vt/logutil"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topotools"
"github.com/youtube/vitess/go/vt/wrangler"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
var (
@ -438,7 +440,7 @@ func dumpAllTablets(ctx context.Context, wr *wrangler.Wrangler, zkVtPath string)
return nil
}
func dumpTablets(ctx context.Context, wr *wrangler.Wrangler, tabletAliases []*pb.TabletAlias) error {
func dumpTablets(ctx context.Context, wr *wrangler.Wrangler, tabletAliases []*topodatapb.TabletAlias) error {
tabletMap, err := wr.TopoServer().GetTabletMap(ctx, tabletAliases)
if err != nil {
return err
@ -532,8 +534,8 @@ func shardParamsToKeyspaceShards(ctx context.Context, wr *wrangler.Wrangler, par
// tabletParamsToTabletAliases takes multiple params and converts them
// to tablet aliases.
func tabletParamsToTabletAliases(params []string) ([]*pb.TabletAlias, error) {
result := make([]*pb.TabletAlias, len(params))
func tabletParamsToTabletAliases(params []string) ([]*topodatapb.TabletAlias, error) {
result := make([]*topodatapb.TabletAlias, len(params))
var err error
for i, param := range params {
result[i], err = topoproto.ParseTabletAlias(param)
@ -546,26 +548,26 @@ func tabletParamsToTabletAliases(params []string) ([]*pb.TabletAlias, error) {
// parseTabletType parses the string tablet type and verifies
// it is an accepted one
func parseTabletType(param string, types []pb.TabletType) (pb.TabletType, error) {
func parseTabletType(param string, types []topodatapb.TabletType) (topodatapb.TabletType, error) {
tabletType, err := topoproto.ParseTabletType(param)
if err != nil {
return pb.TabletType_UNKNOWN, fmt.Errorf("invalid tablet type %v: %v", param, err)
return topodatapb.TabletType_UNKNOWN, fmt.Errorf("invalid tablet type %v: %v", param, err)
}
if !topoproto.IsTypeInList(pb.TabletType(tabletType), types) {
return pb.TabletType_UNKNOWN, fmt.Errorf("Type %v is not one of: %v", tabletType, strings.Join(topoproto.MakeStringTypeList(types), " "))
if !topoproto.IsTypeInList(topodatapb.TabletType(tabletType), types) {
return topodatapb.TabletType_UNKNOWN, fmt.Errorf("Type %v is not one of: %v", tabletType, strings.Join(topoproto.MakeStringTypeList(types), " "))
}
return tabletType, nil
}
// parseServingTabletType3 parses the tablet type into the enum,
// and makes sure the enum is of serving type (MASTER, REPLICA, RDONLY/BATCH)
func parseServingTabletType3(param string) (pb.TabletType, error) {
func parseServingTabletType3(param string) (topodatapb.TabletType, error) {
servedType, err := topoproto.ParseTabletType(param)
if err != nil {
return pb.TabletType_UNKNOWN, err
return topodatapb.TabletType_UNKNOWN, err
}
if !topo.IsInServingGraph(servedType) {
return pb.TabletType_UNKNOWN, fmt.Errorf("served_type has to be in the serving graph, not %v", param)
return topodatapb.TabletType_UNKNOWN, fmt.Errorf("served_type has to be in the serving graph, not %v", param)
}
return servedType, nil
}
@ -602,7 +604,7 @@ func commandInitTablet(ctx context.Context, wr *wrangler.Wrangler, subFlags *fla
}
// create tablet record
tablet := &pb.Tablet{
tablet := &topodatapb.Tablet{
Alias: tabletAlias,
Hostname: *hostname,
PortMap: make(map[string]int32),
@ -665,7 +667,7 @@ func commandUpdateTabletAddrs(ctx context.Context, wr *wrangler.Wrangler, subFla
if err != nil {
return err
}
return wr.TopoServer().UpdateTabletFields(ctx, tabletAlias, func(tablet *pb.Tablet) error {
return wr.TopoServer().UpdateTabletFields(ctx, tabletAlias, func(tablet *topodatapb.Tablet) error {
if *hostname != "" {
tablet.Hostname = *hostname
}
@ -869,7 +871,7 @@ func commandRunHealthCheck(ctx context.Context, wr *wrangler.Wrangler, subFlags
if err != nil {
return err
}
servedType, err := parseTabletType(subFlags.Arg(1), []pb.TabletType{pb.TabletType_REPLICA, pb.TabletType_RDONLY})
servedType, err := parseTabletType(subFlags.Arg(1), []topodatapb.TabletType{topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY})
if err != nil {
return err
}
@ -989,7 +991,7 @@ func commandCreateShard(ctx context.Context, wr *wrangler.Wrangler, subFlags *fl
return err
}
if *parent {
if err := wr.TopoServer().CreateKeyspace(ctx, keyspace, &pb.Keyspace{}); err != nil && err != topo.ErrNodeExists {
if err := wr.TopoServer().CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{}); err != nil && err != topo.ErrNodeExists {
return err
}
}
@ -1100,7 +1102,7 @@ func commandShardReplicationPositions(ctx context.Context, wr *wrangler.Wrangler
lines := make([]string, 0, 24)
for _, rt := range sortReplicatingTablets(tablets, stats) {
status := rt.ReplicationStatus
status := rt.Status
ti := rt.TabletInfo
if status == nil {
lines = append(lines, fmtTabletAwkable(ti)+" <err> <err> <err>")
@ -1229,7 +1231,7 @@ func commandSourceShardAdd(ctx context.Context, wr *wrangler.Wrangler, subFlags
if *tablesStr != "" {
tables = strings.Split(*tablesStr, ",")
}
var kr *pb.KeyRange
var kr *topodatapb.KeyRange
if *keyRange != "" {
if _, kr, err = topo.ValidateShardName(*keyRange); err != nil {
return err
@ -1331,12 +1333,12 @@ func commandWaitForFilteredReplication(ctx context.Context, wr *wrangler.Wrangle
// Always run an explicit healthcheck first to make sure we don't see any outdated values.
// This is especially true for tests and automation where there is no pause of multiple seconds
// between commands and the periodic healthcheck did not run again yet.
if err := wr.TabletManagerClient().RunHealthCheck(ctx, tabletInfo, pb.TabletType_REPLICA); err != nil {
if err := wr.TabletManagerClient().RunHealthCheck(ctx, tabletInfo, topodatapb.TabletType_REPLICA); err != nil {
return fmt.Errorf("failed to run explicit healthcheck on tablet: %v err: %v", tabletInfo, err)
}
// pass in a non-UNKNOWN tablet type to not use sessionId
conn, err := tabletconn.GetDialer()(ctx, ep, "", "", pb.TabletType_MASTER, 30*time.Second)
conn, err := tabletconn.GetDialer()(ctx, ep, "", "", topodatapb.TabletType_MASTER, 30*time.Second)
if err != nil {
return fmt.Errorf("cannot connect to tablet %v: %v", alias, err)
}
@ -1442,7 +1444,7 @@ func commandCreateKeyspace(ctx context.Context, wr *wrangler.Wrangler, subFlags
if err != nil {
return err
}
ki := &pb.Keyspace{
ki := &topodatapb.Keyspace{
ShardingColumnName: *shardingColumnName,
ShardingColumnType: kit,
SplitShardCount: int32(*splitShardCount),
@ -1453,7 +1455,7 @@ func commandCreateKeyspace(ctx context.Context, wr *wrangler.Wrangler, subFlags
if err != nil {
return err
}
ki.ServedFroms = append(ki.ServedFroms, &pb.Keyspace_ServedFrom{
ki.ServedFroms = append(ki.ServedFroms, &topodatapb.Keyspace_ServedFrom{
TabletType: tt,
Keyspace: value,
})
@ -1523,7 +1525,7 @@ func commandSetKeyspaceShardingInfo(ctx context.Context, wr *wrangler.Wrangler,
if subFlags.NArg() >= 2 {
columnName = subFlags.Arg(1)
}
kit := pb.KeyspaceIdType_UNSET
kit := topodatapb.KeyspaceIdType_UNSET
if subFlags.NArg() >= 3 {
var err error
kit, err = key.ParseKeyspaceIDType(subFlags.Arg(2))
@ -1546,7 +1548,7 @@ func commandSetKeyspaceServedFrom(ctx context.Context, wr *wrangler.Wrangler, su
return fmt.Errorf("The <keyspace name> and <tablet type> arguments are required for the SetKeyspaceServedFrom command.")
}
keyspace := subFlags.Arg(0)
servedType, err := parseTabletType(subFlags.Arg(1), []pb.TabletType{pb.TabletType_MASTER, pb.TabletType_REPLICA, pb.TabletType_RDONLY})
servedType, err := parseTabletType(subFlags.Arg(1), []topodatapb.TabletType{topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY})
if err != nil {
return err
}
@ -1618,7 +1620,7 @@ func commandMigrateServedTypes(ctx context.Context, wr *wrangler.Wrangler, subFl
if err != nil {
return err
}
if servedType == pb.TabletType_MASTER && *skipReFreshState {
if servedType == topodatapb.TabletType_MASTER && *skipReFreshState {
return fmt.Errorf("The skip-refresh-state flag can only be specified for non-master migrations.")
}
var cells []string
@ -1643,7 +1645,7 @@ func commandMigrateServedFrom(ctx context.Context, wr *wrangler.Wrangler, subFla
if err != nil {
return err
}
servedType, err := parseTabletType(subFlags.Arg(1), []pb.TabletType{pb.TabletType_MASTER, pb.TabletType_REPLICA, pb.TabletType_RDONLY})
servedType, err := parseTabletType(subFlags.Arg(1), []topodatapb.TabletType{topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY})
if err != nil {
return err
}
@ -1721,7 +1723,7 @@ func commandListTablets(ctx context.Context, wr *wrangler.Wrangler, subFlags *fl
}
paths := subFlags.Args()
aliases := make([]*pb.TabletAlias, len(paths))
aliases := make([]*topodatapb.TabletAlias, len(paths))
var err error
for i, path := range paths {
aliases[i], err = topoproto.ParseTabletAlias(path)
@ -2052,7 +2054,7 @@ func commandGetEndPoints(ctx context.Context, wr *wrangler.Wrangler, subFlags *f
if err != nil {
return err
}
tabletType, err := parseTabletType(subFlags.Arg(2), []pb.TabletType{pb.TabletType_MASTER, pb.TabletType_REPLICA, pb.TabletType_RDONLY})
tabletType, err := parseTabletType(subFlags.Arg(2), []topodatapb.TabletType{topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY})
if err != nil {
return err
}
@ -2105,7 +2107,7 @@ func commandPanic(ctx context.Context, wr *wrangler.Wrangler, subFlags *flag.Fla
type rTablet struct {
*topo.TabletInfo
*myproto.ReplicationStatus
*replicationdatapb.Status
}
type rTablets []*rTablet
@ -2121,32 +2123,40 @@ func (rts rTablets) Less(i, j int) bool {
l, r := rts[j], rts[i]
// l or r ReplicationStatus would be nil if we failed to get
// the position (put them at the beginning of the list)
if l.ReplicationStatus == nil {
return r.ReplicationStatus != nil
if l.Status == nil {
return r.Status != nil
}
if r.ReplicationStatus == nil {
if r.Status == nil {
return false
}
var lTypeMaster, rTypeMaster int
if l.Type == pb.TabletType_MASTER {
lTypeMaster = 1
}
if r.Type == pb.TabletType_MASTER {
rTypeMaster = 1
}
if lTypeMaster < rTypeMaster {
// the type proto has MASTER first, so sort by that. Will show
// the MASTER first, then each slave type sorted by
// replication position.
if l.Type < r.Type {
return true
}
if lTypeMaster == rTypeMaster {
return !l.Position.AtLeast(r.Position)
if l.Type > r.Type {
return false
}
return false
// then compare replication positions
lpos, err := myproto.DecodeReplicationPosition(l.Position)
if err != nil {
return true
}
rpos, err := myproto.DecodeReplicationPosition(r.Position)
if err != nil {
return false
}
return !lpos.AtLeast(rpos)
}
func sortReplicatingTablets(tablets []*topo.TabletInfo, stats []*myproto.ReplicationStatus) []*rTablet {
func sortReplicatingTablets(tablets []*topo.TabletInfo, stats []*replicationdatapb.Status) []*rTablet {
rtablets := make([]*rTablet, len(tablets))
for i, status := range stats {
rtablets[i] = &rTablet{TabletInfo: tablets[i], ReplicationStatus: status}
rtablets[i] = &rTablet{
TabletInfo: tablets[i],
Status: status,
}
}
sort.Sort(rTablets(rtablets))
return rtablets

Просмотреть файл

@ -252,10 +252,6 @@ func (sdw *SplitDiffWorker) synchronizeReplication(ctx context.Context) error {
if blpPos == nil {
return fmt.Errorf("no binlog position on the master for Uid %v", ss.Uid)
}
position, err := myproto.DecodeReplicationPosition(blpPos.Position)
if err != nil {
return err
}
// read the tablet
shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout)
@ -268,14 +264,14 @@ func (sdw *SplitDiffWorker) synchronizeReplication(ctx context.Context) error {
// stop replication
sdw.wr.Logger().Infof("Stopping slave[%v] %v at a minimum of %v", i, sdw.sourceAliases[i], blpPos.Position)
shortCtx, cancel = context.WithTimeout(ctx, *remoteActionsTimeout)
stoppedAt, err := sdw.wr.TabletManagerClient().StopSlaveMinimum(shortCtx, sourceTablet, position, *remoteActionsTimeout)
stoppedAt, err := sdw.wr.TabletManagerClient().StopSlaveMinimum(shortCtx, sourceTablet, blpPos.Position, *remoteActionsTimeout)
cancel()
if err != nil {
return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", sdw.sourceAliases[i], blpPos.Position, err)
}
stopPositionList[i] = &pbt.BlpPosition{
Uid: ss.Uid,
Position: myproto.EncodeReplicationPosition(stoppedAt),
Position: stoppedAt,
}
// change the cleaner actions from ChangeSlaveType(rdonly)

Просмотреть файл

@ -254,10 +254,6 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication(ctx context.Context)
if blpPos == nil {
return fmt.Errorf("no binlog position on the master for Uid %v", ss.Uid)
}
pos, err := myproto.DecodeReplicationPosition(blpPos.Position)
if err != nil {
return err
}
// stop replication
vsdw.wr.Logger().Infof("Stopping slave %v at a minimum of %v", topoproto.TabletAliasString(vsdw.sourceAlias), blpPos.Position)
@ -268,14 +264,14 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication(ctx context.Context)
return err
}
shortCtx, cancel = context.WithTimeout(ctx, *remoteActionsTimeout)
stoppedAt, err := vsdw.wr.TabletManagerClient().StopSlaveMinimum(shortCtx, sourceTablet, pos, *remoteActionsTimeout)
stoppedAt, err := vsdw.wr.TabletManagerClient().StopSlaveMinimum(shortCtx, sourceTablet, blpPos.Position, *remoteActionsTimeout)
cancel()
if err != nil {
return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", topoproto.TabletAliasString(vsdw.sourceAlias), blpPos.Position, err)
}
stopPositionList[0] = &pbt.BlpPosition{
Uid: ss.Uid,
Position: myproto.EncodeReplicationPosition(stoppedAt),
Position: stoppedAt,
}
// change the cleaner actions from ChangeSlaveType(rdonly)

Просмотреть файл

@ -11,7 +11,6 @@ import (
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/vt/concurrency"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
@ -19,8 +18,8 @@ import (
"github.com/youtube/vitess/go/vt/topotools/events"
"golang.org/x/net/context"
pbt "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// keyspace related methods for Wrangler
@ -35,7 +34,7 @@ func (wr *Wrangler) unlockKeyspace(ctx context.Context, keyspace string, actionN
// SetKeyspaceShardingInfo locks a keyspace and sets its ShardingColumnName
// and ShardingColumnType
func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType pb.KeyspaceIdType, splitShardCount int32, force bool) error {
func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType topodatapb.KeyspaceIdType, splitShardCount int32, force bool) error {
actionNode := actionnode.SetKeyspaceShardingInfo()
lockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
if err != nil {
@ -47,7 +46,7 @@ func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shard
}
func (wr *Wrangler) setKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType pb.KeyspaceIdType, splitShardCount int32, force bool) error {
func (wr *Wrangler) setKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType topodatapb.KeyspaceIdType, splitShardCount int32, force bool) error {
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
if err != nil {
return err
@ -61,7 +60,7 @@ func (wr *Wrangler) setKeyspaceShardingInfo(ctx context.Context, keyspace, shard
}
}
if ki.ShardingColumnType != pb.KeyspaceIdType_UNSET && ki.ShardingColumnType != shardingColumnType {
if ki.ShardingColumnType != topodatapb.KeyspaceIdType_UNSET && ki.ShardingColumnType != shardingColumnType {
if force {
wr.Logger().Warningf("Forcing keyspace ShardingColumnType change from %v to %v", ki.ShardingColumnType, shardingColumnType)
} else {
@ -77,8 +76,8 @@ func (wr *Wrangler) setKeyspaceShardingInfo(ctx context.Context, keyspace, shard
// MigrateServedTypes is used during horizontal splits to migrate a
// served type from a list of shards to another.
func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType pb.TabletType, reverse, skipReFreshState bool, filteredReplicationWaitTime time.Duration) error {
if servedType == pb.TabletType_MASTER {
func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType topodatapb.TabletType, reverse, skipReFreshState bool, filteredReplicationWaitTime time.Duration) error {
if servedType == topodatapb.TabletType_MASTER {
// we cannot migrate a master back, since when master migration
// is done, the source shards are dead
if reverse {
@ -175,7 +174,7 @@ func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard stri
// - we're not migrating a master
// - we don't have any errors
// - we're not told to skip the refresh
if servedType != pb.TabletType_MASTER && !rec.HasErrors() && !skipReFreshState {
if servedType != topodatapb.TabletType_MASTER && !rec.HasErrors() && !skipReFreshState {
var refreshShards []*topo.ShardInfo
if reverse {
// For a backwards migration, we just disabled query service on the destination shards
@ -192,9 +191,9 @@ func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard stri
return rec.Error()
}
func (wr *Wrangler) getMastersPosition(ctx context.Context, shards []*topo.ShardInfo) (map[*topo.ShardInfo]myproto.ReplicationPosition, error) {
func (wr *Wrangler) getMastersPosition(ctx context.Context, shards []*topo.ShardInfo) (map[*topo.ShardInfo]string, error) {
mu := sync.Mutex{}
result := make(map[*topo.ShardInfo]myproto.ReplicationPosition)
result := make(map[*topo.ShardInfo]string)
wg := sync.WaitGroup{}
rec := concurrency.AllErrorRecorder{}
@ -225,7 +224,7 @@ func (wr *Wrangler) getMastersPosition(ctx context.Context, shards []*topo.Shard
return result, rec.Error()
}
func (wr *Wrangler) waitForFilteredReplication(ctx context.Context, sourcePositions map[*topo.ShardInfo]myproto.ReplicationPosition, destinationShards []*topo.ShardInfo, waitTime time.Duration) error {
func (wr *Wrangler) waitForFilteredReplication(ctx context.Context, sourcePositions map[*topo.ShardInfo]string, destinationShards []*topo.ShardInfo, waitTime time.Duration) error {
wg := sync.WaitGroup{}
rec := concurrency.AllErrorRecorder{}
for _, si := range destinationShards {
@ -234,14 +233,14 @@ func (wr *Wrangler) waitForFilteredReplication(ctx context.Context, sourcePositi
defer wg.Done()
for _, sourceShard := range si.SourceShards {
// we're waiting on this guy
blpPosition := &pbt.BlpPosition{
blpPosition := &tabletmanagerdatapb.BlpPosition{
Uid: sourceShard.Uid,
}
// find the position it should be at
for s, pos := range sourcePositions {
if s.Keyspace() == sourceShard.Keyspace && s.ShardName() == sourceShard.Shard {
blpPosition.Position = myproto.EncodeReplicationPosition(pos)
blpPosition.Position = pos
}
}
@ -292,7 +291,7 @@ func (wr *Wrangler) refreshMasters(ctx context.Context, shards []*topo.ShardInfo
}
// migrateServedTypes operates with all concerned shards locked.
func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sourceShards, destinationShards []*topo.ShardInfo, cells []string, servedType pb.TabletType, reverse bool, filteredReplicationWaitTime time.Duration) (err error) {
func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sourceShards, destinationShards []*topo.ShardInfo, cells []string, servedType topodatapb.TabletType, reverse bool, filteredReplicationWaitTime time.Duration) (err error) {
// re-read all the shards so we are up to date
wr.Logger().Infof("Re-reading all shards")
@ -326,10 +325,10 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
// - gather all replication points
// - wait for filtered replication to catch up before we continue
// - disable filtered replication after the fact
if servedType == pb.TabletType_MASTER {
if servedType == topodatapb.TabletType_MASTER {
event.DispatchUpdate(ev, "disabling query service on all source masters")
for _, si := range sourceShards {
if err := si.UpdateDisableQueryService(pb.TabletType_MASTER, nil, true); err != nil {
if err := si.UpdateDisableQueryService(topodatapb.TabletType_MASTER, nil, true); err != nil {
return err
}
if err := wr.ts.UpdateShard(ctx, si); err != nil {
@ -373,7 +372,7 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
}
needToRefreshSourceTablets = true
}
if !reverse && servedType != pb.TabletType_MASTER {
if !reverse && servedType != topodatapb.TabletType_MASTER {
// this is a forward migration, we need to disable
// query service on the source shards.
// (this was already done for masters earlier)
@ -398,7 +397,7 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
}
needToRefreshDestinationTablets = true
}
if reverse && servedType != pb.TabletType_MASTER {
if reverse && servedType != topodatapb.TabletType_MASTER {
// this is a backwards migration, we need to disable
// query service on the destination shards.
// (we're not allowed to reverse a master migration)
@ -437,7 +436,7 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
// And tell the new shards masters they can now be read-write.
// Invoking a remote action will also make the tablet stop filtered
// replication.
if servedType == pb.TabletType_MASTER {
if servedType == topodatapb.TabletType_MASTER {
event.DispatchUpdate(ev, "setting destination masters read-write")
if err := wr.refreshMasters(ctx, destinationShards); err != nil {
return err
@ -450,7 +449,7 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
// MigrateServedFrom is used during vertical splits to migrate a
// served type from a keyspace to another.
func (wr *Wrangler) MigrateServedFrom(ctx context.Context, keyspace, shard string, servedType pb.TabletType, cells []string, reverse bool, filteredReplicationWaitTime time.Duration) error {
func (wr *Wrangler) MigrateServedFrom(ctx context.Context, keyspace, shard string, servedType topodatapb.TabletType, cells []string, reverse bool, filteredReplicationWaitTime time.Duration) error {
// read the destination keyspace, check it
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
if err != nil {
@ -516,7 +515,7 @@ func (wr *Wrangler) MigrateServedFrom(ctx context.Context, keyspace, shard strin
return rec.Error()
}
func (wr *Wrangler) migrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo, destinationShard *topo.ShardInfo, servedType pb.TabletType, cells []string, reverse bool, filteredReplicationWaitTime time.Duration) (err error) {
func (wr *Wrangler) migrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo, destinationShard *topo.ShardInfo, servedType topodatapb.TabletType, cells []string, reverse bool, filteredReplicationWaitTime time.Duration) (err error) {
// re-read and update keyspace info record
ki, err = wr.ts.GetKeyspace(ctx, ki.KeyspaceName())
@ -561,7 +560,7 @@ func (wr *Wrangler) migrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo
}
}()
if servedType == pb.TabletType_MASTER {
if servedType == topodatapb.TabletType_MASTER {
err = wr.masterMigrateServedFrom(ctx, ki, sourceShard, destinationShard, tables, ev, filteredReplicationWaitTime)
} else {
err = wr.replicaMigrateServedFrom(ctx, ki, sourceShard, destinationShard, servedType, cells, reverse, tables, ev)
@ -571,7 +570,7 @@ func (wr *Wrangler) migrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo
}
// replicaMigrateServedFrom handles the slave (replica, rdonly) migration.
func (wr *Wrangler) replicaMigrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo, sourceShard *topo.ShardInfo, destinationShard *topo.ShardInfo, servedType pb.TabletType, cells []string, reverse bool, tables []string, ev *events.MigrateServedFrom) error {
func (wr *Wrangler) replicaMigrateServedFrom(ctx context.Context, ki *topo.KeyspaceInfo, sourceShard *topo.ShardInfo, destinationShard *topo.ShardInfo, servedType topodatapb.TabletType, cells []string, reverse bool, tables []string, ev *events.MigrateServedFrom) error {
// Save the destination keyspace (its ServedFrom has been changed)
event.DispatchUpdate(ev, "updating keyspace")
if err := wr.ts.UpdateKeyspace(ctx, ki); err != nil {
@ -620,7 +619,7 @@ func (wr *Wrangler) masterMigrateServedFrom(ctx context.Context, ki *topo.Keyspa
// Update source shard (more blacklisted tables)
event.DispatchUpdate(ev, "updating source shard")
if err := sourceShard.UpdateSourceBlacklistedTables(pb.TabletType_MASTER, nil, false, tables); err != nil {
if err := sourceShard.UpdateSourceBlacklistedTables(topodatapb.TabletType_MASTER, nil, false, tables); err != nil {
return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
}
if err := wr.ts.UpdateShard(ctx, sourceShard); err != nil {
@ -642,9 +641,9 @@ func (wr *Wrangler) masterMigrateServedFrom(ctx context.Context, ki *topo.Keyspa
// wait for it
event.DispatchUpdate(ev, "waiting for destination master to catch up to source master")
if err := wr.tmc.WaitBlpPosition(ctx, destinationMasterTabletInfo, &pbt.BlpPosition{
if err := wr.tmc.WaitBlpPosition(ctx, destinationMasterTabletInfo, &tabletmanagerdatapb.BlpPosition{
Uid: 0,
Position: myproto.EncodeReplicationPosition(masterPosition),
Position: masterPosition,
}, filteredReplicationWaitTime); err != nil {
return err
}
@ -674,7 +673,7 @@ func (wr *Wrangler) masterMigrateServedFrom(ctx context.Context, ki *topo.Keyspa
}
// SetKeyspaceServedFrom locks a keyspace and changes its ServerFromMap
func (wr *Wrangler) SetKeyspaceServedFrom(ctx context.Context, keyspace string, servedType pb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
func (wr *Wrangler) SetKeyspaceServedFrom(ctx context.Context, keyspace string, servedType topodatapb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
actionNode := actionnode.SetKeyspaceServedFrom()
lockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
if err != nil {
@ -685,7 +684,7 @@ func (wr *Wrangler) SetKeyspaceServedFrom(ctx context.Context, keyspace string,
return wr.unlockKeyspace(ctx, keyspace, actionNode, lockPath, err)
}
func (wr *Wrangler) setKeyspaceServedFrom(ctx context.Context, keyspace string, servedType pb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
func (wr *Wrangler) setKeyspaceServedFrom(ctx context.Context, keyspace string, servedType topodatapb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
if err != nil {
return err
@ -699,7 +698,7 @@ func (wr *Wrangler) setKeyspaceServedFrom(ctx context.Context, keyspace string,
// RefreshTablesByShard calls RefreshState on all the tables of a
// given type in a shard. It would work for the master, but the
// discovery wouldn't be very efficient.
func (wr *Wrangler) RefreshTablesByShard(ctx context.Context, si *topo.ShardInfo, tabletType pb.TabletType, cells []string) error {
func (wr *Wrangler) RefreshTablesByShard(ctx context.Context, si *topo.ShardInfo, tabletType topodatapb.TabletType, cells []string) error {
wr.Logger().Infof("RefreshTablesByShard called on shard %v/%v", si.Keyspace(), si.ShardName())
tabletMap, err := wr.ts.GetTabletMapForShardByCell(ctx, si.Keyspace(), si.ShardName(), cells)
switch err {

Просмотреть файл

@ -23,7 +23,8 @@ import (
"github.com/youtube/vitess/go/vt/topotools/events"
"golang.org/x/net/context"
pb "github.com/youtube/vitess/go/vt/proto/topodata"
replicationdatapb "github.com/youtube/vitess/go/vt/proto/replicationdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
const (
@ -38,12 +39,12 @@ const (
// helper struct to queue up results
type rpcContext struct {
tablet *topo.TabletInfo
status *myproto.ReplicationStatus
status *replicationdatapb.Status
err error
}
// ShardReplicationStatuses returns the ReplicationStatus for each tablet in a shard.
func (wr *Wrangler) ShardReplicationStatuses(ctx context.Context, keyspace, shard string) ([]*topo.TabletInfo, []*myproto.ReplicationStatus, error) {
func (wr *Wrangler) ShardReplicationStatuses(ctx context.Context, keyspace, shard string) ([]*topo.TabletInfo, []*replicationdatapb.Status, error) {
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
if err != nil {
return nil, nil, err
@ -60,7 +61,7 @@ func (wr *Wrangler) ShardReplicationStatuses(ctx context.Context, keyspace, shar
return tabletMap, posMap, wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
}
func (wr *Wrangler) shardReplicationStatuses(ctx context.Context, shardInfo *topo.ShardInfo) ([]*topo.TabletInfo, []*myproto.ReplicationStatus, error) {
func (wr *Wrangler) shardReplicationStatuses(ctx context.Context, shardInfo *topo.ShardInfo) ([]*topo.TabletInfo, []*replicationdatapb.Status, error) {
// FIXME(msolomon) this assumes no hierarchical replication, which is currently the case.
tabletMap, err := wr.ts.GetTabletMapForShard(ctx, shardInfo.Keyspace(), shardInfo.ShardName())
if err != nil {
@ -73,16 +74,16 @@ func (wr *Wrangler) shardReplicationStatuses(ctx context.Context, shardInfo *top
// tabletReplicationStatuses returns the ReplicationStatus of each tablet in
// tablets.
func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*topo.TabletInfo) ([]*myproto.ReplicationStatus, error) {
func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*topo.TabletInfo) ([]*replicationdatapb.Status, error) {
wr.logger.Infof("tabletReplicationStatuses: %v", tablets)
wg := sync.WaitGroup{}
rec := concurrency.AllErrorRecorder{}
result := make([]*myproto.ReplicationStatus, len(tablets))
result := make([]*replicationdatapb.Status, len(tablets))
for i, ti := range tablets {
// Don't scan tablets that won't return something
// useful. Otherwise, you'll end up waiting for a timeout.
if ti.Type == pb.TabletType_MASTER {
if ti.Type == topodatapb.TabletType_MASTER {
wg.Add(1)
go func(i int, ti *topo.TabletInfo) {
defer wg.Done()
@ -91,7 +92,9 @@ func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*to
rec.RecordError(fmt.Errorf("MasterPosition(%v) failed: %v", ti.AliasString(), err))
return
}
result[i] = &myproto.ReplicationStatus{Position: pos}
result[i] = &replicationdatapb.Status{
Position: pos,
}
}(i, ti)
} else if ti.IsSlaveType() {
wg.Add(1)
@ -102,7 +105,7 @@ func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*to
rec.RecordError(fmt.Errorf("SlaveStatus(%v) failed: %v", ti.AliasString(), err))
return
}
result[i] = &status
result[i] = status
}(i, ti)
}
}
@ -113,7 +116,7 @@ func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*to
// ReparentTablet tells a tablet to reparent this tablet to the current
// master, based on the current replication position. If there is no
// match, it will fail.
func (wr *Wrangler) ReparentTablet(ctx context.Context, tabletAlias *pb.TabletAlias) error {
func (wr *Wrangler) ReparentTablet(ctx context.Context, tabletAlias *topodatapb.TabletAlias) error {
// Get specified tablet.
// Get current shard master tablet.
// Sanity check they are in the same keyspace/shard.
@ -137,7 +140,7 @@ func (wr *Wrangler) ReparentTablet(ctx context.Context, tabletAlias *pb.TabletAl
}
// Basic sanity checking.
if masterTi.Type != pb.TabletType_MASTER {
if masterTi.Type != topodatapb.TabletType_MASTER {
return fmt.Errorf("TopologyServer has inconsistent state for shard master %v", topoproto.TabletAliasString(shardInfo.MasterAlias))
}
if masterTi.Keyspace != ti.Keyspace || masterTi.Shard != ti.Shard {
@ -149,7 +152,7 @@ func (wr *Wrangler) ReparentTablet(ctx context.Context, tabletAlias *pb.TabletAl
}
// InitShardMaster will make the provided tablet the master for the shard.
func (wr *Wrangler) InitShardMaster(ctx context.Context, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, force bool, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) InitShardMaster(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, force bool, waitSlaveTimeout time.Duration) error {
// lock the shard
actionNode := actionnode.ReparentShard(initShardMasterOperation, masterElectTabletAlias)
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
@ -172,7 +175,7 @@ func (wr *Wrangler) InitShardMaster(ctx context.Context, keyspace, shard string,
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
}
func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, force bool, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, force bool, waitSlaveTimeout time.Duration) error {
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
if err != nil {
return err
@ -207,7 +210,7 @@ func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Repare
wr.logger.Warningf("master-elect tablet %v is not a master in the shard, proceeding anyway as -force was used", topoproto.TabletAliasString(masterElectTabletAlias))
}
haveOtherMaster := false
for alias, _ := range masterTabletMap {
for alias := range masterTabletMap {
if !topoproto.TabletAliasEqual(&alias, masterElectTabletAlias) {
haveOtherMaster = true
}
@ -228,7 +231,7 @@ func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Repare
rec := concurrency.AllErrorRecorder{}
for alias, tabletInfo := range tabletMap {
wg.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wg.Done()
wr.logger.Infof("resetting replication on tablet %v", topoproto.TabletAliasString(&alias))
if err := wr.TabletManagerClient().ResetReplication(ctx, tabletInfo); err != nil {
@ -264,14 +267,14 @@ func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Repare
for alias, tabletInfo := range tabletMap {
if topoproto.TabletAliasEqual(&alias, masterElectTabletAlias) {
wgMaster.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgMaster.Done()
wr.logger.Infof("populating reparent journal on new master %v", topoproto.TabletAliasString(&alias))
masterErr = wr.TabletManagerClient().PopulateReparentJournal(ctx, tabletInfo, now, initShardMasterOperation, &alias, rp)
}(alias, tabletInfo)
} else {
wgSlaves.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgSlaves.Done()
wr.logger.Infof("initializing slave %v", topoproto.TabletAliasString(&alias))
if err := wr.TabletManagerClient().InitSlave(ctx, tabletInfo, masterElectTabletAlias, rp, now); err != nil {
@ -324,7 +327,7 @@ func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Repare
// PlannedReparentShard will make the provided tablet the master for the shard,
// when both the current and new master are reachable and in good shape.
func (wr *Wrangler) PlannedReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) PlannedReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
// lock the shard
actionNode := actionnode.ReparentShard(plannedReparentShardOperation, masterElectTabletAlias)
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
@ -347,7 +350,7 @@ func (wr *Wrangler) PlannedReparentShard(ctx context.Context, keyspace, shard st
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
}
func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
if err != nil {
return err
@ -404,14 +407,14 @@ func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.R
for alias, tabletInfo := range tabletMap {
if topoproto.TabletAliasEqual(&alias, masterElectTabletAlias) {
wgMaster.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgMaster.Done()
wr.logger.Infof("populating reparent journal on new master %v", topoproto.TabletAliasString(&alias))
masterErr = wr.TabletManagerClient().PopulateReparentJournal(ctx, tabletInfo, now, plannedReparentShardOperation, &alias, rp)
}(alias, tabletInfo)
} else {
wgSlaves.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgSlaves.Done()
wr.logger.Infof("setting new master on slave %v", topoproto.TabletAliasString(&alias))
// also restart replication on old master
@ -456,7 +459,7 @@ func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.R
// EmergencyReparentShard will make the provided tablet the master for
// the shard, when the old master is completely unreachable.
func (wr *Wrangler) EmergencyReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) EmergencyReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
// lock the shard
actionNode := actionnode.ReparentShard(emergencyReparentShardOperation, masterElectTabletAlias)
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
@ -479,7 +482,7 @@ func (wr *Wrangler) EmergencyReparentShard(ctx context.Context, keyspace, shard
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
}
func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *pb.TabletAlias, waitSlaveTimeout time.Duration) error {
func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
if err != nil {
return err
@ -535,10 +538,10 @@ func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events
event.DispatchUpdate(ev, "stop replication on all slaves")
wg := sync.WaitGroup{}
mu := sync.Mutex{}
statusMap := make(map[pb.TabletAlias]myproto.ReplicationStatus)
statusMap := make(map[topodatapb.TabletAlias]*replicationdatapb.Status)
for alias, tabletInfo := range tabletMap {
wg.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wg.Done()
wr.logger.Infof("getting replication position from %v", topoproto.TabletAliasString(&alias))
ctx, cancel := context.WithTimeout(ctx, waitSlaveTimeout)
@ -560,11 +563,19 @@ func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events
if !ok {
return fmt.Errorf("couldn't get master elect %v replication position", topoproto.TabletAliasString(masterElectTabletAlias))
}
masterElectPos, err := myproto.DecodeReplicationPosition(masterElectStatus.Position)
if err != nil {
return fmt.Errorf("cannot decode master elect position %v: %v", masterElectStatus.Position, err)
}
for alias, status := range statusMap {
if topoproto.TabletAliasEqual(&alias, masterElectTabletAlias) {
continue
}
if !masterElectStatus.Position.AtLeast(status.Position) {
pos, err := myproto.DecodeReplicationPosition(status.Position)
if err != nil {
return fmt.Errorf("cannot decode slave %v position %v: %v", topoproto.TabletAliasString(&alias), status.Position, err)
}
if !masterElectPos.AtLeast(pos) {
return fmt.Errorf("tablet %v is more advanced than master elect tablet %v: %v > %v", topoproto.TabletAliasString(&alias), topoproto.TabletAliasString(masterElectTabletAlias), status.Position, masterElectStatus)
}
}
@ -591,19 +602,19 @@ func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events
for alias, tabletInfo := range tabletMap {
if topoproto.TabletAliasEqual(&alias, masterElectTabletAlias) {
wgMaster.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgMaster.Done()
wr.logger.Infof("populating reparent journal on new master %v", topoproto.TabletAliasString(&alias))
masterErr = wr.TabletManagerClient().PopulateReparentJournal(ctx, tabletInfo, now, emergencyReparentShardOperation, &alias, rp)
}(alias, tabletInfo)
} else {
wgSlaves.Add(1)
go func(alias pb.TabletAlias, tabletInfo *topo.TabletInfo) {
go func(alias topodatapb.TabletAlias, tabletInfo *topo.TabletInfo) {
defer wgSlaves.Done()
wr.logger.Infof("setting new master on slave %v", topoproto.TabletAliasString(&alias))
forceStartSlave := false
if status, ok := statusMap[alias]; ok {
forceStartSlave = status.SlaveIORunning || status.SlaveSQLRunning
forceStartSlave = status.SlaveIoRunning || status.SlaveSqlRunning
}
if err := wr.TabletManagerClient().SetMaster(ctx, tabletInfo, masterElectTabletAlias, now, forceStartSlave); err != nil {
rec.RecordError(fmt.Errorf("Tablet %v SetMaster failed: %v", topoproto.TabletAliasString(&alias), err))