зеркало из https://github.com/github/vitess-gh.git
Коммит
525f614cd2
|
@ -18,7 +18,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver/querytypes"
|
||||
|
@ -482,7 +481,7 @@ func (itmc *internalTabletManagerClient) Ping(ctx context.Context, tablet *topod
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrap(ctx, actionnode.TabletActionPing, nil, nil, func() error {
|
||||
return t.agent.RPCWrap(ctx, tabletmanager.TabletActionPing, nil, nil, func() error {
|
||||
t.agent.Ping(ctx, "payload")
|
||||
return nil
|
||||
})
|
||||
|
@ -494,7 +493,7 @@ func (itmc *internalTabletManagerClient) GetSchema(ctx context.Context, tablet *
|
|||
return nil, fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
var result *tabletmanagerdatapb.SchemaDefinition
|
||||
if err := t.agent.RPCWrap(ctx, actionnode.TabletActionGetSchema, nil, nil, func() error {
|
||||
if err := t.agent.RPCWrap(ctx, tabletmanager.TabletActionGetSchema, nil, nil, func() error {
|
||||
sd, err := t.agent.GetSchema(ctx, tables, excludeTables, includeViews)
|
||||
if err == nil {
|
||||
result = sd
|
||||
|
@ -512,7 +511,7 @@ func (itmc *internalTabletManagerClient) GetPermissions(ctx context.Context, tab
|
|||
return nil, fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
var result *tabletmanagerdatapb.Permissions
|
||||
if err := t.agent.RPCWrap(ctx, actionnode.TabletActionGetPermissions, nil, nil, func() error {
|
||||
if err := t.agent.RPCWrap(ctx, tabletmanager.TabletActionGetPermissions, nil, nil, func() error {
|
||||
p, err := t.agent.GetPermissions(ctx)
|
||||
if err == nil {
|
||||
result = p
|
||||
|
@ -541,7 +540,7 @@ func (itmc *internalTabletManagerClient) Sleep(ctx context.Context, tablet *topo
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSleep, nil, nil, true, func() error {
|
||||
return t.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSleep, nil, nil, true, func() error {
|
||||
t.agent.Sleep(ctx, duration)
|
||||
return nil
|
||||
})
|
||||
|
@ -556,7 +555,7 @@ func (itmc *internalTabletManagerClient) RefreshState(ctx context.Context, table
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, nil, nil, true, func() error {
|
||||
return t.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionRefreshState, nil, nil, true, func() error {
|
||||
t.agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -567,7 +566,7 @@ func (itmc *internalTabletManagerClient) RunHealthCheck(ctx context.Context, tab
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrap(ctx, actionnode.TabletActionRunHealthCheck, nil, nil, func() error {
|
||||
return t.agent.RPCWrap(ctx, tabletmanager.TabletActionRunHealthCheck, nil, nil, func() error {
|
||||
t.agent.RunHealthCheck(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -578,7 +577,7 @@ func (itmc *internalTabletManagerClient) IgnoreHealthError(ctx context.Context,
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrap(ctx, actionnode.TabletActionIgnoreHealthError, nil, nil, func() error {
|
||||
return t.agent.RPCWrap(ctx, tabletmanager.TabletActionIgnoreHealthError, nil, nil, func() error {
|
||||
t.agent.IgnoreHealthError(ctx, pattern)
|
||||
return nil
|
||||
})
|
||||
|
@ -589,7 +588,7 @@ func (itmc *internalTabletManagerClient) ReloadSchema(ctx context.Context, table
|
|||
if !ok {
|
||||
return fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
return t.agent.RPCWrapLockAction(ctx, actionnode.TabletActionReloadSchema, nil, nil, true, func() error {
|
||||
return t.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionReloadSchema, nil, nil, true, func() error {
|
||||
t.agent.ReloadSchema(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -601,7 +600,7 @@ func (itmc *internalTabletManagerClient) PreflightSchema(ctx context.Context, ta
|
|||
return nil, fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
var result *tmutils.SchemaChangeResult
|
||||
if err := t.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPreflightSchema, nil, nil, true, func() error {
|
||||
if err := t.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionPreflightSchema, nil, nil, true, func() error {
|
||||
scr, err := t.agent.PreflightSchema(ctx, change)
|
||||
if err == nil {
|
||||
result = scr
|
||||
|
@ -619,7 +618,7 @@ func (itmc *internalTabletManagerClient) ApplySchema(ctx context.Context, tablet
|
|||
return nil, fmt.Errorf("tmclient: cannot find tablet %v", tablet.Alias.Uid)
|
||||
}
|
||||
var result *tmutils.SchemaChangeResult
|
||||
if err := t.agent.RPCWrapLockAction(ctx, actionnode.TabletActionApplySchema, nil, nil, true, func() error {
|
||||
if err := t.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionApplySchema, nil, nil, true, func() error {
|
||||
scr, err := t.agent.ApplySchema(ctx, change)
|
||||
if err == nil {
|
||||
result = scr
|
||||
|
@ -715,7 +714,7 @@ func (itmc *internalTabletManagerClient) SetMaster(ctx context.Context, tablet *
|
|||
return fmt.Errorf("not implemented in vtcombo")
|
||||
}
|
||||
|
||||
func (itmc *internalTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, args *actionnode.SlaveWasRestartedArgs) error {
|
||||
func (itmc *internalTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias) error {
|
||||
return fmt.Errorf("not implemented in vtcombo")
|
||||
}
|
||||
|
||||
|
|
|
@ -217,23 +217,31 @@ func (m *Tablet) GetTags() map[string]string {
|
|||
|
||||
// A Shard contains data about a subset of the data whithin a keyspace.
|
||||
type Shard struct {
|
||||
// master_alias is the tablet alias of the master for the shard.
|
||||
// If it is unset, then there is no master in this shard yet.
|
||||
// No lock is necessary to update this field, when for instance
|
||||
// TabletExternallyReparented updates this. However, we lock the
|
||||
// shard for reparenting operations (InitShardMaster,
|
||||
// PlannedReparentShard,EmergencyReparentShard), to guarantee
|
||||
// exclusive operation.
|
||||
MasterAlias *TabletAlias `protobuf:"bytes,1,opt,name=master_alias,json=masterAlias" json:"master_alias,omitempty"`
|
||||
// key_range is the KeyRange for this shard. It can be unset if:
|
||||
// - we are not using range-based sharding in this shard.
|
||||
// - the shard covers the entire keyrange.
|
||||
// This must match the shard name based on our other conventions, but
|
||||
// helpful to have it decomposed here.
|
||||
// Once set at creation time, it is never changed.
|
||||
KeyRange *KeyRange `protobuf:"bytes,2,opt,name=key_range,json=keyRange" json:"key_range,omitempty"`
|
||||
// served_types has at most one entry per TabletType
|
||||
// The keyspace lock is always taken when changing this.
|
||||
ServedTypes []*Shard_ServedType `protobuf:"bytes,3,rep,name=served_types,json=servedTypes" json:"served_types,omitempty"`
|
||||
// SourceShards is the list of shards we're replicating from,
|
||||
// using filtered replication.
|
||||
// The keyspace lock is always taken when changing this.
|
||||
SourceShards []*Shard_SourceShard `protobuf:"bytes,4,rep,name=source_shards,json=sourceShards" json:"source_shards,omitempty"`
|
||||
// Cells is the list of cells that contain tablets for this shard.
|
||||
// No lock is necessary to update this field.
|
||||
Cells []string `protobuf:"bytes,5,rep,name=cells" json:"cells,omitempty"`
|
||||
// tablet_controls has at most one entry per TabletType
|
||||
// tablet_controls has at most one entry per TabletType.
|
||||
// The keyspace lock is always taken when changing this.
|
||||
TabletControls []*Shard_TabletControl `protobuf:"bytes,6,rep,name=tablet_controls,json=tabletControls" json:"tablet_controls,omitempty"`
|
||||
}
|
||||
|
||||
|
|
|
@ -1,292 +0,0 @@
|
|||
// Copyright 2012, Google Inc. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
// Actions modify the state of a tablet, shard or keyspace.
|
||||
//
|
||||
// They are currently managed through a series of queues stored in
|
||||
// topology server, or RPCs. Switching to RPCs only now.
|
||||
|
||||
package actionnode
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"os"
|
||||
"os/user"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
// FIXME(msolomon) why is ActionState a type, but Action is not?
|
||||
|
||||
//
|
||||
// Tablet actions. This first list is RPC only. In the process
|
||||
// of converting them all to RPCs.
|
||||
//
|
||||
|
||||
// TabletActionPing checks a tablet is alive
|
||||
TabletActionPing = "Ping"
|
||||
|
||||
// TabletActionSleep will sleep for a duration (used for tests)
|
||||
TabletActionSleep = "Sleep"
|
||||
|
||||
// TabletActionExecuteHook will execute the provided hook remotely
|
||||
TabletActionExecuteHook = "ExecuteHook"
|
||||
|
||||
// TabletActionSetReadOnly makes the mysql instance read-only
|
||||
TabletActionSetReadOnly = "SetReadOnly"
|
||||
|
||||
// TabletActionSetReadWrite makes the mysql instance read-write
|
||||
TabletActionSetReadWrite = "SetReadWrite"
|
||||
|
||||
// TabletActionChangeType changes the type of the tablet
|
||||
TabletActionChangeType = "ChangeType"
|
||||
|
||||
// TabletActionResetReplication tells the tablet it should
|
||||
// reset its replication state
|
||||
TabletActionResetReplication = "ResetReplication"
|
||||
|
||||
// TabletActionInitMaster tells the tablet it should make itself the new
|
||||
// master for the shard it's currently in.
|
||||
TabletActionInitMaster = "InitMaster"
|
||||
|
||||
// TabletActionPopulateReparentJournal inserts an entry in the
|
||||
// _vt.reparent_journal table
|
||||
TabletActionPopulateReparentJournal = "PopulateReparentJournal"
|
||||
|
||||
// TabletActionInitSlave tells the tablet it should make
|
||||
// itself a slave to the provided master at the given position.
|
||||
TabletActionInitSlave = "InitSlave"
|
||||
|
||||
// TabletActionDemoteMaster tells the current master it's
|
||||
// about to not be a master any more, and should go read-only.
|
||||
TabletActionDemoteMaster = "DemoteMaster"
|
||||
|
||||
// TabletActionPromoteSlaveWhenCaughtUp tells the tablet to wait
|
||||
// for a given replication point, and when it reaches it
|
||||
// switch to be a master.
|
||||
TabletActionPromoteSlaveWhenCaughtUp = "PromoteSlaveWhenCaughtUp"
|
||||
|
||||
// TabletActionSlaveWasPromoted tells a tablet this previously slave
|
||||
// tablet is now the master. The tablet will update its
|
||||
// own topology record.
|
||||
TabletActionSlaveWasPromoted = "SlaveWasPromoted"
|
||||
|
||||
// TabletActionSetMaster tells a tablet it has a new master.
|
||||
// The tablet will reparent to the new master, and wait for
|
||||
// the reparent_journal entry.
|
||||
TabletActionSetMaster = "SetMaster"
|
||||
|
||||
// TabletActionSlaveWasRestarted tells a tablet the mysql
|
||||
// master was changed. The tablet will check it is indeed the
|
||||
// case, and update its own topology record.
|
||||
TabletActionSlaveWasRestarted = "SlaveWasRestarted"
|
||||
|
||||
// TabletActionStopReplicationAndGetStatus will stop replication,
|
||||
// and return the current replication status.
|
||||
TabletActionStopReplicationAndGetStatus = "StopReplicationAndGetStatus"
|
||||
|
||||
// TabletActionPromoteSlave will make this tablet the master
|
||||
TabletActionPromoteSlave = "PromoteSlave"
|
||||
|
||||
// TabletActionStopSlave will stop MySQL replication.
|
||||
TabletActionStopSlave = "StopSlave"
|
||||
|
||||
// TabletActionStopSlaveMinimum will stop MySQL replication
|
||||
// after it reaches a minimum point.
|
||||
TabletActionStopSlaveMinimum = "StopSlaveMinimum"
|
||||
|
||||
// TabletActionStartSlave will start MySQL replication.
|
||||
TabletActionStartSlave = "StartSlave"
|
||||
|
||||
// TabletActionExternallyReparented is sent directly to the new master
|
||||
// tablet when it becomes the master. It is functionnaly equivalent
|
||||
// to calling "ShardExternallyReparented" on the topology.
|
||||
TabletActionExternallyReparented = "TabletExternallyReparented"
|
||||
|
||||
// TabletActionMasterPosition returns the current master position
|
||||
TabletActionMasterPosition = "MasterPosition"
|
||||
|
||||
// TabletActionSlaveStatus returns the current slave status
|
||||
TabletActionSlaveStatus = "SlaveStatus"
|
||||
|
||||
// TabletActionWaitBLPPosition waits until the slave reaches a
|
||||
// replication position in filtered replication
|
||||
TabletActionWaitBLPPosition = "WaitBlpPosition"
|
||||
|
||||
// TabletActionStopBLP stops filtered replication
|
||||
TabletActionStopBLP = "StopBlp"
|
||||
|
||||
// TabletActionStartBLP starts filtered replication
|
||||
TabletActionStartBLP = "StartBlp"
|
||||
|
||||
// TabletActionRunBLPUntil will run filtered replication until
|
||||
// it reaches the provided stop position.
|
||||
TabletActionRunBLPUntil = "RunBlpUntil"
|
||||
|
||||
// TabletActionGetSchema returns the tablet current schema.
|
||||
TabletActionGetSchema = "GetSchema"
|
||||
|
||||
// TabletActionRefreshState tells the tablet to refresh its
|
||||
// tablet record from the topo server.
|
||||
TabletActionRefreshState = "RefreshState"
|
||||
|
||||
// TabletActionRunHealthCheck tells the tablet to run a health check.
|
||||
TabletActionRunHealthCheck = "RunHealthCheck"
|
||||
|
||||
// TabletActionIgnoreHealthError sets the regexp for health errors to ignore.
|
||||
TabletActionIgnoreHealthError = "IgnoreHealthError"
|
||||
|
||||
// TabletActionReloadSchema tells the tablet to reload its schema.
|
||||
TabletActionReloadSchema = "ReloadSchema"
|
||||
|
||||
// TabletActionPreflightSchema will check a schema change works
|
||||
TabletActionPreflightSchema = "PreflightSchema"
|
||||
|
||||
// TabletActionApplySchema will actually apply the schema change
|
||||
TabletActionApplySchema = "ApplySchema"
|
||||
|
||||
// TabletActionExecuteFetchAsDba uses the DBA connection to run queries.
|
||||
TabletActionExecuteFetchAsDba = "ExecuteFetchAsDba"
|
||||
|
||||
// TabletActionExecuteFetchAsApp uses the App connection to run queries.
|
||||
TabletActionExecuteFetchAsApp = "ExecuteFetchAsApp"
|
||||
|
||||
// TabletActionGetPermissions returns the mysql permissions set
|
||||
TabletActionGetPermissions = "GetPermissions"
|
||||
|
||||
// TabletActionGetSlaves returns the current set of mysql
|
||||
// replication slaves.
|
||||
TabletActionGetSlaves = "GetSlaves"
|
||||
|
||||
// TabletActionBackup takes a db backup and stores it into BackupStorage
|
||||
TabletActionBackup = "Backup"
|
||||
|
||||
//
|
||||
// Shard actions - involve all tablets in a shard.
|
||||
// These are just descriptive and used for locking / logging.
|
||||
//
|
||||
|
||||
// ShardActionReparent handles reparenting of the shard
|
||||
ShardActionReparent = "ReparentShard"
|
||||
|
||||
// ShardActionExternallyReparented locks the shard when it's
|
||||
// been reparented
|
||||
ShardActionExternallyReparented = "ShardExternallyReparented"
|
||||
|
||||
// ShardActionCheck takes a generic read lock for inexpensive
|
||||
// shard-wide actions.
|
||||
ShardActionCheck = "CheckShard"
|
||||
|
||||
// ShardActionSetServedTypes changes the ServedTypes inside a shard
|
||||
ShardActionSetServedTypes = "SetShardServedTypes"
|
||||
|
||||
// ShardActionMigrateServedTypes migratew served types from
|
||||
// one shard to another
|
||||
ShardActionMigrateServedTypes = "MigrateServedTypes"
|
||||
|
||||
// ShardActionUpdateShard updates the Shard object (Cells, ...)
|
||||
ShardActionUpdateShard = "UpdateShard"
|
||||
|
||||
//
|
||||
// Keyspace actions - require very high level locking for consistency.
|
||||
// These are just descriptive and used for locking / logging.
|
||||
//
|
||||
|
||||
// KeyspaceActionRebuild rebuilds the keyspace serving graph
|
||||
KeyspaceActionRebuild = "RebuildKeyspace"
|
||||
|
||||
// KeyspaceActionApplySchema applies a schema change on the keyspace
|
||||
KeyspaceActionApplySchema = "ApplySchemaKeyspace"
|
||||
|
||||
// KeyspaceActionSetShardingInfo updates the sharding info
|
||||
KeyspaceActionSetShardingInfo = "SetKeyspaceShardingInfo"
|
||||
|
||||
// KeyspaceActionMigrateServedFrom migrates ServedFrom to
|
||||
// another keyspace
|
||||
KeyspaceActionMigrateServedFrom = "MigrateServedFrom"
|
||||
|
||||
// KeyspaceActionSetServedFrom updates ServedFrom
|
||||
KeyspaceActionSetServedFrom = "SetKeyspaceServedFrom"
|
||||
|
||||
// KeyspaceActionCreateShard protects shard creation within the keyspace
|
||||
KeyspaceActionCreateShard = "KeyspaceCreateShard"
|
||||
|
||||
// all the valid states for an action
|
||||
|
||||
// ActionStateQueued is for an action that is going to be executed
|
||||
ActionStateQueued = ActionState("") // All actions are queued initially
|
||||
|
||||
// ActionStateRunning is for an action that is running
|
||||
ActionStateRunning = ActionState("Running") // Running inside vtaction process
|
||||
|
||||
// ActionStateFailed is for an action that has failed
|
||||
ActionStateFailed = ActionState("Failed") // Ended with a failure
|
||||
|
||||
// ActionStateDone is for an action that is done and successful
|
||||
ActionStateDone = ActionState("Done") // Ended with no failure
|
||||
)
|
||||
|
||||
// ActionState is the state an ActionNode
|
||||
type ActionState string
|
||||
|
||||
// ActionNode describes a long-running action on a tablet, or an action
|
||||
// on a shard or keyspace that locks it.
|
||||
type ActionNode struct {
|
||||
Action string
|
||||
ActionGuid string
|
||||
Error string
|
||||
State ActionState
|
||||
Pid int // only != 0 if State == ActionStateRunning
|
||||
|
||||
// do not serialize the next fields
|
||||
// path in topology server representing this action
|
||||
Path string `json:"-"`
|
||||
Args interface{} `json:"-"`
|
||||
Reply interface{} `json:"-"`
|
||||
}
|
||||
|
||||
// ToJSON returns a JSON representation of the object.
|
||||
func (n *ActionNode) ToJSON() (string, error) {
|
||||
data, err := json.MarshalIndent(n, "", " ")
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("cannot JSON-marshal node: %v", err)
|
||||
}
|
||||
result := string(data) + "\n"
|
||||
if n.Args == nil {
|
||||
result += "{}\n"
|
||||
} else {
|
||||
data, err := json.MarshalIndent(n.Args, "", " ")
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("cannot JSON-marshal node args: %v", err)
|
||||
}
|
||||
result += string(data) + "\n"
|
||||
}
|
||||
if n.Reply == nil {
|
||||
result += "{}\n"
|
||||
} else {
|
||||
data, err := json.MarshalIndent(n.Reply, "", " ")
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("cannot JSON-marshal node reply: %v", err)
|
||||
}
|
||||
result += string(data) + "\n"
|
||||
}
|
||||
return result, nil
|
||||
}
|
||||
|
||||
// SetGuid will set the ActionGuid field for the action node
|
||||
// and return the action node.
|
||||
func (n *ActionNode) SetGuid() *ActionNode {
|
||||
now := time.Now().Format(time.RFC3339)
|
||||
username := "unknown"
|
||||
if u, err := user.Current(); err == nil {
|
||||
username = u.Username
|
||||
}
|
||||
hostname := "unknown"
|
||||
if h, err := os.Hostname(); err == nil {
|
||||
hostname = h
|
||||
}
|
||||
n.ActionGuid = fmt.Sprintf("%v-%v-%v", now, username, hostname)
|
||||
return n
|
||||
}
|
|
@ -1,163 +0,0 @@
|
|||
// Copyright 2012, Google Inc. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package actionnode
|
||||
|
||||
import topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
|
||||
/*
|
||||
This file defines all the payload structures for the ActionNode objects.
|
||||
|
||||
The naming conventions are:
|
||||
- a structure used for Args only is suffixed by 'Args'.
|
||||
- a structure used for Reply only is suffixed by 'Reply'.
|
||||
- a structure used for both Args and Reply is suffixed by 'Data'.
|
||||
|
||||
Note it's OK to rename the structures as the type name is not saved in json.
|
||||
*/
|
||||
|
||||
// tablet action node structures
|
||||
|
||||
// SlaveWasRestartedArgs is the paylod for SlaveWasRestarted
|
||||
type SlaveWasRestartedArgs struct {
|
||||
Parent *topodatapb.TabletAlias
|
||||
}
|
||||
|
||||
// shard action node structures
|
||||
|
||||
// SetShardServedTypesArgs is the payload for SetShardServedTypes
|
||||
type SetShardServedTypesArgs struct {
|
||||
Cells []string
|
||||
ServedType topodatapb.TabletType
|
||||
}
|
||||
|
||||
// MigrateServedTypesArgs is the payload for MigrateServedTypes
|
||||
type MigrateServedTypesArgs struct {
|
||||
ServedType topodatapb.TabletType
|
||||
}
|
||||
|
||||
// keyspace action node structures
|
||||
|
||||
// ApplySchemaKeyspaceArgs is the payload for ApplySchemaKeyspace
|
||||
type ApplySchemaKeyspaceArgs struct {
|
||||
Change string
|
||||
}
|
||||
|
||||
// MigrateServedFromArgs is the payload for MigrateServedFrom
|
||||
type MigrateServedFromArgs struct {
|
||||
ServedType topodatapb.TabletType
|
||||
}
|
||||
|
||||
// methods to build the shard action nodes
|
||||
|
||||
// ReparentShardArgs is the payload for ReparentShard
|
||||
type ReparentShardArgs struct {
|
||||
Operation string
|
||||
MasterElectAlias *topodatapb.TabletAlias
|
||||
}
|
||||
|
||||
// ReparentShard returns an ActionNode
|
||||
func ReparentShard(operation string, masterElectAlias *topodatapb.TabletAlias) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionReparent,
|
||||
Args: &ReparentShardArgs{
|
||||
Operation: operation,
|
||||
MasterElectAlias: masterElectAlias,
|
||||
},
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// ShardExternallyReparented returns an ActionNode
|
||||
func ShardExternallyReparented(tabletAlias *topodatapb.TabletAlias) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionExternallyReparented,
|
||||
Args: &tabletAlias,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// CheckShard returns an ActionNode
|
||||
func CheckShard() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionCheck,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// SetShardServedTypes returns an ActionNode
|
||||
func SetShardServedTypes(cells []string, servedType topodatapb.TabletType) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionSetServedTypes,
|
||||
Args: &SetShardServedTypesArgs{
|
||||
Cells: cells,
|
||||
ServedType: servedType,
|
||||
},
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// MigrateServedTypes returns an ActionNode
|
||||
func MigrateServedTypes(servedType topodatapb.TabletType) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionMigrateServedTypes,
|
||||
Args: &MigrateServedTypesArgs{
|
||||
ServedType: servedType,
|
||||
},
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// UpdateShard returns an ActionNode
|
||||
func UpdateShard() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: ShardActionUpdateShard,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// methods to build the keyspace action nodes
|
||||
|
||||
// RebuildKeyspace returns an ActionNode
|
||||
func RebuildKeyspace() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionRebuild,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// SetKeyspaceShardingInfo returns an ActionNode
|
||||
func SetKeyspaceShardingInfo() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionSetShardingInfo,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// SetKeyspaceServedFrom returns an ActionNode
|
||||
func SetKeyspaceServedFrom() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionSetServedFrom,
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// ApplySchemaKeyspace returns an ActionNode
|
||||
func ApplySchemaKeyspace(change string) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionApplySchema,
|
||||
Args: &ApplySchemaKeyspaceArgs{
|
||||
Change: change,
|
||||
},
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// MigrateServedFrom returns an ActionNode
|
||||
func MigrateServedFrom(servedType topodatapb.TabletType) *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionMigrateServedFrom,
|
||||
Args: &MigrateServedFromArgs{
|
||||
ServedType: servedType,
|
||||
},
|
||||
}).SetGuid()
|
||||
}
|
||||
|
||||
// KeyspaceCreateShard returns an ActionNode to use to lock a keyspace
|
||||
// for shard creation
|
||||
func KeyspaceCreateShard() *ActionNode {
|
||||
return (&ActionNode{
|
||||
Action: KeyspaceActionCreateShard,
|
||||
}).SetGuid()
|
||||
}
|
|
@ -1,56 +0,0 @@
|
|||
package actionnode
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"testing"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
// These tests encode a slaveWasRestartedTestArgs (same as
|
||||
// SlaveWasRestartedArgs but with a few more arguments) and try to
|
||||
// decode it as a SlaveWasRestartedArgs, and vice versa
|
||||
|
||||
type slaveWasRestartedTestArgs struct {
|
||||
Parent *topodatapb.TabletAlias
|
||||
ExpectedMasterAddr string
|
||||
ExpectedMasterIPAddr string
|
||||
}
|
||||
|
||||
func TestMissingFieldsJson(t *testing.T) {
|
||||
swra := &slaveWasRestartedTestArgs{
|
||||
Parent: &topodatapb.TabletAlias{
|
||||
Uid: 1,
|
||||
Cell: "aa",
|
||||
},
|
||||
ExpectedMasterAddr: "a1",
|
||||
ExpectedMasterIPAddr: "i1",
|
||||
}
|
||||
data, err := json.MarshalIndent(swra, "", " ")
|
||||
if err != nil {
|
||||
t.Fatalf("cannot marshal: %v", err)
|
||||
}
|
||||
|
||||
output := &SlaveWasRestartedArgs{}
|
||||
if err = json.Unmarshal(data, output); err != nil {
|
||||
t.Errorf("Cannot re-decode struct without field: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestExtraFieldsJson(t *testing.T) {
|
||||
swra := &SlaveWasRestartedArgs{
|
||||
Parent: &topodatapb.TabletAlias{
|
||||
Uid: 1,
|
||||
Cell: "aa",
|
||||
},
|
||||
}
|
||||
data, err := json.MarshalIndent(swra, "", " ")
|
||||
if err != nil {
|
||||
t.Fatalf("cannot marshal: %v", err)
|
||||
}
|
||||
|
||||
output := &slaveWasRestartedTestArgs{}
|
||||
if err = json.Unmarshal(data, output); err != nil {
|
||||
t.Errorf("Cannot re-decode struct without field: %v", err)
|
||||
}
|
||||
}
|
|
@ -1,160 +0,0 @@
|
|||
// Copyright 2014, Google Inc. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package actionnode
|
||||
|
||||
// This file contains utility functions to be used with actionnode /
|
||||
// topology server.
|
||||
|
||||
import (
|
||||
"flag"
|
||||
"time"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"github.com/youtube/vitess/go/trace"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
var (
|
||||
// DefaultLockTimeout is a good value to use as a default for
|
||||
// locking a shard / keyspace.
|
||||
DefaultLockTimeout = 30 * time.Second
|
||||
|
||||
// LockTimeout is the command line flag that introduces a shorter
|
||||
// timeout for locking topology structures.
|
||||
LockTimeout = flag.Duration("lock_timeout", DefaultLockTimeout, "timeout for acquiring topology locks")
|
||||
)
|
||||
|
||||
// LockKeyspace will lock the keyspace in the topology server.
|
||||
// UnlockKeyspace should be called if this returns no error.
|
||||
func (n *ActionNode) LockKeyspace(ctx context.Context, ts topo.Server, keyspace string) (lockPath string, err error) {
|
||||
log.Infof("Locking keyspace %v for action %v", keyspace, n.Action)
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, *LockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.LockKeyspaceForAction")
|
||||
span.Annotate("action", n.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
defer span.Finish()
|
||||
|
||||
j, err := n.ToJSON()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return ts.LockKeyspaceForAction(ctx, keyspace, j)
|
||||
}
|
||||
|
||||
// UnlockKeyspace unlocks a previously locked keyspace.
|
||||
func (n *ActionNode) UnlockKeyspace(ctx context.Context, ts topo.Server, keyspace string, lockPath string, actionError error) error {
|
||||
// Detach from the parent timeout, but copy the trace span.
|
||||
// We need to still release the lock even if the parent context timed out.
|
||||
ctx = trace.CopySpan(context.TODO(), ctx)
|
||||
ctx, cancel := context.WithTimeout(ctx, DefaultLockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.UnlockKeyspaceForAction")
|
||||
span.Annotate("action", n.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
defer span.Finish()
|
||||
|
||||
// first update the actionNode
|
||||
if actionError != nil {
|
||||
log.Infof("Unlocking keyspace %v for action %v with error %v", keyspace, n.Action, actionError)
|
||||
n.Error = actionError.Error()
|
||||
n.State = ActionStateFailed
|
||||
} else {
|
||||
log.Infof("Unlocking keyspace %v for successful action %v", keyspace, n.Action)
|
||||
n.Error = ""
|
||||
n.State = ActionStateDone
|
||||
}
|
||||
j, err := n.ToJSON()
|
||||
if err != nil {
|
||||
if actionError != nil {
|
||||
// this will be masked
|
||||
log.Warningf("node.ToJSON failed: %v", err)
|
||||
return actionError
|
||||
}
|
||||
return err
|
||||
}
|
||||
err = ts.UnlockKeyspaceForAction(ctx, keyspace, lockPath, j)
|
||||
if actionError != nil {
|
||||
if err != nil {
|
||||
// this will be masked
|
||||
log.Warningf("UnlockKeyspaceForAction failed: %v", err)
|
||||
}
|
||||
return actionError
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// LockShard will lock the shard in the topology server.
|
||||
// UnlockShard should be called if this returns no error.
|
||||
func (n *ActionNode) LockShard(ctx context.Context, ts topo.Server, keyspace, shard string) (lockPath string, err error) {
|
||||
log.Infof("Locking shard %v/%v for action %v", keyspace, shard, n.Action)
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, *LockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.LockShardForAction")
|
||||
span.Annotate("action", n.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
span.Annotate("shard", shard)
|
||||
defer span.Finish()
|
||||
|
||||
j, err := n.ToJSON()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return ts.LockShardForAction(ctx, keyspace, shard, j)
|
||||
}
|
||||
|
||||
// UnlockShard unlocks a previously locked shard.
|
||||
func (n *ActionNode) UnlockShard(ctx context.Context, ts topo.Server, keyspace, shard string, lockPath string, actionError error) error {
|
||||
// Detach from the parent timeout, but copy the trace span.
|
||||
// We need to still release the lock even if the parent context timed out.
|
||||
ctx = trace.CopySpan(context.TODO(), ctx)
|
||||
ctx, cancel := context.WithTimeout(ctx, DefaultLockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.UnlockShardForAction")
|
||||
span.Annotate("action", n.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
span.Annotate("shard", shard)
|
||||
defer span.Finish()
|
||||
|
||||
// first update the actionNode
|
||||
if actionError != nil {
|
||||
log.Infof("Unlocking shard %v/%v for action %v with error %v", keyspace, shard, n.Action, actionError)
|
||||
n.Error = actionError.Error()
|
||||
n.State = ActionStateFailed
|
||||
} else {
|
||||
log.Infof("Unlocking shard %v/%v for successful action %v", keyspace, shard, n.Action)
|
||||
n.Error = ""
|
||||
n.State = ActionStateDone
|
||||
}
|
||||
j, err := n.ToJSON()
|
||||
if err != nil {
|
||||
if actionError != nil {
|
||||
// this will be masked
|
||||
log.Warningf("node.ToJSON failed: %v", err)
|
||||
return actionError
|
||||
}
|
||||
return err
|
||||
}
|
||||
err = ts.UnlockShardForAction(ctx, keyspace, shard, lockPath, j)
|
||||
if actionError != nil {
|
||||
if err != nil {
|
||||
// this will be masked
|
||||
log.Warningf("UnlockShardForAction failed: %v", err)
|
||||
}
|
||||
return actionError
|
||||
}
|
||||
return err
|
||||
}
|
|
@ -21,7 +21,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
|
||||
querypb "github.com/youtube/vitess/go/vt/proto/query"
|
||||
|
@ -1039,30 +1038,28 @@ func agentRPCTestSetMasterPanic(ctx context.Context, t *testing.T, client tmclie
|
|||
expectRPCWrapLockActionPanic(t, err)
|
||||
}
|
||||
|
||||
var testSlaveWasRestartedArgs = &actionnode.SlaveWasRestartedArgs{
|
||||
Parent: &topodatapb.TabletAlias{
|
||||
Cell: "prison",
|
||||
Uid: 42,
|
||||
},
|
||||
var testSlaveWasRestartedParent = &topodatapb.TabletAlias{
|
||||
Cell: "prison",
|
||||
Uid: 42,
|
||||
}
|
||||
var testSlaveWasRestartedCalled = false
|
||||
|
||||
func (fra *fakeRPCAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error {
|
||||
func (fra *fakeRPCAgent) SlaveWasRestarted(ctx context.Context, parent *topodatapb.TabletAlias) error {
|
||||
if fra.panics {
|
||||
panic(fmt.Errorf("test-triggered panic"))
|
||||
}
|
||||
compare(fra.t, "SlaveWasRestarted swrd", swrd, testSlaveWasRestartedArgs)
|
||||
compare(fra.t, "SlaveWasRestarted parent", parent, testSlaveWasRestartedParent)
|
||||
testSlaveWasRestartedCalled = true
|
||||
return nil
|
||||
}
|
||||
|
||||
func agentRPCTestSlaveWasRestarted(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
|
||||
err := client.SlaveWasRestarted(ctx, tablet, testSlaveWasRestartedArgs)
|
||||
err := client.SlaveWasRestarted(ctx, tablet, testSlaveWasRestartedParent)
|
||||
compareError(t, "SlaveWasRestarted", err, true, testSlaveWasRestartedCalled)
|
||||
}
|
||||
|
||||
func agentRPCTestSlaveWasRestartedPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
|
||||
err := client.SlaveWasRestarted(ctx, tablet, testSlaveWasRestartedArgs)
|
||||
err := client.SlaveWasRestarted(ctx, tablet, testSlaveWasRestartedParent)
|
||||
expectRPCWrapLockActionPanic(t, err)
|
||||
}
|
||||
|
||||
|
@ -1143,7 +1140,7 @@ func agentRPCTestBackupPanic(ctx context.Context, t *testing.T, client tmclient.
|
|||
//
|
||||
|
||||
// RPCWrap is part of the RPCAgent interface
|
||||
func (fra *fakeRPCAgent) RPCWrap(ctx context.Context, name string, args, reply interface{}, f func() error) (err error) {
|
||||
func (fra *fakeRPCAgent) RPCWrap(ctx context.Context, name tabletmanager.TabletAction, args, reply interface{}, f func() error) (err error) {
|
||||
defer func() {
|
||||
if x := recover(); x != nil {
|
||||
err = fmt.Errorf("RPCWrap caught panic during %v", name)
|
||||
|
@ -1153,7 +1150,7 @@ func (fra *fakeRPCAgent) RPCWrap(ctx context.Context, name string, args, reply i
|
|||
}
|
||||
|
||||
// RPCWrapLock is part of the RPCAgent interface
|
||||
func (fra *fakeRPCAgent) RPCWrapLock(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) (err error) {
|
||||
func (fra *fakeRPCAgent) RPCWrapLock(ctx context.Context, name tabletmanager.TabletAction, args, reply interface{}, verbose bool, f func() error) (err error) {
|
||||
defer func() {
|
||||
if x := recover(); x != nil {
|
||||
err = fmt.Errorf("RPCWrapLock caught panic during %v", name)
|
||||
|
@ -1163,7 +1160,7 @@ func (fra *fakeRPCAgent) RPCWrapLock(ctx context.Context, name string, args, rep
|
|||
}
|
||||
|
||||
// RPCWrapLockAction is part of the RPCAgent interface
|
||||
func (fra *fakeRPCAgent) RPCWrapLockAction(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) (err error) {
|
||||
func (fra *fakeRPCAgent) RPCWrapLockAction(ctx context.Context, name tabletmanager.TabletAction, args, reply interface{}, verbose bool, f func() error) (err error) {
|
||||
defer func() {
|
||||
if x := recover(); x != nil {
|
||||
err = fmt.Errorf("RPCWrapLockAction caught panic during %v", name)
|
||||
|
|
|
@ -392,18 +392,21 @@ func TestBinlogPlayerMapHorizontalSplit(t *testing.T) {
|
|||
}
|
||||
|
||||
// now add the source in shard
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "ks",
|
||||
Shard: "-80",
|
||||
KeyRange: &topodatapb.KeyRange{
|
||||
End: []byte{0x80},
|
||||
si, err = ts.UpdateShardFields(ctx, si.Keyspace(), si.ShardName(), func(si *topo.ShardInfo) error {
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "ks",
|
||||
Shard: "-80",
|
||||
KeyRange: &topodatapb.KeyRange{
|
||||
End: []byte{0x80},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// now we have a source, adding players
|
||||
|
@ -557,22 +560,21 @@ func TestBinlogPlayerMapHorizontalSplitStopStartUntil(t *testing.T) {
|
|||
Shard: "40-60",
|
||||
}
|
||||
|
||||
si, err := ts.GetShard(ctx, "ks", "40-60")
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "ks",
|
||||
Shard: "-80",
|
||||
KeyRange: &topodatapb.KeyRange{
|
||||
End: []byte{0x80},
|
||||
si, err := ts.UpdateShardFields(ctx, "ks", "40-60", func(si *topo.ShardInfo) error {
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "ks",
|
||||
Shard: "-80",
|
||||
KeyRange: &topodatapb.KeyRange{
|
||||
End: []byte{0x80},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// now we have a source, adding players
|
||||
|
@ -769,23 +771,22 @@ func TestBinlogPlayerMapVerticalSplit(t *testing.T) {
|
|||
Shard: "0",
|
||||
}
|
||||
|
||||
si, err := ts.GetShard(ctx, "destination", "0")
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "source",
|
||||
Shard: "0",
|
||||
Tables: []string{
|
||||
"table1",
|
||||
"funtables_*",
|
||||
si, err := ts.UpdateShardFields(ctx, "destination", "0", func(si *topo.ShardInfo) error {
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "source",
|
||||
Shard: "0",
|
||||
Tables: []string{
|
||||
"table1",
|
||||
"funtables_*",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// now we have a source, adding players
|
||||
|
|
|
@ -18,7 +18,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/hook"
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
|
||||
logutilpb "github.com/youtube/vitess/go/vt/proto/logutil"
|
||||
|
@ -254,7 +253,7 @@ func (client *FakeTabletManagerClient) SetMaster(ctx context.Context, tablet *to
|
|||
}
|
||||
|
||||
// SlaveWasRestarted is part of the tmclient.TabletManagerClient interface.
|
||||
func (client *FakeTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, args *actionnode.SlaveWasRestartedArgs) error {
|
||||
func (client *FakeTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/servenv/grpcutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"golang.org/x/net/context"
|
||||
|
@ -603,14 +602,14 @@ func (client *Client) SetMaster(ctx context.Context, tablet *topodatapb.Tablet,
|
|||
}
|
||||
|
||||
// SlaveWasRestarted is part of the tmclient.TabletManagerClient interface.
|
||||
func (client *Client) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, args *actionnode.SlaveWasRestartedArgs) error {
|
||||
func (client *Client) SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias) error {
|
||||
cc, c, err := client.dial(ctx, tablet)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer cc.Close()
|
||||
_, err = c.SlaveWasRestarted(ctx, &tabletmanagerdatapb.SlaveWasRestartedRequest{
|
||||
Parent: args.Parent,
|
||||
Parent: parent,
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/servenv"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/vterrors"
|
||||
|
||||
logutilpb "github.com/youtube/vitess/go/vt/proto/logutil"
|
||||
|
@ -34,7 +33,7 @@ type server struct {
|
|||
func (s *server) Ping(ctx context.Context, request *tabletmanagerdatapb.PingRequest) (*tabletmanagerdatapb.PingResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.PingResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionPing, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionPing, request, response, func() error {
|
||||
response.Payload = s.agent.Ping(ctx, request.Payload)
|
||||
return nil
|
||||
})
|
||||
|
@ -43,7 +42,7 @@ func (s *server) Ping(ctx context.Context, request *tabletmanagerdatapb.PingRequ
|
|||
func (s *server) Sleep(ctx context.Context, request *tabletmanagerdatapb.SleepRequest) (*tabletmanagerdatapb.SleepResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SleepResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSleep, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSleep, request, response, true, func() error {
|
||||
s.agent.Sleep(ctx, time.Duration(request.Duration))
|
||||
return nil
|
||||
})
|
||||
|
@ -52,7 +51,7 @@ func (s *server) Sleep(ctx context.Context, request *tabletmanagerdatapb.SleepRe
|
|||
func (s *server) ExecuteHook(ctx context.Context, request *tabletmanagerdatapb.ExecuteHookRequest) (*tabletmanagerdatapb.ExecuteHookResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ExecuteHookResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionExecuteHook, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionExecuteHook, request, response, true, func() error {
|
||||
hr := s.agent.ExecuteHook(ctx, &hook.Hook{
|
||||
Name: request.Name,
|
||||
Parameters: request.Parameters,
|
||||
|
@ -68,7 +67,7 @@ func (s *server) ExecuteHook(ctx context.Context, request *tabletmanagerdatapb.E
|
|||
func (s *server) GetSchema(ctx context.Context, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.GetSchemaResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.GetSchemaResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetSchema, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionGetSchema, request, response, func() error {
|
||||
sd, err := s.agent.GetSchema(ctx, request.Tables, request.ExcludeTables, request.IncludeViews)
|
||||
if err == nil {
|
||||
response.SchemaDefinition = sd
|
||||
|
@ -80,7 +79,7 @@ func (s *server) GetSchema(ctx context.Context, request *tabletmanagerdatapb.Get
|
|||
func (s *server) GetPermissions(ctx context.Context, request *tabletmanagerdatapb.GetPermissionsRequest) (*tabletmanagerdatapb.GetPermissionsResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.GetPermissionsResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetPermissions, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionGetPermissions, request, response, func() error {
|
||||
p, err := s.agent.GetPermissions(ctx)
|
||||
if err == nil {
|
||||
response.Permissions = p
|
||||
|
@ -96,7 +95,7 @@ func (s *server) GetPermissions(ctx context.Context, request *tabletmanagerdatap
|
|||
func (s *server) SetReadOnly(ctx context.Context, request *tabletmanagerdatapb.SetReadOnlyRequest) (*tabletmanagerdatapb.SetReadOnlyResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SetReadOnlyResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetReadOnly, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSetReadOnly, request, response, true, func() error {
|
||||
return s.agent.SetReadOnly(ctx, true)
|
||||
})
|
||||
}
|
||||
|
@ -104,7 +103,7 @@ func (s *server) SetReadOnly(ctx context.Context, request *tabletmanagerdatapb.S
|
|||
func (s *server) SetReadWrite(ctx context.Context, request *tabletmanagerdatapb.SetReadWriteRequest) (*tabletmanagerdatapb.SetReadWriteResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SetReadWriteResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetReadWrite, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSetReadWrite, request, response, true, func() error {
|
||||
return s.agent.SetReadOnly(ctx, false)
|
||||
})
|
||||
}
|
||||
|
@ -112,7 +111,7 @@ func (s *server) SetReadWrite(ctx context.Context, request *tabletmanagerdatapb.
|
|||
func (s *server) ChangeType(ctx context.Context, request *tabletmanagerdatapb.ChangeTypeRequest) (*tabletmanagerdatapb.ChangeTypeResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ChangeTypeResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionChangeType, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionChangeType, request, response, true, func() error {
|
||||
return s.agent.ChangeType(ctx, request.TabletType)
|
||||
})
|
||||
}
|
||||
|
@ -120,7 +119,7 @@ func (s *server) ChangeType(ctx context.Context, request *tabletmanagerdatapb.Ch
|
|||
func (s *server) RefreshState(ctx context.Context, request *tabletmanagerdatapb.RefreshStateRequest) (*tabletmanagerdatapb.RefreshStateResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.RefreshStateResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionRefreshState, request, response, true, func() error {
|
||||
s.agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -129,7 +128,7 @@ func (s *server) RefreshState(ctx context.Context, request *tabletmanagerdatapb.
|
|||
func (s *server) RunHealthCheck(ctx context.Context, request *tabletmanagerdatapb.RunHealthCheckRequest) (*tabletmanagerdatapb.RunHealthCheckResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.RunHealthCheckResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionRunHealthCheck, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionRunHealthCheck, request, response, func() error {
|
||||
s.agent.RunHealthCheck(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -138,7 +137,7 @@ func (s *server) RunHealthCheck(ctx context.Context, request *tabletmanagerdatap
|
|||
func (s *server) IgnoreHealthError(ctx context.Context, request *tabletmanagerdatapb.IgnoreHealthErrorRequest) (*tabletmanagerdatapb.IgnoreHealthErrorResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.IgnoreHealthErrorResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionIgnoreHealthError, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionIgnoreHealthError, request, response, func() error {
|
||||
return s.agent.IgnoreHealthError(ctx, request.Pattern)
|
||||
})
|
||||
}
|
||||
|
@ -146,7 +145,7 @@ func (s *server) IgnoreHealthError(ctx context.Context, request *tabletmanagerda
|
|||
func (s *server) ReloadSchema(ctx context.Context, request *tabletmanagerdatapb.ReloadSchemaRequest) (*tabletmanagerdatapb.ReloadSchemaResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ReloadSchemaResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionReloadSchema, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionReloadSchema, request, response, true, func() error {
|
||||
s.agent.ReloadSchema(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -155,7 +154,7 @@ func (s *server) ReloadSchema(ctx context.Context, request *tabletmanagerdatapb.
|
|||
func (s *server) PreflightSchema(ctx context.Context, request *tabletmanagerdatapb.PreflightSchemaRequest) (*tabletmanagerdatapb.PreflightSchemaResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.PreflightSchemaResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPreflightSchema, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionPreflightSchema, request, response, true, func() error {
|
||||
scr, err := s.agent.PreflightSchema(ctx, request.Change)
|
||||
if err == nil {
|
||||
response.BeforeSchema = scr.BeforeSchema
|
||||
|
@ -168,7 +167,7 @@ func (s *server) PreflightSchema(ctx context.Context, request *tabletmanagerdata
|
|||
func (s *server) ApplySchema(ctx context.Context, request *tabletmanagerdatapb.ApplySchemaRequest) (*tabletmanagerdatapb.ApplySchemaResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ApplySchemaResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionApplySchema, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionApplySchema, request, response, true, func() error {
|
||||
scr, err := s.agent.ApplySchema(ctx, &tmutils.SchemaChange{
|
||||
SQL: request.Sql,
|
||||
Force: request.Force,
|
||||
|
@ -187,7 +186,7 @@ func (s *server) ApplySchema(ctx context.Context, request *tabletmanagerdatapb.A
|
|||
func (s *server) ExecuteFetchAsDba(ctx context.Context, request *tabletmanagerdatapb.ExecuteFetchAsDbaRequest) (*tabletmanagerdatapb.ExecuteFetchAsDbaResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ExecuteFetchAsDbaResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionExecuteFetchAsDba, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionExecuteFetchAsDba, request, response, func() error {
|
||||
qr, err := s.agent.ExecuteFetchAsDba(ctx, request.Query, request.DbName, int(request.MaxRows), request.DisableBinlogs, request.ReloadSchema)
|
||||
if err != nil {
|
||||
return vterrors.ToGRPCError(err)
|
||||
|
@ -200,7 +199,7 @@ func (s *server) ExecuteFetchAsDba(ctx context.Context, request *tabletmanagerda
|
|||
func (s *server) ExecuteFetchAsApp(ctx context.Context, request *tabletmanagerdatapb.ExecuteFetchAsAppRequest) (*tabletmanagerdatapb.ExecuteFetchAsAppResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ExecuteFetchAsAppResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionExecuteFetchAsApp, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionExecuteFetchAsApp, request, response, func() error {
|
||||
qr, err := s.agent.ExecuteFetchAsApp(ctx, request.Query, int(request.MaxRows))
|
||||
if err != nil {
|
||||
return vterrors.ToGRPCError(err)
|
||||
|
@ -217,7 +216,7 @@ func (s *server) ExecuteFetchAsApp(ctx context.Context, request *tabletmanagerda
|
|||
func (s *server) SlaveStatus(ctx context.Context, request *tabletmanagerdatapb.SlaveStatusRequest) (*tabletmanagerdatapb.SlaveStatusResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SlaveStatusResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionSlaveStatus, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionSlaveStatus, request, response, func() error {
|
||||
status, err := s.agent.SlaveStatus(ctx)
|
||||
if err == nil {
|
||||
response.Status = status
|
||||
|
@ -229,7 +228,7 @@ func (s *server) SlaveStatus(ctx context.Context, request *tabletmanagerdatapb.S
|
|||
func (s *server) MasterPosition(ctx context.Context, request *tabletmanagerdatapb.MasterPositionRequest) (*tabletmanagerdatapb.MasterPositionResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.MasterPositionResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionMasterPosition, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionMasterPosition, request, response, func() error {
|
||||
position, err := s.agent.MasterPosition(ctx)
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -241,7 +240,7 @@ func (s *server) MasterPosition(ctx context.Context, request *tabletmanagerdatap
|
|||
func (s *server) StopSlave(ctx context.Context, request *tabletmanagerdatapb.StopSlaveRequest) (*tabletmanagerdatapb.StopSlaveResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StopSlaveResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopSlave, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionStopSlave, request, response, true, func() error {
|
||||
return s.agent.StopSlave(ctx)
|
||||
})
|
||||
}
|
||||
|
@ -249,7 +248,7 @@ func (s *server) StopSlave(ctx context.Context, request *tabletmanagerdatapb.Sto
|
|||
func (s *server) StopSlaveMinimum(ctx context.Context, request *tabletmanagerdatapb.StopSlaveMinimumRequest) (*tabletmanagerdatapb.StopSlaveMinimumResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StopSlaveMinimumResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopSlaveMinimum, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionStopSlaveMinimum, request, response, true, func() error {
|
||||
position, err := s.agent.StopSlaveMinimum(ctx, request.Position, time.Duration(request.WaitTimeout))
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -261,7 +260,7 @@ func (s *server) StopSlaveMinimum(ctx context.Context, request *tabletmanagerdat
|
|||
func (s *server) StartSlave(ctx context.Context, request *tabletmanagerdatapb.StartSlaveRequest) (*tabletmanagerdatapb.StartSlaveResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StartSlaveResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStartSlave, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionStartSlave, request, response, true, func() error {
|
||||
return s.agent.StartSlave(ctx)
|
||||
})
|
||||
}
|
||||
|
@ -269,7 +268,7 @@ func (s *server) StartSlave(ctx context.Context, request *tabletmanagerdatapb.St
|
|||
func (s *server) TabletExternallyReparented(ctx context.Context, request *tabletmanagerdatapb.TabletExternallyReparentedRequest) (*tabletmanagerdatapb.TabletExternallyReparentedResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.TabletExternallyReparentedResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionExternallyReparented, request, response, false, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionExternallyReparented, request, response, false, func() error {
|
||||
return s.agent.TabletExternallyReparented(ctx, request.ExternalId)
|
||||
})
|
||||
}
|
||||
|
@ -281,7 +280,7 @@ func (s *server) TabletExternallyElected(ctx context.Context, request *tabletman
|
|||
func (s *server) GetSlaves(ctx context.Context, request *tabletmanagerdatapb.GetSlavesRequest) (*tabletmanagerdatapb.GetSlavesResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.GetSlavesResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionGetSlaves, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionGetSlaves, request, response, func() error {
|
||||
addrs, err := s.agent.GetSlaves(ctx)
|
||||
if err == nil {
|
||||
response.Addrs = addrs
|
||||
|
@ -293,7 +292,7 @@ func (s *server) GetSlaves(ctx context.Context, request *tabletmanagerdatapb.Get
|
|||
func (s *server) WaitBlpPosition(ctx context.Context, request *tabletmanagerdatapb.WaitBlpPositionRequest) (*tabletmanagerdatapb.WaitBlpPositionResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.WaitBlpPositionResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionWaitBLPPosition, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionWaitBLPPosition, request, response, true, func() error {
|
||||
return s.agent.WaitBlpPosition(ctx, request.BlpPosition, time.Duration(request.WaitTimeout))
|
||||
})
|
||||
}
|
||||
|
@ -301,7 +300,7 @@ func (s *server) WaitBlpPosition(ctx context.Context, request *tabletmanagerdata
|
|||
func (s *server) StopBlp(ctx context.Context, request *tabletmanagerdatapb.StopBlpRequest) (*tabletmanagerdatapb.StopBlpResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StopBlpResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStopBLP, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionStopBLP, request, response, true, func() error {
|
||||
positions, err := s.agent.StopBlp(ctx)
|
||||
if err == nil {
|
||||
response.BlpPositions = positions
|
||||
|
@ -313,7 +312,7 @@ func (s *server) StopBlp(ctx context.Context, request *tabletmanagerdatapb.StopB
|
|||
func (s *server) StartBlp(ctx context.Context, request *tabletmanagerdatapb.StartBlpRequest) (*tabletmanagerdatapb.StartBlpResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StartBlpResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionStartBLP, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionStartBLP, request, response, true, func() error {
|
||||
return s.agent.StartBlp(ctx)
|
||||
})
|
||||
}
|
||||
|
@ -321,7 +320,7 @@ func (s *server) StartBlp(ctx context.Context, request *tabletmanagerdatapb.Star
|
|||
func (s *server) RunBlpUntil(ctx context.Context, request *tabletmanagerdatapb.RunBlpUntilRequest) (*tabletmanagerdatapb.RunBlpUntilResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.RunBlpUntilResponse{}
|
||||
return response, s.agent.RPCWrapLock(ctx, actionnode.TabletActionRunBLPUntil, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLock(ctx, tabletmanager.TabletActionRunBLPUntil, request, response, true, func() error {
|
||||
position, err := s.agent.RunBlpUntil(ctx, request.BlpPositions, time.Duration(request.WaitTimeout))
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -337,7 +336,7 @@ func (s *server) RunBlpUntil(ctx context.Context, request *tabletmanagerdatapb.R
|
|||
func (s *server) ResetReplication(ctx context.Context, request *tabletmanagerdatapb.ResetReplicationRequest) (*tabletmanagerdatapb.ResetReplicationResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.ResetReplicationResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionResetReplication, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionResetReplication, request, response, true, func() error {
|
||||
return s.agent.ResetReplication(ctx)
|
||||
})
|
||||
}
|
||||
|
@ -345,7 +344,7 @@ func (s *server) ResetReplication(ctx context.Context, request *tabletmanagerdat
|
|||
func (s *server) InitMaster(ctx context.Context, request *tabletmanagerdatapb.InitMasterRequest) (*tabletmanagerdatapb.InitMasterResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.InitMasterResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionInitMaster, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionInitMaster, request, response, true, func() error {
|
||||
position, err := s.agent.InitMaster(ctx)
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -357,7 +356,7 @@ func (s *server) InitMaster(ctx context.Context, request *tabletmanagerdatapb.In
|
|||
func (s *server) PopulateReparentJournal(ctx context.Context, request *tabletmanagerdatapb.PopulateReparentJournalRequest) (*tabletmanagerdatapb.PopulateReparentJournalResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.PopulateReparentJournalResponse{}
|
||||
return response, s.agent.RPCWrap(ctx, actionnode.TabletActionPopulateReparentJournal, request, response, func() error {
|
||||
return response, s.agent.RPCWrap(ctx, tabletmanager.TabletActionPopulateReparentJournal, request, response, func() error {
|
||||
return s.agent.PopulateReparentJournal(ctx, request.TimeCreatedNs, request.ActionName, request.MasterAlias, request.ReplicationPosition)
|
||||
})
|
||||
}
|
||||
|
@ -365,7 +364,7 @@ func (s *server) PopulateReparentJournal(ctx context.Context, request *tabletman
|
|||
func (s *server) InitSlave(ctx context.Context, request *tabletmanagerdatapb.InitSlaveRequest) (*tabletmanagerdatapb.InitSlaveResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.InitSlaveResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionInitSlave, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionInitSlave, request, response, true, func() error {
|
||||
return s.agent.InitSlave(ctx, request.Parent, request.ReplicationPosition, request.TimeCreatedNs)
|
||||
})
|
||||
}
|
||||
|
@ -373,7 +372,7 @@ func (s *server) InitSlave(ctx context.Context, request *tabletmanagerdatapb.Ini
|
|||
func (s *server) DemoteMaster(ctx context.Context, request *tabletmanagerdatapb.DemoteMasterRequest) (*tabletmanagerdatapb.DemoteMasterResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.DemoteMasterResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionDemoteMaster, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionDemoteMaster, request, response, true, func() error {
|
||||
position, err := s.agent.DemoteMaster(ctx)
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -385,7 +384,7 @@ func (s *server) DemoteMaster(ctx context.Context, request *tabletmanagerdatapb.
|
|||
func (s *server) PromoteSlaveWhenCaughtUp(ctx context.Context, request *tabletmanagerdatapb.PromoteSlaveWhenCaughtUpRequest) (*tabletmanagerdatapb.PromoteSlaveWhenCaughtUpResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.PromoteSlaveWhenCaughtUpResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPromoteSlaveWhenCaughtUp, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionPromoteSlaveWhenCaughtUp, request, response, true, func() error {
|
||||
position, err := s.agent.PromoteSlaveWhenCaughtUp(ctx, request.Position)
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -397,7 +396,7 @@ func (s *server) PromoteSlaveWhenCaughtUp(ctx context.Context, request *tabletma
|
|||
func (s *server) SlaveWasPromoted(ctx context.Context, request *tabletmanagerdatapb.SlaveWasPromotedRequest) (*tabletmanagerdatapb.SlaveWasPromotedResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SlaveWasPromotedResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSlaveWasPromoted, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSlaveWasPromoted, request, response, true, func() error {
|
||||
return s.agent.SlaveWasPromoted(ctx)
|
||||
})
|
||||
}
|
||||
|
@ -405,7 +404,7 @@ func (s *server) SlaveWasPromoted(ctx context.Context, request *tabletmanagerdat
|
|||
func (s *server) SetMaster(ctx context.Context, request *tabletmanagerdatapb.SetMasterRequest) (*tabletmanagerdatapb.SetMasterResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.SetMasterResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionSetMaster, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSetMaster, request, response, true, func() error {
|
||||
return s.agent.SetMaster(ctx, request.Parent, request.TimeCreatedNs, request.ForceStartSlave)
|
||||
})
|
||||
}
|
||||
|
@ -413,17 +412,15 @@ func (s *server) SetMaster(ctx context.Context, request *tabletmanagerdatapb.Set
|
|||
func (s *server) SlaveWasRestarted(ctx context.Context, request *tabletmanagerdatapb.SlaveWasRestartedRequest) (*tabletmanagerdatapb.SlaveWasRestartedResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
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,
|
||||
})
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionSlaveWasRestarted, request, response, true, func() error {
|
||||
return s.agent.SlaveWasRestarted(ctx, request.Parent)
|
||||
})
|
||||
}
|
||||
|
||||
func (s *server) StopReplicationAndGetStatus(ctx context.Context, request *tabletmanagerdatapb.StopReplicationAndGetStatusRequest) (*tabletmanagerdatapb.StopReplicationAndGetStatusResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.StopReplicationAndGetStatusResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionStopReplicationAndGetStatus, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionStopReplicationAndGetStatus, request, response, true, func() error {
|
||||
status, err := s.agent.StopReplicationAndGetStatus(ctx)
|
||||
if err == nil {
|
||||
response.Status = status
|
||||
|
@ -435,7 +432,7 @@ func (s *server) StopReplicationAndGetStatus(ctx context.Context, request *table
|
|||
func (s *server) PromoteSlave(ctx context.Context, request *tabletmanagerdatapb.PromoteSlaveRequest) (*tabletmanagerdatapb.PromoteSlaveResponse, error) {
|
||||
ctx = callinfo.GRPCCallInfo(ctx)
|
||||
response := &tabletmanagerdatapb.PromoteSlaveResponse{}
|
||||
return response, s.agent.RPCWrapLockAction(ctx, actionnode.TabletActionPromoteSlave, request, response, true, func() error {
|
||||
return response, s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionPromoteSlave, request, response, true, func() error {
|
||||
position, err := s.agent.PromoteSlave(ctx)
|
||||
if err == nil {
|
||||
response.Position = position
|
||||
|
@ -446,7 +443,7 @@ func (s *server) PromoteSlave(ctx context.Context, request *tabletmanagerdatapb.
|
|||
|
||||
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 {
|
||||
return s.agent.RPCWrapLockAction(ctx, tabletmanager.TabletActionBackup, request, nil, true, func() error {
|
||||
// create a logger, send the result back to the caller
|
||||
logger := logutil.NewCallbackLogger(func(e *logutilpb.Event) {
|
||||
// If the client disconnects, we will just fail
|
||||
|
|
|
@ -18,9 +18,9 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/binlog/binlogplayer"
|
||||
"github.com/youtube/vitess/go/vt/health"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver"
|
||||
"github.com/youtube/vitess/go/vt/tabletserver/tabletservermock"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/zktopo/zktestserver"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
|
@ -516,22 +516,21 @@ func TestTabletControl(t *testing.T) {
|
|||
}
|
||||
|
||||
// now update the shard
|
||||
si, err := agent.TopoServer.GetShard(ctx, "test_keyspace", "0")
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.TabletControls = []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_REPLICA,
|
||||
DisableQueryService: true,
|
||||
},
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.TabletControls = []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_REPLICA,
|
||||
DisableQueryService: true,
|
||||
},
|
||||
}
|
||||
if err := agent.TopoServer.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// now refresh the tablet state, as the resharding process would do
|
||||
agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RPCWrapLockAction(ctx, TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -641,19 +640,18 @@ func TestTabletControl(t *testing.T) {
|
|||
t.Errorf("invalid tabletserver target: got = %v, want = %v", got, topodatapb.TabletType_REPLICA)
|
||||
}
|
||||
|
||||
// now clear TabletControl, run health check, make sure we go back healthy
|
||||
// and serving.
|
||||
si, err = agent.TopoServer.GetShard(ctx, "test_keyspace", "0")
|
||||
// now clear TabletControl, run health check, make sure we go
|
||||
// back healthy and serving.
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.TabletControls = nil
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.TabletControls = nil
|
||||
if err := agent.TopoServer.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// now refresh the tablet state, as the resharding process would do
|
||||
agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RPCWrapLockAction(ctx, TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -711,7 +709,7 @@ func TestStateChangeImmediateHealthBroadcast(t *testing.T) {
|
|||
|
||||
// Run TER to turn us into a proper master, wait for it to finish.
|
||||
agent.HealthReporter.(*fakeHealthCheck).reportReplicationDelay = 19 * time.Second
|
||||
if err := agent.RPCWrapLock(ctx, actionnode.TabletActionExternallyReparented, "", "", false, func() error {
|
||||
if err := agent.RPCWrapLock(ctx, TabletActionExternallyReparented, "", "", false, func() error {
|
||||
return agent.TabletExternallyReparented(ctx, "unused_id")
|
||||
}); err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -761,25 +759,25 @@ func TestStateChangeImmediateHealthBroadcast(t *testing.T) {
|
|||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Simulate a vertical split resharding where we set SourceShards in the topo
|
||||
// and enable filtered replication.
|
||||
si, err := agent.TopoServer.GetShard(ctx, "test_keyspace", "0")
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "source_keyspace",
|
||||
Shard: "0",
|
||||
Tables: []string{
|
||||
"table1",
|
||||
// Simulate a vertical split resharding where we set
|
||||
// SourceShards in the topo and enable filtered replication.
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.SourceShards = []*topodatapb.Shard_SourceShard{
|
||||
{
|
||||
Uid: 1,
|
||||
Keyspace: "source_keyspace",
|
||||
Shard: "0",
|
||||
Tables: []string{
|
||||
"table1",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
if err := agent.TopoServer.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// Mock out the BinlogPlayer client. Tell the BinlogPlayer not to start.
|
||||
vtClientMock := binlogplayer.NewVtClientMock()
|
||||
vtClientMock.AddResult(&sqltypes.Result{
|
||||
|
@ -798,7 +796,7 @@ func TestStateChangeImmediateHealthBroadcast(t *testing.T) {
|
|||
// Refresh the tablet state, as vtworker would do.
|
||||
// Since we change the QueryService state, we'll also trigger a health broadcast.
|
||||
agent.HealthReporter.(*fakeHealthCheck).reportReplicationDelay = 21 * time.Second
|
||||
agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RPCWrapLockAction(ctx, TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
@ -847,19 +845,19 @@ func TestStateChangeImmediateHealthBroadcast(t *testing.T) {
|
|||
// NOTE: No state change here since nothing has changed.
|
||||
|
||||
// Simulate migration to destination master i.e. remove SourceShards.
|
||||
si, err = agent.TopoServer.GetShard(ctx, "test_keyspace", "0")
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.SourceShards = nil
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.SourceShards = nil
|
||||
if err = agent.TopoServer.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// Refresh the tablet state, as vtctl MigrateServedFrom would do.
|
||||
// This should also trigger a health broadcast since the QueryService state
|
||||
// changes from NOT_SERVING to SERVING.
|
||||
agent.HealthReporter.(*fakeHealthCheck).reportReplicationDelay = 23 * time.Second
|
||||
agent.RPCWrapLockAction(ctx, actionnode.TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RPCWrapLockAction(ctx, TabletActionRefreshState, "", "", true, func() error {
|
||||
agent.RefreshState(ctx)
|
||||
return nil
|
||||
})
|
||||
|
|
|
@ -17,7 +17,6 @@ import (
|
|||
"github.com/youtube/vitess/go/netutil"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/topotools"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
|
@ -101,7 +100,7 @@ func (agent *ActionAgent) InitTablet(port, gRPCPort int32) error {
|
|||
log.Infof("Reading shard record %v/%v", *initKeyspace, shard)
|
||||
|
||||
// read the shard, create it if necessary
|
||||
si, err := topotools.GetOrCreateShard(ctx, agent.TopoServer, *initKeyspace, shard)
|
||||
si, err := agent.TopoServer.GetOrCreateShard(ctx, *initKeyspace, shard)
|
||||
if err != nil {
|
||||
return fmt.Errorf("InitTablet cannot GetOrCreateShard shard: %v", err)
|
||||
}
|
||||
|
@ -129,12 +128,12 @@ func (agent *ActionAgent) InitTablet(port, gRPCPort int32) error {
|
|||
|
||||
// See if we need to add the tablet's cell to the shard's cell list.
|
||||
if !si.HasCell(agent.TabletAlias.Cell) {
|
||||
si, err = agent.TopoServer.UpdateShardFields(ctx, *initKeyspace, shard, func(shard *topodatapb.Shard) error {
|
||||
if topoproto.ShardHasCell(shard, agent.TabletAlias.Cell) {
|
||||
si, err = agent.TopoServer.UpdateShardFields(ctx, *initKeyspace, shard, func(si *topo.ShardInfo) error {
|
||||
if si.HasCell(agent.TabletAlias.Cell) {
|
||||
// Someone else already did it.
|
||||
return topo.ErrNoUpdateNeeded
|
||||
}
|
||||
shard.Cells = append(shard.Cells, agent.TabletAlias.Cell)
|
||||
si.Cells = append(si.Cells, agent.TabletAlias.Cell)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"github.com/youtube/vitess/go/history"
|
||||
"github.com/youtube/vitess/go/vt/dbconfigs"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
|
||||
"github.com/youtube/vitess/go/vt/zktopo/zktestserver"
|
||||
"golang.org/x/net/context"
|
||||
|
@ -99,13 +100,12 @@ func TestInitTablet(t *testing.T) {
|
|||
}
|
||||
|
||||
// update shard's master to our alias, then try to init again
|
||||
si, err = ts.GetShard(ctx, "test_keyspace", "-80")
|
||||
si, err = agent.TopoServer.UpdateShardFields(ctx, "test_keyspace", "-80", func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = tabletAlias
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.MasterAlias = tabletAlias
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
if err := agent.InitTablet(port, gRPCPort); err != nil {
|
||||
t.Fatalf("InitTablet(type, healthcheck) failed: %v", err)
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/hook"
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
querypb "github.com/youtube/vitess/go/vt/proto/query"
|
||||
|
@ -19,6 +18,150 @@ import (
|
|||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
// TabletAction is the name of an action. It is a string (and not an
|
||||
// enum) so we can print it easily.
|
||||
type TabletAction string
|
||||
|
||||
const (
|
||||
// TabletActionPing checks a tablet is alive
|
||||
TabletActionPing TabletAction = "Ping"
|
||||
|
||||
// TabletActionSleep will sleep for a duration (used for tests)
|
||||
TabletActionSleep TabletAction = "Sleep"
|
||||
|
||||
// TabletActionExecuteHook will execute the provided hook remotely
|
||||
TabletActionExecuteHook TabletAction = "ExecuteHook"
|
||||
|
||||
// TabletActionSetReadOnly makes the mysql instance read-only
|
||||
TabletActionSetReadOnly TabletAction = "SetReadOnly"
|
||||
|
||||
// TabletActionSetReadWrite makes the mysql instance read-write
|
||||
TabletActionSetReadWrite TabletAction = "SetReadWrite"
|
||||
|
||||
// TabletActionChangeType changes the type of the tablet
|
||||
TabletActionChangeType TabletAction = "ChangeType"
|
||||
|
||||
// TabletActionResetReplication tells the tablet it should
|
||||
// reset its replication state
|
||||
TabletActionResetReplication TabletAction = "ResetReplication"
|
||||
|
||||
// TabletActionInitMaster tells the tablet it should make itself the new
|
||||
// master for the shard it's currently in.
|
||||
TabletActionInitMaster TabletAction = "InitMaster"
|
||||
|
||||
// TabletActionPopulateReparentJournal inserts an entry in the
|
||||
// _vt.reparent_journal table
|
||||
TabletActionPopulateReparentJournal TabletAction = "PopulateReparentJournal"
|
||||
|
||||
// TabletActionInitSlave tells the tablet it should make
|
||||
// itself a slave to the provided master at the given position.
|
||||
TabletActionInitSlave TabletAction = "InitSlave"
|
||||
|
||||
// TabletActionDemoteMaster tells the current master it's
|
||||
// about to not be a master any more, and should go read-only.
|
||||
TabletActionDemoteMaster TabletAction = "DemoteMaster"
|
||||
|
||||
// TabletActionPromoteSlaveWhenCaughtUp tells the tablet to wait
|
||||
// for a given replication point, and when it reaches it
|
||||
// switch to be a master.
|
||||
TabletActionPromoteSlaveWhenCaughtUp TabletAction = "PromoteSlaveWhenCaughtUp"
|
||||
|
||||
// TabletActionSlaveWasPromoted tells a tablet this previously slave
|
||||
// tablet is now the master. The tablet will update its
|
||||
// own topology record.
|
||||
TabletActionSlaveWasPromoted TabletAction = "SlaveWasPromoted"
|
||||
|
||||
// TabletActionSetMaster tells a tablet it has a new master.
|
||||
// The tablet will reparent to the new master, and wait for
|
||||
// the reparent_journal entry.
|
||||
TabletActionSetMaster TabletAction = "SetMaster"
|
||||
|
||||
// TabletActionSlaveWasRestarted tells a tablet the mysql
|
||||
// master was changed. The tablet will check it is indeed the
|
||||
// case, and update its own topology record.
|
||||
TabletActionSlaveWasRestarted TabletAction = "SlaveWasRestarted"
|
||||
|
||||
// TabletActionStopReplicationAndGetStatus will stop replication,
|
||||
// and return the current replication status.
|
||||
TabletActionStopReplicationAndGetStatus TabletAction = "StopReplicationAndGetStatus"
|
||||
|
||||
// TabletActionPromoteSlave will make this tablet the master
|
||||
TabletActionPromoteSlave TabletAction = "PromoteSlave"
|
||||
|
||||
// TabletActionStopSlave will stop MySQL replication.
|
||||
TabletActionStopSlave TabletAction = "StopSlave"
|
||||
|
||||
// TabletActionStopSlaveMinimum will stop MySQL replication
|
||||
// after it reaches a minimum point.
|
||||
TabletActionStopSlaveMinimum TabletAction = "StopSlaveMinimum"
|
||||
|
||||
// TabletActionStartSlave will start MySQL replication.
|
||||
TabletActionStartSlave TabletAction = "StartSlave"
|
||||
|
||||
// TabletActionExternallyReparented is sent directly to the new master
|
||||
// tablet when it becomes the master. It is functionnaly equivalent
|
||||
// to calling "ShardExternallyReparented" on the topology.
|
||||
TabletActionExternallyReparented TabletAction = "TabletExternallyReparented"
|
||||
|
||||
// TabletActionMasterPosition returns the current master position
|
||||
TabletActionMasterPosition TabletAction = "MasterPosition"
|
||||
|
||||
// TabletActionSlaveStatus returns the current slave status
|
||||
TabletActionSlaveStatus TabletAction = "SlaveStatus"
|
||||
|
||||
// TabletActionWaitBLPPosition waits until the slave reaches a
|
||||
// replication position in filtered replication
|
||||
TabletActionWaitBLPPosition TabletAction = "WaitBlpPosition"
|
||||
|
||||
// TabletActionStopBLP stops filtered replication
|
||||
TabletActionStopBLP TabletAction = "StopBlp"
|
||||
|
||||
// TabletActionStartBLP starts filtered replication
|
||||
TabletActionStartBLP TabletAction = "StartBlp"
|
||||
|
||||
// TabletActionRunBLPUntil will run filtered replication until
|
||||
// it reaches the provided stop position.
|
||||
TabletActionRunBLPUntil TabletAction = "RunBlpUntil"
|
||||
|
||||
// TabletActionGetSchema returns the tablet current schema.
|
||||
TabletActionGetSchema TabletAction = "GetSchema"
|
||||
|
||||
// TabletActionRefreshState tells the tablet to refresh its
|
||||
// tablet record from the topo server.
|
||||
TabletActionRefreshState TabletAction = "RefreshState"
|
||||
|
||||
// TabletActionRunHealthCheck tells the tablet to run a health check.
|
||||
TabletActionRunHealthCheck TabletAction = "RunHealthCheck"
|
||||
|
||||
// TabletActionIgnoreHealthError sets the regexp for health errors to ignore.
|
||||
TabletActionIgnoreHealthError TabletAction = "IgnoreHealthError"
|
||||
|
||||
// TabletActionReloadSchema tells the tablet to reload its schema.
|
||||
TabletActionReloadSchema TabletAction = "ReloadSchema"
|
||||
|
||||
// TabletActionPreflightSchema will check a schema change works
|
||||
TabletActionPreflightSchema TabletAction = "PreflightSchema"
|
||||
|
||||
// TabletActionApplySchema will actually apply the schema change
|
||||
TabletActionApplySchema TabletAction = "ApplySchema"
|
||||
|
||||
// TabletActionExecuteFetchAsDba uses the DBA connection to run queries.
|
||||
TabletActionExecuteFetchAsDba TabletAction = "ExecuteFetchAsDba"
|
||||
|
||||
// TabletActionExecuteFetchAsApp uses the App connection to run queries.
|
||||
TabletActionExecuteFetchAsApp TabletAction = "ExecuteFetchAsApp"
|
||||
|
||||
// TabletActionGetPermissions returns the mysql permissions set
|
||||
TabletActionGetPermissions TabletAction = "GetPermissions"
|
||||
|
||||
// TabletActionGetSlaves returns the current set of mysql
|
||||
// replication slaves.
|
||||
TabletActionGetSlaves TabletAction = "GetSlaves"
|
||||
|
||||
// TabletActionBackup takes a db backup and stores it into BackupStorage
|
||||
TabletActionBackup TabletAction = "Backup"
|
||||
)
|
||||
|
||||
// RPCAgent defines the interface implemented by the Agent for RPCs.
|
||||
// It is useful for RPC implementations to test their full stack.
|
||||
type RPCAgent interface {
|
||||
|
@ -100,7 +243,7 @@ type RPCAgent interface {
|
|||
|
||||
SetMaster(ctx context.Context, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
|
||||
|
||||
SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error
|
||||
SlaveWasRestarted(ctx context.Context, parent *topodatapb.TabletAlias) error
|
||||
|
||||
StopReplicationAndGetStatus(ctx context.Context) (*replicationdatapb.Status, error)
|
||||
|
||||
|
@ -111,7 +254,7 @@ type RPCAgent interface {
|
|||
Backup(ctx context.Context, concurrency int, logger logutil.Logger) error
|
||||
|
||||
// RPC helpers
|
||||
RPCWrap(ctx context.Context, name string, args, reply interface{}, f func() error) error
|
||||
RPCWrapLock(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) error
|
||||
RPCWrapLockAction(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) error
|
||||
RPCWrap(ctx context.Context, name TabletAction, args, reply interface{}, f func() error) error
|
||||
RPCWrapLock(ctx context.Context, name TabletAction, args, reply interface{}, verbose bool, f func() error) error
|
||||
RPCWrapLockAction(ctx context.Context, name TabletAction, args, reply interface{}, verbose bool, f func() error) error
|
||||
}
|
||||
|
|
|
@ -193,11 +193,11 @@ func (agent *ActionAgent) finalizeTabletExternallyReparented(ctx context.Context
|
|||
// write it back. Now we use an update loop pattern to do that instead.
|
||||
event.DispatchUpdate(ev, "updating global shard record")
|
||||
log.Infof("finalizeTabletExternallyReparented: updating global shard record if needed")
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, tablet.Keyspace, tablet.Shard, func(shard *topodatapb.Shard) error {
|
||||
if topoproto.TabletAliasEqual(shard.MasterAlias, tablet.Alias) {
|
||||
_, err = agent.TopoServer.UpdateShardFields(ctx, tablet.Keyspace, tablet.Shard, func(si *topo.ShardInfo) error {
|
||||
if topoproto.TabletAliasEqual(si.MasterAlias, tablet.Alias) {
|
||||
return topo.ErrNoUpdateNeeded
|
||||
}
|
||||
shard.MasterAlias = tablet.Alias
|
||||
si.MasterAlias = tablet.Alias
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
|
|
|
@ -14,7 +14,6 @@ import (
|
|||
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/topotools"
|
||||
|
@ -369,7 +368,7 @@ func (agent *ActionAgent) SetMaster(ctx context.Context, parentAlias *topodatapb
|
|||
|
||||
// SlaveWasRestarted updates the parent record for a tablet.
|
||||
// Should be called under RPCWrapLockAction.
|
||||
func (agent *ActionAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error {
|
||||
func (agent *ActionAgent) SlaveWasRestarted(ctx context.Context, parent *topodatapb.TabletAlias) error {
|
||||
runHealthCheck := false
|
||||
|
||||
// Once this action completes, update authoritative tablet node first.
|
||||
|
|
|
@ -30,7 +30,7 @@ const rpcTimeout = time.Second * 30
|
|||
//
|
||||
|
||||
// rpcWrapper handles all the logic for rpc calls.
|
||||
func (agent *ActionAgent) rpcWrapper(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error, lock, runAfterAction bool) (err error) {
|
||||
func (agent *ActionAgent) rpcWrapper(ctx context.Context, name TabletAction, args, reply interface{}, verbose bool, f func() error, lock, runAfterAction bool) (err error) {
|
||||
defer func() {
|
||||
if x := recover(); x != nil {
|
||||
log.Errorf("TabletManager.%v(%v) on %v panic: %v\n%s", name, args, topoproto.TabletAliasString(agent.TabletAlias), x, tb.Stack(4))
|
||||
|
@ -49,7 +49,7 @@ func (agent *ActionAgent) rpcWrapper(ctx context.Context, name string, args, rep
|
|||
agent.actionMutex.Lock()
|
||||
defer agent.actionMutex.Unlock()
|
||||
if time.Now().Sub(beforeLock) > rpcTimeout {
|
||||
return fmt.Errorf("server timeout for " + name)
|
||||
return fmt.Errorf("server timeout for %v", name)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -61,27 +61,27 @@ func (agent *ActionAgent) rpcWrapper(ctx context.Context, name string, args, rep
|
|||
log.Infof("TabletManager.%v(%v)(on %v from %v): %#v", name, args, topoproto.TabletAliasString(agent.TabletAlias), from, reply)
|
||||
}
|
||||
if runAfterAction {
|
||||
err = agent.refreshTablet(ctx, "RPC("+name+")")
|
||||
err = agent.refreshTablet(ctx, "RPC("+string(name)+")")
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// RPCWrap is for read-only actions that can be executed concurrently.
|
||||
// verbose is forced to false.
|
||||
func (agent *ActionAgent) RPCWrap(ctx context.Context, name string, args, reply interface{}, f func() error) error {
|
||||
func (agent *ActionAgent) RPCWrap(ctx context.Context, name TabletAction, args, reply interface{}, f func() error) error {
|
||||
return agent.rpcWrapper(ctx, name, args, reply, false /*verbose*/, f,
|
||||
false /*lock*/, false /*runAfterAction*/)
|
||||
}
|
||||
|
||||
// RPCWrapLock is for actions that should not run concurrently with each other.
|
||||
func (agent *ActionAgent) RPCWrapLock(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) error {
|
||||
func (agent *ActionAgent) RPCWrapLock(ctx context.Context, name TabletAction, args, reply interface{}, verbose bool, f func() error) error {
|
||||
return agent.rpcWrapper(ctx, name, args, reply, verbose, f,
|
||||
true /*lock*/, false /*runAfterAction*/)
|
||||
}
|
||||
|
||||
// RPCWrapLockAction is the same as RPCWrapLock, plus it will call refreshTablet
|
||||
// after the action returns.
|
||||
func (agent *ActionAgent) RPCWrapLockAction(ctx context.Context, name string, args, reply interface{}, verbose bool, f func() error) error {
|
||||
func (agent *ActionAgent) RPCWrapLockAction(ctx context.Context, name TabletAction, args, reply interface{}, verbose bool, f func() error) error {
|
||||
return agent.rpcWrapper(ctx, name, args, reply, verbose, f,
|
||||
true /*lock*/, true /*runAfterAction*/)
|
||||
}
|
||||
|
|
|
@ -12,7 +12,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/hook"
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
querypb "github.com/youtube/vitess/go/vt/proto/query"
|
||||
|
@ -168,7 +167,7 @@ type TabletManagerClient interface {
|
|||
SetMaster(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, timeCreatedNS int64, forceStartSlave bool) error
|
||||
|
||||
// SlaveWasRestarted tells the remote tablet its master has changed
|
||||
SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, args *actionnode.SlaveWasRestartedArgs) error
|
||||
SlaveWasRestarted(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias) error
|
||||
|
||||
// StopReplicationAndGetStatus stops replication and returns the
|
||||
// current position.
|
||||
|
|
|
@ -137,7 +137,7 @@ func (ki *KeyspaceInfo) ComputeCellServedFrom(cell string) []*topodatapb.SrvKeys
|
|||
return result
|
||||
}
|
||||
|
||||
// CreateKeyspace wraps the underlying Impl.DeleteKeyspaceShards
|
||||
// CreateKeyspace wraps the underlying Impl.CreateKeyspace
|
||||
// and dispatches the event.
|
||||
func (ts Server) CreateKeyspace(ctx context.Context, keyspace string, value *topodatapb.Keyspace) error {
|
||||
if err := ts.Impl.CreateKeyspace(ctx, keyspace, value); err != nil {
|
||||
|
@ -165,14 +165,15 @@ func (ts Server) GetKeyspace(ctx context.Context, keyspace string) (*KeyspaceInf
|
|||
}, nil
|
||||
}
|
||||
|
||||
// UpdateKeyspace updates the keyspace data, with the right version
|
||||
// UpdateKeyspace updates the keyspace data. It checks the keyspace is locked.
|
||||
func (ts Server) UpdateKeyspace(ctx context.Context, ki *KeyspaceInfo) error {
|
||||
var version int64 = -1
|
||||
if ki.version != 0 {
|
||||
version = ki.version
|
||||
// make sure it is locked first
|
||||
if err := CheckKeyspaceLocked(ctx, ki.keyspace); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
newVersion, err := ts.Impl.UpdateKeyspace(ctx, ki.keyspace, ki.Keyspace, version)
|
||||
// call the Impl's version
|
||||
newVersion, err := ts.Impl.UpdateKeyspace(ctx, ki.keyspace, ki.Keyspace, ki.version)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -183,7 +184,6 @@ func (ts Server) UpdateKeyspace(ctx context.Context, ki *KeyspaceInfo) error {
|
|||
Keyspace: ki.Keyspace,
|
||||
Status: "updated",
|
||||
})
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,391 @@
|
|||
// Copyright 2016, Google Inc. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package topo
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"flag"
|
||||
"fmt"
|
||||
"os"
|
||||
"os/user"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/trace"
|
||||
)
|
||||
|
||||
// This file contains utility methods and definitions to lock
|
||||
// keyspaces and shards.
|
||||
|
||||
var (
|
||||
// DefaultLockTimeout is a good value to use as a default for
|
||||
// locking a shard / keyspace.
|
||||
DefaultLockTimeout = 30 * time.Second
|
||||
|
||||
// LockTimeout is the command line flag that introduces a shorter
|
||||
// timeout for locking topology structures.
|
||||
LockTimeout = flag.Duration("lock_timeout", DefaultLockTimeout, "timeout for acquiring topology locks")
|
||||
)
|
||||
|
||||
// Lock describes a long-running lock on a keyspace or a shard.
|
||||
// It needs to be public as we JSON-serialize it.
|
||||
type Lock struct {
|
||||
// Action and the following fields are set at construction time.
|
||||
Action string
|
||||
HostName string
|
||||
UserName string
|
||||
Time string
|
||||
|
||||
// Status is the current status of the Lock.
|
||||
Status string
|
||||
}
|
||||
|
||||
// newLock creates a new Lock.
|
||||
func newLock(action string) *Lock {
|
||||
l := &Lock{
|
||||
Action: action,
|
||||
HostName: "unknown",
|
||||
UserName: "unknown",
|
||||
Time: time.Now().Format(time.RFC3339),
|
||||
Status: "Running",
|
||||
}
|
||||
if h, err := os.Hostname(); err == nil {
|
||||
l.HostName = h
|
||||
}
|
||||
if u, err := user.Current(); err == nil {
|
||||
l.UserName = u.Username
|
||||
}
|
||||
return l
|
||||
}
|
||||
|
||||
// ToJSON returns a JSON representation of the object.
|
||||
func (l *Lock) ToJSON() (string, error) {
|
||||
data, err := json.MarshalIndent(l, "", " ")
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("cannot JSON-marshal node: %v", err)
|
||||
}
|
||||
return string(data), nil
|
||||
}
|
||||
|
||||
// lockInfo is an individual info structure for a lock
|
||||
type lockInfo struct {
|
||||
lockPath string
|
||||
actionNode *Lock
|
||||
}
|
||||
|
||||
// locksInfo is the structure used to remember which locks we took
|
||||
type locksInfo struct {
|
||||
// mu protects the following members of the structure.
|
||||
// Safer to be thread safe here, in case multiple go routines
|
||||
// lock different things.
|
||||
mu sync.Mutex
|
||||
|
||||
// info contans all the locks we took. It is indexed by
|
||||
// keyspace (for keyspaces) or keyspace/shard (for shards).
|
||||
info map[string]*lockInfo
|
||||
}
|
||||
|
||||
// Context glue
|
||||
type locksKeyType int
|
||||
|
||||
var locksKey locksKeyType
|
||||
|
||||
// LockKeyspace will lock the keyspace, and return:
|
||||
// - a context with a locksInfo structure for future reference.
|
||||
// - an unlock method
|
||||
// - an error if anything failed.
|
||||
//
|
||||
// We lock a keyspace for the following operations to be guaranteed
|
||||
// exclusive operation:
|
||||
// * changing a keyspace sharding info fields (is this one necessary?)
|
||||
// * changing a keyspace 'ServedFrom' field (is this one necessary?)
|
||||
// * resharding operations:
|
||||
// * horizontal resharding: includes changing the shard's 'ServedType',
|
||||
// as well as the associated horizontal resharding operations.
|
||||
// * vertical resharding: includes changing the keyspace 'ServedFrom'
|
||||
// field, as well as the associated vertical resharding operations.
|
||||
// * 'vtctl SetShardServedTypes' emergency operations
|
||||
// * 'vtctl SetShardTabletControl' emergency operations
|
||||
// * 'vtctl SourceShardAdd' and 'vtctl SourceShardDelete' emergency operations
|
||||
// * keyspace-wide schema changes
|
||||
func (ts Server) LockKeyspace(ctx context.Context, keyspace, action string) (context.Context, func(*error), error) {
|
||||
i, ok := ctx.Value(locksKey).(*locksInfo)
|
||||
if !ok {
|
||||
i = &locksInfo{
|
||||
info: make(map[string]*lockInfo),
|
||||
}
|
||||
ctx = context.WithValue(ctx, locksKey, i)
|
||||
}
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
// check that we're not already locked
|
||||
if _, ok = i.info[keyspace]; ok {
|
||||
return nil, nil, fmt.Errorf("lock for keyspace %v is already held", keyspace)
|
||||
}
|
||||
|
||||
// lock
|
||||
l := newLock(action)
|
||||
lockPath, err := l.lockKeyspace(ctx, ts, keyspace)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// and update our structure
|
||||
i.info[keyspace] = &lockInfo{
|
||||
lockPath: lockPath,
|
||||
actionNode: l,
|
||||
}
|
||||
return ctx, func(finalErr *error) {
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
if _, ok := i.info[keyspace]; !ok {
|
||||
if *finalErr != nil {
|
||||
log.Errorf("trying to unlock keyspace %v multiple times", keyspace)
|
||||
} else {
|
||||
*finalErr = fmt.Errorf("trying to unlock keyspace %v multiple times", keyspace)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
err := l.unlockKeyspace(ctx, ts, keyspace, lockPath, *finalErr)
|
||||
if *finalErr != nil {
|
||||
if err != nil {
|
||||
// both error are set, just log the unlock error
|
||||
log.Errorf("unlockKeyspace(%v) failed: %v", keyspace, err)
|
||||
}
|
||||
} else {
|
||||
*finalErr = err
|
||||
}
|
||||
delete(i.info, keyspace)
|
||||
}, nil
|
||||
}
|
||||
|
||||
// CheckKeyspaceLocked can be called on a context to make sure we have the lock
|
||||
// for a given keyspace.
|
||||
func CheckKeyspaceLocked(ctx context.Context, keyspace string) error {
|
||||
// extract the locksInfo pointer
|
||||
i, ok := ctx.Value(locksKey).(*locksInfo)
|
||||
if !ok {
|
||||
return fmt.Errorf("keyspace %v is not locked (no locksInfo)", keyspace)
|
||||
}
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
// find the individual entry
|
||||
_, ok = i.info[keyspace]
|
||||
if !ok {
|
||||
return fmt.Errorf("keyspace %v is not locked (no lockInfo in map)", keyspace)
|
||||
}
|
||||
|
||||
// TODO(alainjobart): check the lock server implementation
|
||||
// still holds the lock. Will need to look at the lockInfo struct.
|
||||
|
||||
// and we're good for now.
|
||||
return nil
|
||||
}
|
||||
|
||||
// lockKeyspace will lock the keyspace in the topology server.
|
||||
// unlockKeyspace should be called if this returns no error.
|
||||
func (l *Lock) lockKeyspace(ctx context.Context, ts Server, keyspace string) (lockPath string, err error) {
|
||||
log.Infof("Locking keyspace %v for action %v", keyspace, l.Action)
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, *LockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.LockKeyspaceForAction")
|
||||
span.Annotate("action", l.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
defer span.Finish()
|
||||
|
||||
j, err := l.ToJSON()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return ts.LockKeyspaceForAction(ctx, keyspace, j)
|
||||
}
|
||||
|
||||
// unlockKeyspace unlocks a previously locked keyspace.
|
||||
func (l *Lock) unlockKeyspace(ctx context.Context, ts Server, keyspace string, lockPath string, actionError error) error {
|
||||
// Detach from the parent timeout, but copy the trace span.
|
||||
// We need to still release the lock even if the parent
|
||||
// context timed out.
|
||||
ctx = trace.CopySpan(context.TODO(), ctx)
|
||||
ctx, cancel := context.WithTimeout(ctx, DefaultLockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.UnlockKeyspaceForAction")
|
||||
span.Annotate("action", l.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
defer span.Finish()
|
||||
|
||||
// first update the actionNode
|
||||
if actionError != nil {
|
||||
log.Infof("Unlocking keyspace %v for action %v with error %v", keyspace, l.Action, actionError)
|
||||
l.Status = "Error: " + actionError.Error()
|
||||
} else {
|
||||
log.Infof("Unlocking keyspace %v for successful action %v", keyspace, l.Action)
|
||||
l.Status = "Done"
|
||||
}
|
||||
j, err := l.ToJSON()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return ts.UnlockKeyspaceForAction(ctx, keyspace, lockPath, j)
|
||||
}
|
||||
|
||||
// LockShard will lock the shard, and return:
|
||||
// - a context with a locksInfo structure for future reference.
|
||||
// - an unlock method
|
||||
// - an error if anything failed.
|
||||
//
|
||||
// We are currently only using this method to lock actions that would
|
||||
// impact each-other. Most changes of the Shard object are done by
|
||||
// UpdateShardFields, which is not locking the shard object. The
|
||||
// current list of actions that lock a shard are:
|
||||
// * all Vitess-controlled re-parenting operations:
|
||||
// * InitShardMaster
|
||||
// * PlannedReparentShard
|
||||
// * EmergencyReparentShard
|
||||
// * operations that we don't want to conflict with re-parenting:
|
||||
// * DeleteTablet when it's the shard's current master
|
||||
//
|
||||
func (ts Server) LockShard(ctx context.Context, keyspace, shard, action string) (context.Context, func(*error), error) {
|
||||
i, ok := ctx.Value(locksKey).(*locksInfo)
|
||||
if !ok {
|
||||
i = &locksInfo{
|
||||
info: make(map[string]*lockInfo),
|
||||
}
|
||||
ctx = context.WithValue(ctx, locksKey, i)
|
||||
}
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
// check that we're not already locked
|
||||
mapKey := keyspace + "/" + shard
|
||||
if _, ok = i.info[mapKey]; ok {
|
||||
return nil, nil, fmt.Errorf("lock for shard %v/%v is already held", keyspace, shard)
|
||||
}
|
||||
|
||||
// lock
|
||||
l := newLock(action)
|
||||
lockPath, err := l.lockShard(ctx, ts, keyspace, shard)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// and update our structure
|
||||
i.info[mapKey] = &lockInfo{
|
||||
lockPath: lockPath,
|
||||
actionNode: l,
|
||||
}
|
||||
return ctx, func(finalErr *error) {
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
if _, ok := i.info[mapKey]; !ok {
|
||||
if *finalErr != nil {
|
||||
log.Errorf("trying to unlock shard %v/%v multiple times", keyspace, shard)
|
||||
} else {
|
||||
*finalErr = fmt.Errorf("trying to unlock shard %v/%v multiple times", keyspace, shard)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
err := l.unlockShard(ctx, ts, keyspace, shard, lockPath, *finalErr)
|
||||
if *finalErr != nil {
|
||||
if err != nil {
|
||||
// both error are set, just log the unlock error
|
||||
log.Warningf("unlockShard(%s/%s) failed: %v", keyspace, shard, err)
|
||||
}
|
||||
} else {
|
||||
*finalErr = err
|
||||
}
|
||||
delete(i.info, mapKey)
|
||||
}, nil
|
||||
}
|
||||
|
||||
// CheckShardLocked can be called on a context to make sure we have the lock
|
||||
// for a given shard.
|
||||
func CheckShardLocked(ctx context.Context, keyspace, shard string) error {
|
||||
// extract the locksInfo pointer
|
||||
i, ok := ctx.Value(locksKey).(*locksInfo)
|
||||
if !ok {
|
||||
return fmt.Errorf("shard %v/%v is not locked (no locksInfo)", keyspace, shard)
|
||||
}
|
||||
i.mu.Lock()
|
||||
defer i.mu.Unlock()
|
||||
|
||||
// func the individual entry
|
||||
mapKey := keyspace + "/" + shard
|
||||
_, ok = i.info[mapKey]
|
||||
if !ok {
|
||||
return fmt.Errorf("shard %v/%v is not locked (no lockInfo in map)", keyspace, shard)
|
||||
}
|
||||
|
||||
// TODO(alainjobart): check the lock server implementation
|
||||
// still holds the lock. Will need to look at the lockInfo struct.
|
||||
|
||||
// and we're good for now.
|
||||
return nil
|
||||
}
|
||||
|
||||
// lockShard will lock the shard in the topology server.
|
||||
// UnlockShard should be called if this returns no error.
|
||||
func (l *Lock) lockShard(ctx context.Context, ts Server, keyspace, shard string) (lockPath string, err error) {
|
||||
log.Infof("Locking shard %v/%v for action %v", keyspace, shard, l.Action)
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, *LockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.LockShardForAction")
|
||||
span.Annotate("action", l.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
span.Annotate("shard", shard)
|
||||
defer span.Finish()
|
||||
|
||||
j, err := l.ToJSON()
|
||||
if err != nil {
|
||||
return "", err
|
||||
}
|
||||
return ts.LockShardForAction(ctx, keyspace, shard, j)
|
||||
}
|
||||
|
||||
// unlockShard unlocks a previously locked shard.
|
||||
func (l *Lock) unlockShard(ctx context.Context, ts Server, keyspace, shard string, lockPath string, actionError error) error {
|
||||
// Detach from the parent timeout, but copy the trace span.
|
||||
// We need to still release the lock even if the parent context timed out.
|
||||
ctx = trace.CopySpan(context.TODO(), ctx)
|
||||
ctx, cancel := context.WithTimeout(ctx, DefaultLockTimeout)
|
||||
defer cancel()
|
||||
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.UnlockShardForAction")
|
||||
span.Annotate("action", l.Action)
|
||||
span.Annotate("keyspace", keyspace)
|
||||
span.Annotate("shard", shard)
|
||||
defer span.Finish()
|
||||
|
||||
// first update the actionNode
|
||||
if actionError != nil {
|
||||
log.Infof("Unlocking shard %v/%v for action %v with error %v", keyspace, shard, l.Action, actionError)
|
||||
l.Status = "Error: " + actionError.Error()
|
||||
} else {
|
||||
log.Infof("Unlocking shard %v/%v for successful action %v", keyspace, shard, l.Action)
|
||||
l.Status = "Done"
|
||||
}
|
||||
j, err := l.ToJSON()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return ts.UnlockShardForAction(ctx, keyspace, shard, lockPath, j)
|
||||
}
|
|
@ -85,7 +85,7 @@ type Impl interface {
|
|||
// Can return ErrNoNode if the keyspace doesn't exist yet,
|
||||
// or ErrBadVersion if the version has changed.
|
||||
//
|
||||
// Do not use directly, but instead use topo.UpdateKeyspace.
|
||||
// Do not use directly, but instead use Server.UpdateKeyspace.
|
||||
UpdateKeyspace(ctx context.Context, keyspace string, value *topodatapb.Keyspace, existingVersion int64) (newVersion int64, err error)
|
||||
|
||||
// DeleteKeyspace deletes the specified keyspace.
|
||||
|
@ -115,11 +115,10 @@ type Impl interface {
|
|||
|
||||
// UpdateShard updates the shard information
|
||||
// pointed at by si.keyspace / si.shard to the *si value.
|
||||
// This will only be called with a lock on the shard.
|
||||
// Can return ErrNoNode if the shard doesn't exist yet,
|
||||
// or ErrBadVersion if the version has changed.
|
||||
//
|
||||
// Do not use directly, but instead use topo.UpdateShard.
|
||||
// Do not use directly, but instead use topo.UpdateShardFields.
|
||||
UpdateShard(ctx context.Context, keyspace, shard string, value *topodatapb.Shard, existingVersion int64) (newVersion int64, err error)
|
||||
|
||||
// ValidateShard performs routine checks on the shard.
|
||||
|
|
|
@ -172,21 +172,21 @@ func (ts Server) GetShard(ctx context.Context, keyspace, shard string) (*ShardIn
|
|||
}, nil
|
||||
}
|
||||
|
||||
// UpdateShard updates the shard data, with the right version.
|
||||
// UpdateShard masks ts.Impl.UpdateShard so nobody is tempted to use it.
|
||||
func (ts Server) UpdateShard() error {
|
||||
panic("do not call this function directly, use UpdateShardFields instead")
|
||||
}
|
||||
|
||||
// updateShard updates the shard data, with the right version.
|
||||
// It also creates a span, and dispatches the event.
|
||||
func (ts Server) UpdateShard(ctx context.Context, si *ShardInfo) error {
|
||||
func (ts Server) updateShard(ctx context.Context, si *ShardInfo) error {
|
||||
span := trace.NewSpanFromContext(ctx)
|
||||
span.StartClient("TopoServer.UpdateShard")
|
||||
span.Annotate("keyspace", si.keyspace)
|
||||
span.Annotate("shard", si.shardName)
|
||||
defer span.Finish()
|
||||
|
||||
var version int64 = -1
|
||||
if si.version != 0 {
|
||||
version = si.version
|
||||
}
|
||||
|
||||
newVersion, err := ts.Impl.UpdateShard(ctx, si.keyspace, si.shardName, si.Shard, version)
|
||||
newVersion, err := ts.Impl.UpdateShard(ctx, si.keyspace, si.shardName, si.Shard, si.version)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -207,29 +207,39 @@ func (ts Server) UpdateShard(ctx context.Context, si *ShardInfo) error {
|
|||
// If the update succeeds, it returns the updated ShardInfo.
|
||||
// If the update method returns ErrNoUpdateNeeded, nothing is written,
|
||||
// and nil,nil is returned.
|
||||
func (ts Server) UpdateShardFields(ctx context.Context, keyspace, shard string, update func(*topodatapb.Shard) error) (*ShardInfo, error) {
|
||||
//
|
||||
// Note the callback method takes a ShardInfo, so it can get the
|
||||
// keyspace and shard from it, or use all the ShardInfo methods.
|
||||
func (ts Server) UpdateShardFields(ctx context.Context, keyspace, shard string, update func(*ShardInfo) error) (*ShardInfo, error) {
|
||||
for {
|
||||
si, err := ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err = update(si.Shard); err != nil {
|
||||
if err = update(si); err != nil {
|
||||
if err == ErrNoUpdateNeeded {
|
||||
return nil, nil
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
if err = ts.UpdateShard(ctx, si); err != ErrBadVersion {
|
||||
if err = ts.updateShard(ctx, si); err != ErrBadVersion {
|
||||
return si, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// CreateShard creates a new shard and tries to fill in the right information.
|
||||
// This should be called while holding the keyspace lock for the shard.
|
||||
// (call topotools.CreateShard to do that for you).
|
||||
// In unit tests (that are not parallel), this function can be called directly.
|
||||
func (ts Server) CreateShard(ctx context.Context, keyspace, shard string) error {
|
||||
// This will lock the Keyspace, as we may be looking at other shard servedTypes.
|
||||
// Using GetOrCreateShard is probably a better idea for most use cases.
|
||||
func (ts Server) CreateShard(ctx context.Context, keyspace, shard string) (err error) {
|
||||
// Lock the keyspace, because we'll be looking at ServedTypes.
|
||||
ctx, unlock, lockErr := ts.LockKeyspace(ctx, keyspace, "CreateShard")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// validate parameters
|
||||
name, keyRange, err := ValidateShardName(shard)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -269,6 +279,8 @@ func (ts Server) CreateShard(ctx context.Context, keyspace, shard string) error
|
|||
}
|
||||
|
||||
if err := ts.Impl.CreateShard(ctx, keyspace, name, value); err != nil {
|
||||
// return error as is, we need to propagate
|
||||
// ErrNodeExists for instance.
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -281,6 +293,29 @@ func (ts Server) CreateShard(ctx context.Context, keyspace, shard string) error
|
|||
return nil
|
||||
}
|
||||
|
||||
// GetOrCreateShard will return the shard object, or create one if it doesn't
|
||||
// already exist. Note the shard creation is protected by a keyspace Lock.
|
||||
func (ts Server) GetOrCreateShard(ctx context.Context, keyspace, shard string) (si *ShardInfo, err error) {
|
||||
si, err = ts.GetShard(ctx, keyspace, shard)
|
||||
if err != ErrNoNode {
|
||||
return
|
||||
}
|
||||
|
||||
// create the keyspace, maybe it already exists
|
||||
if err = ts.CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{}); err != nil && err != ErrNodeExists {
|
||||
return nil, fmt.Errorf("CreateKeyspace(%v) failed: %v", keyspace, err)
|
||||
}
|
||||
|
||||
// now try to create with the lock, may already exist
|
||||
if err = ts.CreateShard(ctx, keyspace, shard); err != nil && err != ErrNodeExists {
|
||||
return nil, fmt.Errorf("CreateShard(%v/%v) failed: %v", keyspace, shard, err)
|
||||
}
|
||||
|
||||
// try to read the shard again, maybe someone created it
|
||||
// in between the original GetShard and the LockKeyspace
|
||||
return ts.GetShard(ctx, keyspace, shard)
|
||||
}
|
||||
|
||||
// DeleteShard wraps the underlying Impl.DeleteShard
|
||||
// and dispatches the event.
|
||||
func (ts Server) DeleteShard(ctx context.Context, keyspace, shard string) error {
|
||||
|
@ -314,7 +349,12 @@ func (si *ShardInfo) GetTabletControl(tabletType topodatapb.TabletType) *topodat
|
|||
// table list that the provided one, we error out.
|
||||
// - we don't support DisableQueryService at the same time as BlacklistedTables,
|
||||
// because it's not used in the same context (vertical vs horizontal sharding)
|
||||
func (si *ShardInfo) UpdateSourceBlacklistedTables(tabletType topodatapb.TabletType, cells []string, remove bool, tables []string) error {
|
||||
//
|
||||
// This function should be called while holding the keyspace lock.
|
||||
func (si *ShardInfo) UpdateSourceBlacklistedTables(ctx context.Context, tabletType topodatapb.TabletType, cells []string, remove bool, tables []string) error {
|
||||
if err := CheckKeyspaceLocked(ctx, si.keyspace); err != nil {
|
||||
return err
|
||||
}
|
||||
tc := si.GetTabletControl(tabletType)
|
||||
if tc == nil {
|
||||
// handle the case where the TabletControl object is new
|
||||
|
@ -358,7 +398,11 @@ func (si *ShardInfo) UpdateSourceBlacklistedTables(tabletType topodatapb.TabletT
|
|||
// of the corner cases:
|
||||
// - we don't support DisableQueryService at the same time as BlacklistedTables,
|
||||
// because it's not used in the same context (vertical vs horizontal sharding)
|
||||
func (si *ShardInfo) UpdateDisableQueryService(tabletType topodatapb.TabletType, cells []string, disableQueryService bool) error {
|
||||
// This function should be called while holding the keyspace lock.
|
||||
func (si *ShardInfo) UpdateDisableQueryService(ctx context.Context, tabletType topodatapb.TabletType, cells []string, disableQueryService bool) error {
|
||||
if err := CheckKeyspaceLocked(ctx, si.keyspace); err != nil {
|
||||
return err
|
||||
}
|
||||
tc := si.GetTabletControl(tabletType)
|
||||
if tc == nil {
|
||||
// handle the case where the TabletControl object is new
|
||||
|
@ -370,7 +414,7 @@ func (si *ShardInfo) UpdateDisableQueryService(tabletType topodatapb.TabletType,
|
|||
BlacklistedTables: nil,
|
||||
})
|
||||
} else {
|
||||
log.Warningf("Trying to remove TabletControl.DisableQueryService for missing type: %v", tabletType)
|
||||
log.Warningf("Trying to remove TabletControl.DisableQueryService for missing type %v for shard %v/%v", tabletType, si.keyspace, si.shardName)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -381,7 +425,7 @@ func (si *ShardInfo) UpdateDisableQueryService(tabletType topodatapb.TabletType,
|
|||
return fmt.Errorf("cannot safely alter DisableQueryService as BlacklistedTables is set")
|
||||
}
|
||||
if !tc.DisableQueryService {
|
||||
return fmt.Errorf("cannot safely alter DisableQueryService as DisableQueryService is not set, this record should not be there")
|
||||
return fmt.Errorf("cannot safely alter DisableQueryService as DisableQueryService is not set, this record should not be there for shard %v/%v", si.keyspace, si.shardName)
|
||||
}
|
||||
|
||||
if disableQueryService {
|
||||
|
|
|
@ -8,6 +8,8 @@ import (
|
|||
"reflect"
|
||||
"testing"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
|
@ -61,13 +63,31 @@ func TestRemoveCells(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func lockedKeyspaceContext(keyspace string) context.Context {
|
||||
ctx := context.Background()
|
||||
return context.WithValue(ctx, locksKey, &locksInfo{
|
||||
info: map[string]*lockInfo{
|
||||
keyspace: {
|
||||
lockPath: "path",
|
||||
},
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
func TestUpdateSourceBlacklistedTables(t *testing.T) {
|
||||
si := NewShardInfo("ks", "sh", &topodatapb.Shard{
|
||||
Cells: []string{"first", "second", "third"},
|
||||
}, 1)
|
||||
|
||||
// check we enforce the keyspace lock
|
||||
ctx := context.Background()
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, nil, false, nil); err == nil || err.Error() != "keyspace ks is not locked (no locksInfo)" {
|
||||
t.Fatalf("unlocked keyspace produced wrong error: %v", err)
|
||||
}
|
||||
ctx = lockedKeyspaceContext("ks")
|
||||
|
||||
// add one cell
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"first"}, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first"},
|
||||
|
@ -78,24 +98,24 @@ func TestUpdateSourceBlacklistedTables(t *testing.T) {
|
|||
}
|
||||
|
||||
// remove that cell, going back
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"first"}, true, nil); err != nil || len(si.TabletControls) != 0 {
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, true, nil); err != nil || len(si.TabletControls) != 0 {
|
||||
t.Fatalf("going back should have remove the record: %v", si)
|
||||
}
|
||||
|
||||
// re-add a cell, then another with different table list to
|
||||
// make sure it fails
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"first"}, false, []string{"t1", "t2"}); err != nil {
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, false, []string{"t1", "t2"}); err != nil {
|
||||
t.Fatalf("one cell add failed: %v", si)
|
||||
}
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t2", "t3"}); err == nil || err.Error() != "trying to use two different sets of blacklisted tables for shard ks/sh: [t1 t2] and [t2 t3]" {
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t2", "t3"}); err == nil || err.Error() != "trying to use two different sets of blacklisted tables for shard ks/sh: [t1 t2] and [t2 t3]" {
|
||||
t.Fatalf("different table list should fail: %v", err)
|
||||
}
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"first"}, true); err == nil || err.Error() != "cannot safely alter DisableQueryService as BlacklistedTables is set" {
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, true); err == nil || err.Error() != "cannot safely alter DisableQueryService as BlacklistedTables is set" {
|
||||
t.Fatalf("UpdateDisableQueryService should fail: %v", err)
|
||||
}
|
||||
|
||||
// add another cell, see the list grow
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first", "second"},
|
||||
|
@ -106,7 +126,7 @@ func TestUpdateSourceBlacklistedTables(t *testing.T) {
|
|||
}
|
||||
|
||||
// add all cells, see the list grow to all
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, nil, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, nil, false, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: nil,
|
||||
|
@ -117,7 +137,7 @@ func TestUpdateSourceBlacklistedTables(t *testing.T) {
|
|||
}
|
||||
|
||||
// remove one cell from the full list
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"second"}, true, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, true, []string{"t1", "t2"}); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first", "third"},
|
||||
|
@ -133,8 +153,15 @@ func TestUpdateDisableQueryService(t *testing.T) {
|
|||
Cells: []string{"first", "second", "third"},
|
||||
}, 1)
|
||||
|
||||
// check we enforce the keyspace lock
|
||||
ctx := context.Background()
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, nil, true); err == nil || err.Error() != "keyspace ks is not locked (no locksInfo)" {
|
||||
t.Fatalf("unlocked keyspace produced wrong error: %v", err)
|
||||
}
|
||||
ctx = lockedKeyspaceContext("ks")
|
||||
|
||||
// add one cell
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"first"}, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first"},
|
||||
|
@ -145,21 +172,21 @@ func TestUpdateDisableQueryService(t *testing.T) {
|
|||
}
|
||||
|
||||
// remove that cell, going back
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"first"}, false); err != nil || len(si.TabletControls) != 0 {
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, false); err != nil || len(si.TabletControls) != 0 {
|
||||
t.Fatalf("going back should have remove the record: %v %v", err, si)
|
||||
}
|
||||
|
||||
// re-add a cell, then another with a table list to
|
||||
// make sure it fails
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"first"}, true); err != nil {
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"first"}, true); err != nil {
|
||||
t.Fatalf("one cell add failed: %v", si)
|
||||
}
|
||||
if err := si.UpdateSourceBlacklistedTables(topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t1", "t1"}); err == nil || err.Error() != "cannot safely alter BlacklistedTables as DisableQueryService is set for shard ks/sh" {
|
||||
if err := si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, false, []string{"t1", "t1"}); err == nil || err.Error() != "cannot safely alter BlacklistedTables as DisableQueryService is set for shard ks/sh" {
|
||||
t.Fatalf("UpdateSourceBlacklistedTables should fail: %v", err)
|
||||
}
|
||||
|
||||
// add another cell, see the list grow
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"second"}, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first", "second"},
|
||||
|
@ -170,7 +197,7 @@ func TestUpdateDisableQueryService(t *testing.T) {
|
|||
}
|
||||
|
||||
// add all cells, see the list grow to all
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, nil, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, nil, true); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: nil,
|
||||
|
@ -181,7 +208,7 @@ func TestUpdateDisableQueryService(t *testing.T) {
|
|||
}
|
||||
|
||||
// remove one cell from the full list
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_RDONLY, []string{"second"}, false); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
if err := si.UpdateDisableQueryService(ctx, topodatapb.TabletType_RDONLY, []string{"second"}, false); err != nil || !reflect.DeepEqual(si.TabletControls, []*topodatapb.Shard_TabletControl{
|
||||
{
|
||||
TabletType: topodatapb.TabletType_RDONLY,
|
||||
Cells: []string{"first", "third"},
|
||||
|
|
|
@ -113,8 +113,8 @@ func CheckShard(ctx context.Context, t *testing.T, ts topo.Impl) {
|
|||
}
|
||||
|
||||
other := &topodatapb.TabletAlias{Cell: "ny", Uid: 82873}
|
||||
_, err = tts.UpdateShardFields(ctx, "test_keyspace", "b0-c0", func(shard *topodatapb.Shard) error {
|
||||
shard.MasterAlias = other
|
||||
_, err = tts.UpdateShardFields(ctx, "test_keyspace", "b0-c0", func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = other
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
"fmt"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"golang.org/x/net/context"
|
||||
|
@ -19,15 +18,14 @@ import (
|
|||
)
|
||||
|
||||
// RebuildKeyspace rebuilds the serving graph data while locking out other changes.
|
||||
func RebuildKeyspace(ctx context.Context, log logutil.Logger, ts topo.Server, keyspace string, cells []string) error {
|
||||
node := actionnode.RebuildKeyspace()
|
||||
lockPath, err := node.LockKeyspace(ctx, ts, keyspace)
|
||||
if err != nil {
|
||||
return err
|
||||
func RebuildKeyspace(ctx context.Context, log logutil.Logger, ts topo.Server, keyspace string, cells []string) (err error) {
|
||||
ctx, unlock, lockErr := ts.LockKeyspace(ctx, keyspace, "RebuildKeyspace")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = rebuildKeyspace(ctx, log, ts, keyspace, cells)
|
||||
return node.UnlockKeyspace(ctx, ts, keyspace, lockPath, err)
|
||||
return RebuildKeyspaceLocked(ctx, log, ts, keyspace, cells)
|
||||
}
|
||||
|
||||
// findCellsForRebuild will find all the cells in the given keyspace
|
||||
|
@ -49,14 +47,17 @@ func findCellsForRebuild(ki *topo.KeyspaceInfo, shardMap map[string]*topo.ShardI
|
|||
}
|
||||
}
|
||||
|
||||
// rebuildKeyspace should only be used with an action lock on the keyspace
|
||||
// RebuildKeyspaceLocked should only be used with an action lock on the keyspace
|
||||
// - otherwise the consistency of the serving graph data can't be
|
||||
// guaranteed.
|
||||
//
|
||||
// Take data from the global keyspace and rebuild the local serving
|
||||
// copies in each cell.
|
||||
func rebuildKeyspace(ctx context.Context, log logutil.Logger, ts topo.Server, keyspace string, cells []string) error {
|
||||
func RebuildKeyspaceLocked(ctx context.Context, log logutil.Logger, ts topo.Server, keyspace string, cells []string) error {
|
||||
log.Infof("rebuildKeyspace %v", keyspace)
|
||||
if err := topo.CheckKeyspaceLocked(ctx, keyspace); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
ki, err := ts.GetKeyspace(ctx, keyspace)
|
||||
if err != nil {
|
||||
|
|
|
@ -13,7 +13,6 @@ import (
|
|||
"golang.org/x/net/context"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
|
@ -23,20 +22,16 @@ import (
|
|||
// that they have a new master, and also tell all the masters. The
|
||||
// masters will be scrapped if they don't answer.
|
||||
// We execute all the actions in parallel.
|
||||
func RestartSlavesExternal(ts topo.Server, log logutil.Logger, slaveTabletMap, masterTabletMap map[topodatapb.TabletAlias]*topo.TabletInfo, masterElectTabletAlias *topodatapb.TabletAlias, slaveWasRestarted func(*topo.TabletInfo, *actionnode.SlaveWasRestartedArgs) error) {
|
||||
func RestartSlavesExternal(ts topo.Server, log logutil.Logger, slaveTabletMap, masterTabletMap map[topodatapb.TabletAlias]*topo.TabletInfo, masterElectTabletAlias *topodatapb.TabletAlias, slaveWasRestarted func(*topo.TabletInfo, *topodatapb.TabletAlias) error) {
|
||||
wg := sync.WaitGroup{}
|
||||
|
||||
swrd := actionnode.SlaveWasRestartedArgs{
|
||||
Parent: masterElectTabletAlias,
|
||||
}
|
||||
|
||||
log.Infof("Updating individual tablets with the right master...")
|
||||
|
||||
// do all the slaves
|
||||
for _, ti := range slaveTabletMap {
|
||||
wg.Add(1)
|
||||
go func(ti *topo.TabletInfo) {
|
||||
if err := slaveWasRestarted(ti, &swrd); err != nil {
|
||||
if err := slaveWasRestarted(ti, masterElectTabletAlias); err != nil {
|
||||
log.Warningf("Slave %v had an error: %v", ti.Alias, err)
|
||||
}
|
||||
wg.Done()
|
||||
|
@ -47,7 +42,7 @@ func RestartSlavesExternal(ts topo.Server, log logutil.Logger, slaveTabletMap, m
|
|||
for _, ti := range masterTabletMap {
|
||||
wg.Add(1)
|
||||
go func(ti *topo.TabletInfo) {
|
||||
err := slaveWasRestarted(ti, &swrd)
|
||||
err := slaveWasRestarted(ti, masterElectTabletAlias)
|
||||
if err != nil {
|
||||
// the old master can be annoying if left
|
||||
// around in the replication graph, so if we
|
||||
|
|
|
@ -1,65 +0,0 @@
|
|||
// Copyright 2015, Google Inc. All rights reserved.
|
||||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package topotools
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"golang.org/x/net/context"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
// CreateShard will create the shard, while holding the keyspace lock
|
||||
func CreateShard(ctx context.Context, ts topo.Server, keyspace, shard string) error {
|
||||
// Lock the keyspace
|
||||
node := actionnode.KeyspaceCreateShard()
|
||||
lockPath, err := node.LockKeyspace(ctx, ts, keyspace)
|
||||
if err != nil {
|
||||
return fmt.Errorf("LockKeyspace failed: %v", err)
|
||||
}
|
||||
|
||||
// now try to create within the lock, may already exist
|
||||
err = ts.CreateShard(ctx, keyspace, shard)
|
||||
|
||||
// and unlock and return
|
||||
return node.UnlockKeyspace(ctx, ts, keyspace, lockPath, err)
|
||||
}
|
||||
|
||||
// GetOrCreateShard will return the shard object, or create one if it doesn't
|
||||
// already exist. Note the shard creation is protected by a keyspace Lock.
|
||||
func GetOrCreateShard(ctx context.Context, ts topo.Server, keyspace, shard string) (*topo.ShardInfo, error) {
|
||||
si, finalErr := ts.GetShard(ctx, keyspace, shard)
|
||||
if finalErr == topo.ErrNoNode {
|
||||
// create the keyspace, maybe it already exists
|
||||
if err := ts.CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{}); err != nil && err != topo.ErrNodeExists {
|
||||
return nil, fmt.Errorf("CreateKeyspace(%v) failed: %v", keyspace, err)
|
||||
}
|
||||
|
||||
// now we can lock the keyspace
|
||||
node := actionnode.KeyspaceCreateShard()
|
||||
lockPath, err := node.LockKeyspace(ctx, ts, keyspace)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("LockKeyspace failed: %v", err)
|
||||
}
|
||||
|
||||
// now try to create within the lock, may already exist
|
||||
if err := ts.CreateShard(ctx, keyspace, shard); err != nil && err != topo.ErrNodeExists {
|
||||
return nil, node.UnlockKeyspace(ctx, ts, keyspace, lockPath, fmt.Errorf("CreateShard(%v/%v) failed: %v", keyspace, shard, err))
|
||||
}
|
||||
|
||||
// try to read the shard again, maybe someone created it
|
||||
// in between the original GetShard and the LockKeyspace
|
||||
si, finalErr = ts.GetShard(ctx, keyspace, shard)
|
||||
|
||||
// and unlock
|
||||
if err := node.UnlockKeyspace(ctx, ts, keyspace, lockPath, finalErr); err != nil {
|
||||
return nil, fmt.Errorf("UnlockKeyspace failed: %v", err)
|
||||
}
|
||||
}
|
||||
return si, finalErr
|
||||
}
|
|
@ -2,7 +2,7 @@
|
|||
// Use of this source code is governed by a BSD-style
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package topotools_test
|
||||
package topotools
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
@ -13,13 +13,12 @@ import (
|
|||
|
||||
"golang.org/x/net/context"
|
||||
|
||||
. "github.com/youtube/vitess/go/vt/topotools"
|
||||
"github.com/youtube/vitess/go/vt/zktopo/zktestserver"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
)
|
||||
|
||||
// TestCreateShard tests a few cases for CreateShard
|
||||
// TestCreateShard tests a few cases for topo.CreateShard
|
||||
func TestCreateShard(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
cells := []string{"test_cell"}
|
||||
|
@ -31,7 +30,7 @@ func TestCreateShard(t *testing.T) {
|
|||
shard := "0"
|
||||
|
||||
// create shard in a non-existing keyspace
|
||||
if err := CreateShard(ctx, ts, keyspace, shard); err == nil {
|
||||
if err := ts.CreateShard(ctx, keyspace, shard); err == nil {
|
||||
t.Fatalf("CreateShard(invalid keyspace) didn't fail")
|
||||
}
|
||||
|
||||
|
@ -41,7 +40,7 @@ func TestCreateShard(t *testing.T) {
|
|||
}
|
||||
|
||||
// create shard should now work
|
||||
if err := CreateShard(ctx, ts, keyspace, shard); err != nil {
|
||||
if err := ts.CreateShard(ctx, keyspace, shard); err != nil {
|
||||
t.Fatalf("CreateShard failed: %v", err)
|
||||
}
|
||||
}
|
||||
|
@ -63,7 +62,7 @@ func TestCreateShardCustomSharding(t *testing.T) {
|
|||
|
||||
// create first shard in keyspace
|
||||
shard0 := "0"
|
||||
if err := CreateShard(ctx, ts, keyspace, shard0); err != nil {
|
||||
if err := ts.CreateShard(ctx, keyspace, shard0); err != nil {
|
||||
t.Fatalf("CreateShard(shard0) failed: %v", err)
|
||||
}
|
||||
if si, err := ts.GetShard(ctx, keyspace, shard0); err != nil {
|
||||
|
@ -76,7 +75,7 @@ func TestCreateShardCustomSharding(t *testing.T) {
|
|||
|
||||
// create second shard in keyspace
|
||||
shard1 := "1"
|
||||
if err := CreateShard(ctx, ts, keyspace, shard1); err != nil {
|
||||
if err := ts.CreateShard(ctx, keyspace, shard1); err != nil {
|
||||
t.Fatalf("CreateShard(shard1) failed: %v", err)
|
||||
}
|
||||
if si, err := ts.GetShard(ctx, keyspace, shard1); err != nil {
|
||||
|
@ -110,7 +109,7 @@ func TestGetOrCreateShard(t *testing.T) {
|
|||
for j := 0; j < 100; j++ {
|
||||
index := rand.Intn(10)
|
||||
shard := fmt.Sprintf("%v", index)
|
||||
si, err := GetOrCreateShard(ctx, ts, keyspace, shard)
|
||||
si, err := ts.GetOrCreateShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
t.Errorf("GetOrCreateShard(%v, %v) failed: %v", i, shard, err)
|
||||
}
|
||||
|
|
|
@ -85,6 +85,7 @@ import (
|
|||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
log "github.com/golang/glog"
|
||||
|
@ -96,7 +97,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/key"
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"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"
|
||||
|
@ -131,6 +131,10 @@ type commandGroup struct {
|
|||
commands []command
|
||||
}
|
||||
|
||||
// commandsMutex protects commands at init time. We use servenv, which calls
|
||||
// all Run hooks in parallel.
|
||||
var commandsMutex sync.Mutex
|
||||
|
||||
var commands = []commandGroup{
|
||||
{
|
||||
"Tablets", []command{
|
||||
|
@ -381,6 +385,8 @@ func init() {
|
|||
}
|
||||
|
||||
func addCommand(groupName string, c command) {
|
||||
commandsMutex.Lock()
|
||||
defer commandsMutex.Unlock()
|
||||
for i, group := range commands {
|
||||
if group.name == groupName {
|
||||
commands[i].commands = append(commands[i].commands, c)
|
||||
|
@ -391,6 +397,8 @@ func addCommand(groupName string, c command) {
|
|||
}
|
||||
|
||||
func addCommandGroup(groupName string) {
|
||||
commandsMutex.Lock()
|
||||
defer commandsMutex.Unlock()
|
||||
commands = append(commands, commandGroup{
|
||||
name: groupName,
|
||||
})
|
||||
|
@ -419,15 +427,6 @@ func fmtTabletAwkable(ti *topo.TabletInfo) string {
|
|||
return fmt.Sprintf("%v %v %v %v %v %v %v", topoproto.TabletAliasString(ti.Alias), keyspace, shard, strings.ToLower(ti.Type.String()), ti.Addr(), ti.MysqlAddr(), fmtMapAwkable(ti.Tags))
|
||||
}
|
||||
|
||||
func fmtAction(action *actionnode.ActionNode) string {
|
||||
state := string(action.State)
|
||||
// FIXME(msolomon) The default state should really just have the value "queued".
|
||||
if action.State == actionnode.ActionStateQueued {
|
||||
state = "queued"
|
||||
}
|
||||
return fmt.Sprintf("%v %v %v %v %v", action.Path, action.Action, state, action.ActionGuid, action.Error)
|
||||
}
|
||||
|
||||
func listTabletsByShard(ctx context.Context, wr *wrangler.Wrangler, keyspace, shard string) error {
|
||||
tabletAliases, err := wr.TopoServer().FindAllTabletAliasesInShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
|
@ -1061,7 +1060,7 @@ func commandCreateShard(ctx context.Context, wr *wrangler.Wrangler, subFlags *fl
|
|||
}
|
||||
}
|
||||
|
||||
err = topotools.CreateShard(ctx, wr.TopoServer(), keyspace, shard)
|
||||
err = wr.TopoServer().CreateShard(ctx, keyspace, shard)
|
||||
if *force && err == topo.ErrNodeExists {
|
||||
log.Infof("shard %v/%v already exists (ignoring error with -force)", keyspace, shard)
|
||||
err = nil
|
||||
|
|
|
@ -13,7 +13,6 @@ import (
|
|||
"github.com/youtube/vitess/go/event"
|
||||
"github.com/youtube/vitess/go/vt/concurrency"
|
||||
"github.com/youtube/vitess/go/vt/discovery"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/topotools"
|
||||
|
@ -26,29 +25,17 @@ import (
|
|||
|
||||
// keyspace related methods for Wrangler
|
||||
|
||||
func (wr *Wrangler) lockKeyspace(ctx context.Context, keyspace string, actionNode *actionnode.ActionNode) (lockPath string, err error) {
|
||||
return actionNode.LockKeyspace(ctx, wr.ts, keyspace)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) unlockKeyspace(ctx context.Context, keyspace string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error {
|
||||
return actionNode.UnlockKeyspace(ctx, wr.ts, keyspace, lockPath, actionError)
|
||||
}
|
||||
|
||||
// SetKeyspaceShardingInfo locks a keyspace and sets its ShardingColumnName
|
||||
// and ShardingColumnType
|
||||
func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType topodatapb.KeyspaceIdType, force bool) error {
|
||||
actionNode := actionnode.SetKeyspaceShardingInfo()
|
||||
lockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
func (wr *Wrangler) SetKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType topodatapb.KeyspaceIdType, force bool) (err error) {
|
||||
// Lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, "SetKeyspaceShardingInfo")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.setKeyspaceShardingInfo(ctx, keyspace, shardingColumnName, shardingColumnType, force)
|
||||
return wr.unlockKeyspace(ctx, keyspace, actionNode, lockPath, err)
|
||||
|
||||
}
|
||||
|
||||
func (wr *Wrangler) setKeyspaceShardingInfo(ctx context.Context, keyspace, shardingColumnName string, shardingColumnType topodatapb.KeyspaceIdType, force bool) error {
|
||||
// and change it
|
||||
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -77,7 +64,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 topodatapb.TabletType, reverse, skipReFreshState bool, filteredReplicationWaitTime time.Duration) error {
|
||||
func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType topodatapb.TabletType, reverse, skipReFreshState bool, filteredReplicationWaitTime time.Duration) (err error) {
|
||||
// check input parameters
|
||||
if servedType == topodatapb.TabletType_MASTER {
|
||||
// we cannot migrate a master back, since when master migration
|
||||
// is done, the source shards are dead
|
||||
|
@ -90,6 +78,13 @@ func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard stri
|
|||
}
|
||||
}
|
||||
|
||||
// lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, fmt.Sprintf("MigrateServedTypes(%v)", servedType))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// find overlapping shards in this keyspace
|
||||
wr.Logger().Infof("Finding the overlapping shards in keyspace %v", keyspace)
|
||||
osList, err := topotools.FindOverlappingShards(ctx, wr.ts, keyspace)
|
||||
|
@ -132,50 +127,22 @@ func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard stri
|
|||
}
|
||||
}
|
||||
|
||||
// lock the shards: sources, then destinations
|
||||
// (note they're all ordered by shard name)
|
||||
actionNode := actionnode.MigrateServedTypes(servedType)
|
||||
sourceLockPath := make([]string, len(sourceShards))
|
||||
for i, si := range sourceShards {
|
||||
sourceLockPath[i], err = wr.lockShard(ctx, si.Keyspace(), si.ShardName(), actionNode)
|
||||
if err != nil {
|
||||
wr.Logger().Errorf("Failed to lock source shard %v/%v, may need to unlock other shards manually", si.Keyspace(), si.ShardName())
|
||||
return err
|
||||
}
|
||||
}
|
||||
destinationLockPath := make([]string, len(destinationShards))
|
||||
for i, si := range destinationShards {
|
||||
destinationLockPath[i], err = wr.lockShard(ctx, si.Keyspace(), si.ShardName(), actionNode)
|
||||
if err != nil {
|
||||
wr.Logger().Errorf("Failed to lock destination shard %v/%v, may need to unlock other shards manually", si.Keyspace(), si.ShardName())
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// record the action error and all unlock errors
|
||||
rec := concurrency.AllErrorRecorder{}
|
||||
|
||||
// execute the migration
|
||||
rec.RecordError(wr.migrateServedTypes(ctx, keyspace, sourceShards, destinationShards, cells, servedType, reverse, filteredReplicationWaitTime))
|
||||
|
||||
// unlock the shards, we're done
|
||||
for i := len(destinationShards) - 1; i >= 0; i-- {
|
||||
rec.RecordError(wr.unlockShard(ctx, destinationShards[i].Keyspace(), destinationShards[i].ShardName(), actionNode, destinationLockPath[i], nil))
|
||||
}
|
||||
for i := len(sourceShards) - 1; i >= 0; i-- {
|
||||
rec.RecordError(wr.unlockShard(ctx, sourceShards[i].Keyspace(), sourceShards[i].ShardName(), actionNode, sourceLockPath[i], nil))
|
||||
if err = wr.migrateServedTypesLocked(ctx, keyspace, sourceShards, destinationShards, cells, servedType, reverse, filteredReplicationWaitTime); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// rebuild the keyspace serving graph if there was no error
|
||||
if !rec.HasErrors() {
|
||||
rec.RecordError(wr.RebuildKeyspaceGraph(ctx, keyspace, cells))
|
||||
// rebuild the keyspace serving graph now that there is no error
|
||||
if err = topotools.RebuildKeyspaceLocked(ctx, wr.logger, wr.ts, keyspace, cells); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Send a refresh to the tablets we just disabled, iff:
|
||||
// - we're not migrating a master
|
||||
// - we don't have any errors
|
||||
// - we're not told to skip the refresh
|
||||
if servedType != topodatapb.TabletType_MASTER && !rec.HasErrors() && !skipReFreshState {
|
||||
if servedType != topodatapb.TabletType_MASTER && !skipReFreshState {
|
||||
rec := concurrency.AllErrorRecorder{}
|
||||
var refreshShards []*topo.ShardInfo
|
||||
if reverse {
|
||||
// For a backwards migration, we just disabled query service on the destination shards
|
||||
|
@ -185,11 +152,12 @@ func (wr *Wrangler) MigrateServedTypes(ctx context.Context, keyspace, shard stri
|
|||
refreshShards = sourceShards
|
||||
}
|
||||
for _, si := range refreshShards {
|
||||
rec.RecordError(wr.RefreshTablesByShard(ctx, si, servedType, cells))
|
||||
rec.RecordError(wr.RefreshTabletsByShard(ctx, si, servedType, cells))
|
||||
}
|
||||
return rec.Error()
|
||||
}
|
||||
|
||||
return rec.Error()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (wr *Wrangler) getMastersPosition(ctx context.Context, shards []*topo.ShardInfo) (map[*topo.ShardInfo]string, error) {
|
||||
|
@ -291,8 +259,8 @@ func (wr *Wrangler) refreshMasters(ctx context.Context, shards []*topo.ShardInfo
|
|||
return rec.Error()
|
||||
}
|
||||
|
||||
// migrateServedTypes operates with all concerned shards locked.
|
||||
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) {
|
||||
// migrateServedTypesLocked operates with the keyspace locked
|
||||
func (wr *Wrangler) migrateServedTypesLocked(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")
|
||||
|
@ -325,14 +293,15 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
|
|||
// - switch the source shards to read-only by disabling query service
|
||||
// - gather all replication points
|
||||
// - wait for filtered replication to catch up before we continue
|
||||
// - disable filtered replication after the fact
|
||||
// - we will disable filtered replication after the fact in the
|
||||
// next phases
|
||||
if servedType == topodatapb.TabletType_MASTER {
|
||||
event.DispatchUpdate(ev, "disabling query service on all source masters")
|
||||
for _, si := range sourceShards {
|
||||
if err := si.UpdateDisableQueryService(topodatapb.TabletType_MASTER, nil, true); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := wr.ts.UpdateShard(ctx, si); err != nil {
|
||||
for i, si := range sourceShards {
|
||||
// update our internal record too
|
||||
if sourceShards[i], err = wr.ts.UpdateShardFields(ctx, si.Keyspace(), si.ShardName(), func(si *topo.ShardInfo) error {
|
||||
return si.UpdateDisableQueryService(ctx, topodatapb.TabletType_MASTER, nil, true)
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -350,87 +319,95 @@ func (wr *Wrangler) migrateServedTypes(ctx context.Context, keyspace string, sou
|
|||
if err := wr.waitForFilteredReplication(ctx, masterPositions, destinationShards, filteredReplicationWaitTime); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, si := range destinationShards {
|
||||
si.SourceShards = nil
|
||||
}
|
||||
}
|
||||
|
||||
// Check and update all shard records, in memory only.
|
||||
// Check and update all source shard records.
|
||||
// We remember if we need to refresh the state of the source tablets
|
||||
// so their query service is enabled again, for reverse migration.
|
||||
needToRefreshSourceTablets := false
|
||||
for _, si := range sourceShards {
|
||||
if err := si.UpdateServedTypesMap(servedType, cells, !reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
if tc := si.GetTabletControl(servedType); reverse && tc != nil && tc.DisableQueryService {
|
||||
// this is a backward migration, where the
|
||||
// source tablets were disabled previously, so
|
||||
// we need to refresh them
|
||||
if err := si.UpdateDisableQueryService(servedType, cells, false); err != nil {
|
||||
return err
|
||||
}
|
||||
needToRefreshSourceTablets = true
|
||||
}
|
||||
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)
|
||||
if err := si.UpdateDisableQueryService(servedType, cells, true); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
// We remember if we need to refresh the state of the destination tablets
|
||||
// so their query service will be enabled.
|
||||
needToRefreshDestinationTablets := false
|
||||
for _, si := range destinationShards {
|
||||
if err := si.UpdateServedTypesMap(servedType, cells, reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
if tc := si.GetTabletControl(servedType); !reverse && tc != nil && tc.DisableQueryService {
|
||||
// This is a forwards migration, and the destination query service was already in a disabled state.
|
||||
// We need to enable and force a refresh, otherwise it's possible that both the source and destination
|
||||
// will have query service disabled at the same time, and queries would have nowhere to go.
|
||||
if err := si.UpdateDisableQueryService(servedType, cells, false); err != nil {
|
||||
return err
|
||||
}
|
||||
needToRefreshDestinationTablets = true
|
||||
}
|
||||
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)
|
||||
if err := si.UpdateDisableQueryService(servedType, cells, true); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// All is good, we can save the shards now
|
||||
event.DispatchUpdate(ev, "updating source shards")
|
||||
for _, si := range sourceShards {
|
||||
if err := wr.ts.UpdateShard(ctx, si); err != nil {
|
||||
needToRefreshSourceTablets := false
|
||||
for i, si := range sourceShards {
|
||||
sourceShards[i], err = wr.ts.UpdateShardFields(ctx, si.Keyspace(), si.ShardName(), func(si *topo.ShardInfo) error {
|
||||
if err := si.UpdateServedTypesMap(servedType, cells, !reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
if tc := si.GetTabletControl(servedType); reverse && tc != nil && tc.DisableQueryService {
|
||||
// this is a backward migration, where the
|
||||
// source tablets were disabled previously, so
|
||||
// we need to refresh them
|
||||
if err := si.UpdateDisableQueryService(ctx, servedType, cells, false); err != nil {
|
||||
return err
|
||||
}
|
||||
needToRefreshSourceTablets = true
|
||||
}
|
||||
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)
|
||||
if err := si.UpdateDisableQueryService(ctx, servedType, cells, true); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if needToRefreshSourceTablets {
|
||||
event.DispatchUpdate(ev, "refreshing source shard tablets so they restart their query service")
|
||||
for _, si := range sourceShards {
|
||||
wr.RefreshTablesByShard(ctx, si, servedType, cells)
|
||||
wr.RefreshTabletsByShard(ctx, si, servedType, cells)
|
||||
}
|
||||
}
|
||||
|
||||
// We remember if we need to refresh the state of the
|
||||
// destination tablets so their query service will be enabled.
|
||||
event.DispatchUpdate(ev, "updating destination shards")
|
||||
for _, si := range destinationShards {
|
||||
if err := wr.ts.UpdateShard(ctx, si); err != nil {
|
||||
needToRefreshDestinationTablets := false
|
||||
for i, si := range destinationShards {
|
||||
destinationShards[i], err = wr.ts.UpdateShardFields(ctx, si.Keyspace(), si.ShardName(), func(si *topo.ShardInfo) error {
|
||||
if err := si.UpdateServedTypesMap(servedType, cells, reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
if tc := si.GetTabletControl(servedType); !reverse && tc != nil && tc.DisableQueryService {
|
||||
// This is a forwards migration, and the
|
||||
// destination query service was already in a
|
||||
// disabled state. We need to enable and force
|
||||
// a refresh, otherwise it's possible that both
|
||||
// the source and destination will have query
|
||||
// service disabled at the same time, and
|
||||
// queries would have nowhere to go.
|
||||
if err := si.UpdateDisableQueryService(ctx, servedType, cells, false); err != nil {
|
||||
return err
|
||||
}
|
||||
needToRefreshDestinationTablets = true
|
||||
}
|
||||
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).
|
||||
if err := si.UpdateDisableQueryService(ctx, servedType, cells, true); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// for master migration, also disable filtered
|
||||
// replication
|
||||
if servedType == topodatapb.TabletType_MASTER {
|
||||
si.SourceShards = nil
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if needToRefreshDestinationTablets {
|
||||
event.DispatchUpdate(ev, "refreshing destination shard tablets so they restart their query service")
|
||||
for _, si := range destinationShards {
|
||||
wr.RefreshTablesByShard(ctx, si, servedType, cells)
|
||||
wr.RefreshTabletsByShard(ctx, si, servedType, cells)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -558,7 +535,7 @@ func formatTabletStats(ts *discovery.TabletStats) string {
|
|||
|
||||
// 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 topodatapb.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) (err error) {
|
||||
// read the destination keyspace, check it
|
||||
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
|
||||
if err != nil {
|
||||
|
@ -579,52 +556,35 @@ func (wr *Wrangler) MigrateServedFrom(ctx context.Context, keyspace, shard strin
|
|||
|
||||
// check the migration is valid before locking (will also be checked
|
||||
// after locking to be sure)
|
||||
if err := ki.CheckServedFromMigration(servedType, cells, si.SourceShards[0].Keyspace, !reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// lock the keyspace and shards
|
||||
actionNode := actionnode.MigrateServedFrom(servedType)
|
||||
keyspaceLockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
|
||||
if err != nil {
|
||||
wr.Logger().Errorf("Failed to lock destination keyspace %v", keyspace)
|
||||
return err
|
||||
}
|
||||
destinationShardLockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
wr.Logger().Errorf("Failed to lock destination shard %v/%v", keyspace, shard)
|
||||
wr.unlockKeyspace(ctx, keyspace, actionNode, keyspaceLockPath, nil)
|
||||
return err
|
||||
}
|
||||
sourceKeyspace := si.SourceShards[0].Keyspace
|
||||
sourceShard := si.SourceShards[0].Shard
|
||||
sourceShardLockPath, err := wr.lockShard(ctx, sourceKeyspace, sourceShard, actionNode)
|
||||
if err != nil {
|
||||
wr.Logger().Errorf("Failed to lock source shard %v/%v", sourceKeyspace, sourceShard)
|
||||
wr.unlockShard(ctx, keyspace, shard, actionNode, destinationShardLockPath, nil)
|
||||
wr.unlockKeyspace(ctx, keyspace, actionNode, keyspaceLockPath, nil)
|
||||
if err := ki.CheckServedFromMigration(servedType, cells, sourceKeyspace, !reverse); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// record the action error and all unlock errors
|
||||
rec := concurrency.AllErrorRecorder{}
|
||||
// lock the keyspaces, source first.
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, sourceKeyspace, fmt.Sprintf("MigrateServedFrom(%v)", servedType))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
ctx, unlock, lockErr = wr.ts.LockKeyspace(ctx, keyspace, fmt.Sprintf("MigrateServedFrom(%v)", servedType))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// execute the migration
|
||||
rec.RecordError(wr.migrateServedFrom(ctx, ki, si, servedType, cells, reverse, filteredReplicationWaitTime))
|
||||
|
||||
rec.RecordError(wr.unlockShard(ctx, sourceKeyspace, sourceShard, actionNode, sourceShardLockPath, nil))
|
||||
rec.RecordError(wr.unlockShard(ctx, keyspace, shard, actionNode, destinationShardLockPath, nil))
|
||||
rec.RecordError(wr.unlockKeyspace(ctx, keyspace, actionNode, keyspaceLockPath, nil))
|
||||
err = wr.migrateServedFromLocked(ctx, ki, si, servedType, cells, reverse, filteredReplicationWaitTime)
|
||||
|
||||
// rebuild the keyspace serving graph if there was no error
|
||||
if rec.Error() == nil {
|
||||
rec.RecordError(wr.RebuildKeyspaceGraph(ctx, keyspace, cells))
|
||||
if err == nil {
|
||||
err = topotools.RebuildKeyspaceLocked(ctx, wr.logger, wr.ts, keyspace, cells)
|
||||
}
|
||||
|
||||
return rec.Error()
|
||||
return err
|
||||
}
|
||||
|
||||
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) {
|
||||
func (wr *Wrangler) migrateServedFromLocked(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())
|
||||
|
@ -688,17 +648,16 @@ func (wr *Wrangler) replicaMigrateServedFrom(ctx context.Context, ki *topo.Keysp
|
|||
|
||||
// Save the source shard (its blacklisted tables field has changed)
|
||||
event.DispatchUpdate(ev, "updating source shard")
|
||||
if err := sourceShard.UpdateSourceBlacklistedTables(servedType, cells, reverse, 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 {
|
||||
return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, sourceShard.Keyspace(), sourceShard.ShardName(), func(si *topo.ShardInfo) error {
|
||||
return si.UpdateSourceBlacklistedTables(ctx, servedType, cells, reverse, tables)
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Now refresh the source servers so they reload their
|
||||
// blacklisted table list
|
||||
event.DispatchUpdate(ev, "refreshing sources tablets state so they update their blacklisted tables")
|
||||
if err := wr.RefreshTablesByShard(ctx, sourceShard, servedType, cells); err != nil {
|
||||
if err := wr.RefreshTabletsByShard(ctx, sourceShard, servedType, cells); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -728,11 +687,10 @@ 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(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 {
|
||||
return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, sourceShard.Keyspace(), sourceShard.ShardName(), func(si *topo.ShardInfo) error {
|
||||
return si.UpdateSourceBlacklistedTables(ctx, topodatapb.TabletType_MASTER, nil, false, tables)
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Now refresh the blacklisted table list on the source master
|
||||
|
@ -765,8 +723,14 @@ func (wr *Wrangler) masterMigrateServedFrom(ctx context.Context, ki *topo.Keyspa
|
|||
|
||||
// Update the destination shard (no more source shard)
|
||||
event.DispatchUpdate(ev, "updating destination shard")
|
||||
destinationShard.SourceShards = nil
|
||||
if err := wr.ts.UpdateShard(ctx, destinationShard); err != nil {
|
||||
destinationShard, err = wr.ts.UpdateShardFields(ctx, destinationShard.Keyspace(), destinationShard.ShardName(), func(si *topo.ShardInfo) error {
|
||||
if len(si.SourceShards) != 1 {
|
||||
return fmt.Errorf("unexpected concurrent access for destination shard %v/%v SourceShards array", si.Keyspace(), si.ShardName())
|
||||
}
|
||||
si.SourceShards = nil
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -782,18 +746,15 @@ 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 topodatapb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
|
||||
actionNode := actionnode.SetKeyspaceServedFrom()
|
||||
lockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
func (wr *Wrangler) SetKeyspaceServedFrom(ctx context.Context, keyspace string, servedType topodatapb.TabletType, cells []string, sourceKeyspace string, remove bool) (err error) {
|
||||
// Lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, "SetKeyspaceServedFrom")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.setKeyspaceServedFrom(ctx, keyspace, servedType, cells, sourceKeyspace, remove)
|
||||
return wr.unlockKeyspace(ctx, keyspace, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) setKeyspaceServedFrom(ctx context.Context, keyspace string, servedType topodatapb.TabletType, cells []string, sourceKeyspace string, remove bool) error {
|
||||
// and update it
|
||||
ki, err := wr.ts.GetKeyspace(ctx, keyspace)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -804,17 +765,17 @@ func (wr *Wrangler) setKeyspaceServedFrom(ctx context.Context, keyspace string,
|
|||
return wr.ts.UpdateKeyspace(ctx, ki)
|
||||
}
|
||||
|
||||
// RefreshTablesByShard calls RefreshState on all the tables of a
|
||||
// RefreshTabletsByShard 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 topodatapb.TabletType, cells []string) error {
|
||||
wr.Logger().Infof("RefreshTablesByShard called on shard %v/%v", si.Keyspace(), si.ShardName())
|
||||
func (wr *Wrangler) RefreshTabletsByShard(ctx context.Context, si *topo.ShardInfo, tabletType topodatapb.TabletType, cells []string) error {
|
||||
wr.Logger().Infof("RefreshTabletsByShard called on shard %v/%v", si.Keyspace(), si.ShardName())
|
||||
tabletMap, err := wr.ts.GetTabletMapForShardByCell(ctx, si.Keyspace(), si.ShardName(), cells)
|
||||
switch err {
|
||||
case nil:
|
||||
// keep going
|
||||
case topo.ErrPartialResult:
|
||||
wr.Logger().Warningf("RefreshTablesByShard: got partial result for shard %v/%v, may not refresh all tablets everywhere", si.Keyspace(), si.ShardName())
|
||||
wr.Logger().Warningf("RefreshTabletsByShard: got partial result for shard %v/%v, may not refresh all tablets everywhere", si.Keyspace(), si.ShardName())
|
||||
default:
|
||||
return err
|
||||
}
|
||||
|
@ -834,7 +795,7 @@ func (wr *Wrangler) RefreshTablesByShard(ctx context.Context, si *topo.ShardInfo
|
|||
// (RefreshState will restart the tablet's QueryService and most time will be spent on the shutdown, i.e. waiting up to 30 seconds on transactions (see Config.TransactionTimeout)).
|
||||
ctx, cancel := context.WithTimeout(ctx, 60*time.Second)
|
||||
if err := wr.tmc.RefreshState(ctx, ti.Tablet); err != nil {
|
||||
wr.Logger().Warningf("RefreshTablesByShard: failed to refresh %v: %v", ti.AliasString(), err)
|
||||
wr.Logger().Warningf("RefreshTabletsByShard: failed to refresh %v: %v", ti.AliasString(), err)
|
||||
}
|
||||
cancel()
|
||||
wg.Done()
|
||||
|
|
|
@ -16,7 +16,6 @@ import (
|
|||
"github.com/youtube/vitess/go/event"
|
||||
"github.com/youtube/vitess/go/vt/concurrency"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/topotools"
|
||||
|
@ -33,49 +32,15 @@ const (
|
|||
emergencyReparentShardOperation = "EmergencyReparentShard"
|
||||
)
|
||||
|
||||
// FIXME(alainjobart) rework this ShardReplicationStatuses function,
|
||||
// it's clumpsy
|
||||
|
||||
// helper struct to queue up results
|
||||
type rpcContext struct {
|
||||
tablet *topo.TabletInfo
|
||||
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, []*replicationdatapb.Status, error) {
|
||||
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
// lock the shard
|
||||
actionNode := actionnode.CheckShard()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
tabletMap, posMap, err := wr.shardReplicationStatuses(ctx, shardInfo)
|
||||
return tabletMap, posMap, wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
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())
|
||||
tabletMap, err := wr.ts.GetTabletMapForShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
tablets := topotools.CopyMapValues(tabletMap, []*topo.TabletInfo{}).([]*topo.TabletInfo)
|
||||
stats, err := wr.tabletReplicationStatuses(ctx, tablets)
|
||||
return tablets, stats, err
|
||||
}
|
||||
|
||||
// tabletReplicationStatuses returns the ReplicationStatus of each tablet in
|
||||
// tablets.
|
||||
func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*topo.TabletInfo) ([]*replicationdatapb.Status, error) {
|
||||
wr.logger.Infof("tabletReplicationStatuses: %v", tablets)
|
||||
wr.logger.Infof("Gathering tablet replication status for: %v", tablets)
|
||||
wg := sync.WaitGroup{}
|
||||
rec := concurrency.AllErrorRecorder{}
|
||||
result := make([]*replicationdatapb.Status, len(tablets))
|
||||
|
@ -110,7 +75,7 @@ func (wr *Wrangler) tabletReplicationStatuses(ctx context.Context, tablets []*to
|
|||
}
|
||||
}
|
||||
wg.Wait()
|
||||
return result, rec.Error()
|
||||
return tablets, result, rec.Error()
|
||||
}
|
||||
|
||||
// ReparentTablet tells a tablet to reparent this tablet to the current
|
||||
|
@ -152,13 +117,13 @@ func (wr *Wrangler) ReparentTablet(ctx context.Context, tabletAlias *topodatapb.
|
|||
}
|
||||
|
||||
// InitShardMaster will make the provided tablet the master for the shard.
|
||||
func (wr *Wrangler) InitShardMaster(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.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) (err error) {
|
||||
// lock the shard
|
||||
actionNode := actionnode.ReparentShard(initShardMasterOperation, masterElectTabletAlias)
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
ctx, unlock, lockErr := wr.ts.LockShard(ctx, keyspace, shard, fmt.Sprintf("InitShardMaster(%v)", topoproto.TabletAliasString(masterElectTabletAlias)))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// Create reusable Reparent event with available info
|
||||
ev := &events.Reparent{}
|
||||
|
@ -170,9 +135,7 @@ func (wr *Wrangler) InitShardMaster(ctx context.Context, keyspace, shard string,
|
|||
} else {
|
||||
event.DispatchUpdate(ev, "finished InitShardMaster")
|
||||
}
|
||||
|
||||
// and unlock
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
return err
|
||||
}
|
||||
|
||||
func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, force bool, waitSlaveTimeout time.Duration) error {
|
||||
|
@ -292,8 +255,8 @@ func (wr *Wrangler) initShardMasterLocked(ctx context.Context, ev *events.Repare
|
|||
return fmt.Errorf("failed to PopulateReparentJournal on master: %v", masterErr)
|
||||
}
|
||||
if !topoproto.TabletAliasEqual(shardInfo.MasterAlias, masterElectTabletAlias) {
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(s *topodatapb.Shard) error {
|
||||
s.MasterAlias = masterElectTabletAlias
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = masterElectTabletAlias
|
||||
return nil
|
||||
}); err != nil {
|
||||
wgSlaves.Wait()
|
||||
|
@ -325,13 +288,13 @@ 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 *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
|
||||
func (wr *Wrangler) PlannedReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) (err error) {
|
||||
// lock the shard
|
||||
actionNode := actionnode.ReparentShard(plannedReparentShardOperation, masterElectTabletAlias)
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
ctx, unlock, lockErr := wr.ts.LockShard(ctx, keyspace, shard, fmt.Sprintf("PlannedReparentShard(%v)", topoproto.TabletAliasString(masterElectTabletAlias)))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// Create reusable Reparent event with available info
|
||||
ev := &events.Reparent{}
|
||||
|
@ -343,9 +306,7 @@ func (wr *Wrangler) PlannedReparentShard(ctx context.Context, keyspace, shard st
|
|||
} else {
|
||||
event.DispatchUpdate(ev, "finished PlannedReparentShard")
|
||||
}
|
||||
|
||||
// and unlock
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
return err
|
||||
}
|
||||
|
||||
func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
|
||||
|
@ -433,8 +394,8 @@ func (wr *Wrangler) plannedReparentShardLocked(ctx context.Context, ev *events.R
|
|||
return fmt.Errorf("failed to PopulateReparentJournal on master: %v", masterErr)
|
||||
}
|
||||
wr.logger.Infof("updating shard record with new master %v", masterElectTabletAlias)
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(s *topodatapb.Shard) error {
|
||||
s.MasterAlias = masterElectTabletAlias
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = masterElectTabletAlias
|
||||
return nil
|
||||
}); err != nil {
|
||||
wgSlaves.Wait()
|
||||
|
@ -454,13 +415,13 @@ 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 *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
|
||||
func (wr *Wrangler) EmergencyReparentShard(ctx context.Context, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) (err error) {
|
||||
// lock the shard
|
||||
actionNode := actionnode.ReparentShard(emergencyReparentShardOperation, masterElectTabletAlias)
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
ctx, unlock, lockErr := wr.ts.LockShard(ctx, keyspace, shard, fmt.Sprintf("EmergencyReparentShard(%v)", topoproto.TabletAliasString(masterElectTabletAlias)))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// Create reusable Reparent event with available info
|
||||
ev := &events.Reparent{}
|
||||
|
@ -472,9 +433,7 @@ func (wr *Wrangler) EmergencyReparentShard(ctx context.Context, keyspace, shard
|
|||
} else {
|
||||
event.DispatchUpdate(ev, "finished EmergencyReparentShard")
|
||||
}
|
||||
|
||||
// and unlock
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
return err
|
||||
}
|
||||
|
||||
func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events.Reparent, keyspace, shard string, masterElectTabletAlias *topodatapb.TabletAlias, waitSlaveTimeout time.Duration) error {
|
||||
|
@ -626,8 +585,8 @@ func (wr *Wrangler) emergencyReparentShardLocked(ctx context.Context, ev *events
|
|||
return fmt.Errorf("failed to PopulateReparentJournal on master: %v", masterErr)
|
||||
}
|
||||
wr.logger.Infof("updating shard record with new master %v", topoproto.TabletAliasString(masterElectTabletAlias))
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(s *topodatapb.Shard) error {
|
||||
s.MasterAlias = masterElectTabletAlias
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = masterElectTabletAlias
|
||||
return nil
|
||||
}); err != nil {
|
||||
wgSlaves.Wait()
|
||||
|
|
|
@ -18,7 +18,6 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/concurrency"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
|
||||
"github.com/youtube/vitess/go/vt/schemamanager"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
|
||||
|
@ -195,26 +194,27 @@ func (wr *Wrangler) PreflightSchema(ctx context.Context, tabletAlias *topodatapb
|
|||
}
|
||||
|
||||
// ApplySchemaKeyspace applies a schema change to an entire keyspace.
|
||||
// take a keyspace lock to do this.
|
||||
// first we will validate the Preflight works the same on all shard masters
|
||||
// and fail if not (unless force is specified)
|
||||
func (wr *Wrangler) ApplySchemaKeyspace(ctx context.Context, keyspace, change string, allowLongUnavailability bool, waitSlaveTimeout time.Duration) error {
|
||||
actionNode := actionnode.ApplySchemaKeyspace(change)
|
||||
lockPath, err := wr.lockKeyspace(ctx, keyspace, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
// Takes a keyspace lock to do this.
|
||||
// First we will validate the Preflight works the same on all shard masters
|
||||
// and fail if not (unless force is specified).
|
||||
func (wr *Wrangler) ApplySchemaKeyspace(ctx context.Context, keyspace, change string, allowLongUnavailability bool, waitSlaveTimeout time.Duration) (err error) {
|
||||
// lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, "ApplySchemaKeyspace")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// apply the schema change
|
||||
executor := schemamanager.NewTabletExecutor(wr.tmc, wr.ts)
|
||||
if allowLongUnavailability {
|
||||
executor.AllowBigSchemaChange()
|
||||
}
|
||||
err = schemamanager.Run(
|
||||
return schemamanager.Run(
|
||||
ctx,
|
||||
schemamanager.NewPlainController(change, keyspace),
|
||||
executor,
|
||||
)
|
||||
|
||||
return wr.unlockKeyspace(ctx, keyspace, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
// CopySchemaShardFromShard copies the schema from a source shard to the specified destination shard.
|
||||
|
|
|
@ -7,7 +7,6 @@ package wrangler
|
|||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"golang.org/x/net/context"
|
||||
|
@ -17,14 +16,6 @@ import (
|
|||
|
||||
// shard related methods for Wrangler
|
||||
|
||||
func (wr *Wrangler) lockShard(ctx context.Context, keyspace, shard string, actionNode *actionnode.ActionNode) (lockPath string, err error) {
|
||||
return actionNode.LockShard(ctx, wr.ts, keyspace, shard)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) unlockShard(ctx context.Context, keyspace, shard string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error {
|
||||
return actionNode.UnlockShard(ctx, wr.ts, keyspace, shard, lockPath, actionError)
|
||||
}
|
||||
|
||||
// updateShardCellsAndMaster will update the 'Cells' and possibly
|
||||
// MasterAlias records for the shard, if needed.
|
||||
func (wr *Wrangler) updateShardCellsAndMaster(ctx context.Context, si *topo.ShardInfo, tabletAlias *topodatapb.TabletAlias, tabletType topodatapb.TabletType, allowMasterOverride bool) error {
|
||||
|
@ -42,27 +33,17 @@ func (wr *Wrangler) updateShardCellsAndMaster(ctx context.Context, si *topo.Shar
|
|||
return nil
|
||||
}
|
||||
|
||||
// we do need to update the shard, lock it to not interfere with
|
||||
// reparenting operations.
|
||||
actionNode := actionnode.UpdateShard()
|
||||
keyspace := si.Keyspace()
|
||||
shard := si.ShardName()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// run the update
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(s *topodatapb.Shard) error {
|
||||
_, err := wr.ts.UpdateShardFields(ctx, si.Keyspace(), si.ShardName(), func(s *topo.ShardInfo) error {
|
||||
wasUpdated := false
|
||||
if !topoproto.ShardHasCell(s, tabletAlias.Cell) {
|
||||
if !s.HasCell(tabletAlias.Cell) {
|
||||
s.Cells = append(s.Cells, tabletAlias.Cell)
|
||||
wasUpdated = true
|
||||
}
|
||||
|
||||
if tabletType == topodatapb.TabletType_MASTER && !topoproto.TabletAliasEqual(s.MasterAlias, tabletAlias) {
|
||||
if !topoproto.TabletAliasIsZero(s.MasterAlias) && !allowMasterOverride {
|
||||
return fmt.Errorf("creating this tablet would override old master %v in shard %v/%v", topoproto.TabletAliasString(s.MasterAlias), keyspace, shard)
|
||||
return fmt.Errorf("creating this tablet would override old master %v in shard %v/%v", topoproto.TabletAliasString(s.MasterAlias), si.Keyspace(), si.ShardName())
|
||||
}
|
||||
s.MasterAlias = tabletAlias
|
||||
wasUpdated = true
|
||||
|
@ -73,33 +54,25 @@ func (wr *Wrangler) updateShardCellsAndMaster(ctx context.Context, si *topo.Shar
|
|||
}
|
||||
return nil
|
||||
})
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
return err
|
||||
}
|
||||
|
||||
// SetShardServedTypes changes the ServedTypes parameter of a shard.
|
||||
// It does not rebuild any serving graph or do any consistency check.
|
||||
func (wr *Wrangler) SetShardServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType topodatapb.TabletType, remove bool) error {
|
||||
|
||||
actionNode := actionnode.SetShardServedTypes(cells, servedType)
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
// This is an emergency manual operation.
|
||||
func (wr *Wrangler) SetShardServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType topodatapb.TabletType, remove bool) (err error) {
|
||||
// lock the keyspace to not conflict with resharding operations
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, fmt.Sprintf("SetShardServedTypes(%v,%v,%v)", cells, servedType, remove))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.setShardServedTypes(ctx, keyspace, shard, cells, servedType, remove)
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) setShardServedTypes(ctx context.Context, keyspace, shard string, cells []string, servedType topodatapb.TabletType, remove bool) error {
|
||||
si, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err := si.UpdateServedTypesMap(servedType, cells, remove); err != nil {
|
||||
return err
|
||||
}
|
||||
return wr.ts.UpdateShard(ctx, si)
|
||||
// and update the shard
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
return si.UpdateServedTypesMap(servedType, cells, remove)
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
// SetShardTabletControl changes the TabletControl records
|
||||
|
@ -108,40 +81,32 @@ func (wr *Wrangler) setShardServedTypes(ctx context.Context, keyspace, shard str
|
|||
// - if disableQueryService is set, tables has to be empty
|
||||
// - if disableQueryService is not set, and tables is empty, we remove
|
||||
// the TabletControl record for the cells
|
||||
func (wr *Wrangler) SetShardTabletControl(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, cells []string, remove, disableQueryService bool, tables []string) error {
|
||||
|
||||
//
|
||||
// This takes the keyspace lock as to not interfere with resharding operations.
|
||||
func (wr *Wrangler) SetShardTabletControl(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, cells []string, remove, disableQueryService bool, tables []string) (err error) {
|
||||
// check input
|
||||
if disableQueryService && len(tables) > 0 {
|
||||
return fmt.Errorf("SetShardTabletControl cannot have both DisableQueryService set and tables set")
|
||||
}
|
||||
|
||||
actionNode := actionnode.UpdateShard()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
// lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, "SetShardTabletControl")
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.setShardTabletControl(ctx, keyspace, shard, tabletType, cells, remove, disableQueryService, tables)
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) setShardTabletControl(ctx context.Context, keyspace, shard string, tabletType topodatapb.TabletType, cells []string, remove, disableQueryService bool, tables []string) error {
|
||||
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(tables) == 0 && !remove {
|
||||
// we are setting the DisableQueryService flag only
|
||||
if err := shardInfo.UpdateDisableQueryService(tabletType, cells, disableQueryService); err != nil {
|
||||
return fmt.Errorf("UpdateDisableQueryService(%v/%v) failed: %v", shardInfo.Keyspace(), shardInfo.ShardName(), err)
|
||||
// update the shard
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
if len(tables) == 0 && !remove {
|
||||
// we are setting the DisableQueryService flag only
|
||||
return si.UpdateDisableQueryService(ctx, tabletType, cells, disableQueryService)
|
||||
}
|
||||
} else {
|
||||
|
||||
// we are setting / removing the blacklisted tables only
|
||||
if err := shardInfo.UpdateSourceBlacklistedTables(tabletType, cells, remove, tables); err != nil {
|
||||
return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", shardInfo.Keyspace(), shardInfo.ShardName(), err)
|
||||
}
|
||||
}
|
||||
return wr.ts.UpdateShard(ctx, shardInfo)
|
||||
return si.UpdateSourceBlacklistedTables(ctx, tabletType, cells, remove, tables)
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
// DeleteShard will do all the necessary changes in the topology server
|
||||
|
@ -197,17 +162,6 @@ func (wr *Wrangler) DeleteShard(ctx context.Context, keyspace, shard string, rec
|
|||
// If 'recursive' is specified, it will delete any tablets in the cell/shard,
|
||||
// with the assumption that the tablet processes have already been terminated.
|
||||
func (wr *Wrangler) RemoveShardCell(ctx context.Context, keyspace, shard, cell string, force, recursive bool) error {
|
||||
actionNode := actionnode.UpdateShard()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
err = wr.removeShardCell(ctx, keyspace, shard, cell, force, recursive)
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) removeShardCell(ctx context.Context, keyspace, shard, cell string, force, recursive bool) error {
|
||||
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -260,81 +214,77 @@ func (wr *Wrangler) removeShardCell(ctx context.Context, keyspace, shard, cell s
|
|||
|
||||
// now we can update the shard
|
||||
wr.Logger().Infof("Removing cell %v from shard %v/%v", cell, keyspace, shard)
|
||||
newCells := make([]string, 0, len(shardInfo.Cells)-1)
|
||||
for _, c := range shardInfo.Cells {
|
||||
if c != cell {
|
||||
newCells = append(newCells, c)
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
// since no lock is taken, protect against corner cases.
|
||||
if len(si.Cells) == 0 {
|
||||
return topo.ErrNoUpdateNeeded
|
||||
}
|
||||
}
|
||||
shardInfo.Cells = newCells
|
||||
|
||||
return wr.ts.UpdateShard(ctx, shardInfo)
|
||||
var newCells []string
|
||||
for _, c := range si.Cells {
|
||||
if c != cell {
|
||||
newCells = append(newCells, c)
|
||||
}
|
||||
}
|
||||
si.Cells = newCells
|
||||
return nil
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
// SourceShardDelete will delete a SourceShard inside a shard, by index.
|
||||
func (wr *Wrangler) SourceShardDelete(ctx context.Context, keyspace, shard string, uid uint32) error {
|
||||
actionNode := actionnode.UpdateShard()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
//
|
||||
// This takes the keyspace lock as not to interfere with resharding operations.
|
||||
func (wr *Wrangler) SourceShardDelete(ctx context.Context, keyspace, shard string, uid uint32) (err error) {
|
||||
// lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, fmt.Sprintf("SourceShardDelete(%v)", uid))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.sourceShardDelete(ctx, keyspace, shard, uid)
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) sourceShardDelete(ctx context.Context, keyspace, shard string, uid uint32) error {
|
||||
si, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
newSourceShards := make([]*topodatapb.Shard_SourceShard, 0, 0)
|
||||
for _, ss := range si.SourceShards {
|
||||
if ss.Uid != uid {
|
||||
newSourceShards = append(newSourceShards, ss)
|
||||
// remove the source shard
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
var newSourceShards []*topodatapb.Shard_SourceShard
|
||||
for _, ss := range si.SourceShards {
|
||||
if ss.Uid != uid {
|
||||
newSourceShards = append(newSourceShards, ss)
|
||||
}
|
||||
}
|
||||
}
|
||||
if len(newSourceShards) == len(si.SourceShards) {
|
||||
return fmt.Errorf("no SourceShard with uid %v", uid)
|
||||
}
|
||||
if len(newSourceShards) == 0 {
|
||||
newSourceShards = nil
|
||||
}
|
||||
si.SourceShards = newSourceShards
|
||||
return wr.ts.UpdateShard(ctx, si)
|
||||
if len(newSourceShards) == len(si.SourceShards) {
|
||||
return fmt.Errorf("no SourceShard with uid %v", uid)
|
||||
}
|
||||
si.SourceShards = newSourceShards
|
||||
return nil
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
// SourceShardAdd will add a new SourceShard inside a shard
|
||||
func (wr *Wrangler) SourceShardAdd(ctx context.Context, keyspace, shard string, uid uint32, skeyspace, sshard string, keyRange *topodatapb.KeyRange, tables []string) error {
|
||||
actionNode := actionnode.UpdateShard()
|
||||
lockPath, err := wr.lockShard(ctx, keyspace, shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
func (wr *Wrangler) SourceShardAdd(ctx context.Context, keyspace, shard string, uid uint32, skeyspace, sshard string, keyRange *topodatapb.KeyRange, tables []string) (err error) {
|
||||
// lock the keyspace
|
||||
ctx, unlock, lockErr := wr.ts.LockKeyspace(ctx, keyspace, fmt.Sprintf("SourceShardAdd(%v)", uid))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
err = wr.sourceShardAdd(ctx, keyspace, shard, uid, skeyspace, sshard, keyRange, tables)
|
||||
return wr.unlockShard(ctx, keyspace, shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
func (wr *Wrangler) sourceShardAdd(ctx context.Context, keyspace, shard string, uid uint32, skeyspace, sshard string, keyRange *topodatapb.KeyRange, tables []string) error {
|
||||
si, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// check the uid is not used already
|
||||
for _, ss := range si.SourceShards {
|
||||
if ss.Uid == uid {
|
||||
return fmt.Errorf("uid %v is already in use", uid)
|
||||
// and update the shard
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
// check the uid is not used already
|
||||
for _, ss := range si.SourceShards {
|
||||
if ss.Uid == uid {
|
||||
return fmt.Errorf("uid %v is already in use", uid)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
si.SourceShards = append(si.SourceShards, &topodatapb.Shard_SourceShard{
|
||||
Uid: uid,
|
||||
Keyspace: skeyspace,
|
||||
Shard: sshard,
|
||||
KeyRange: keyRange,
|
||||
Tables: tables,
|
||||
si.SourceShards = append(si.SourceShards, &topodatapb.Shard_SourceShard{
|
||||
Uid: uid,
|
||||
Keyspace: skeyspace,
|
||||
Shard: sshard,
|
||||
KeyRange: keyRange,
|
||||
Tables: tables,
|
||||
})
|
||||
return nil
|
||||
})
|
||||
return wr.ts.UpdateShard(ctx, si)
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
"golang.org/x/net/context"
|
||||
|
||||
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
)
|
||||
|
||||
// SetSourceShards is a utility function to override the SourceShards fields
|
||||
|
@ -39,14 +40,14 @@ func (wr *Wrangler) SetSourceShards(ctx context.Context, keyspace, shard string,
|
|||
}
|
||||
|
||||
// Update the shard with the new source shards.
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(s *topodatapb.Shard) error {
|
||||
_, err = wr.ts.UpdateShardFields(ctx, keyspace, shard, func(si *topo.ShardInfo) error {
|
||||
// If the shard already has sources, maybe it's already been restored,
|
||||
// so let's be safe and abort right here.
|
||||
if len(s.SourceShards) > 0 {
|
||||
return fmt.Errorf("Shard %v/%v already has SourceShards, not overwriting them (full record: %v)", keyspace, shard, s)
|
||||
if len(si.SourceShards) > 0 {
|
||||
return fmt.Errorf("Shard %v/%v already has SourceShards, not overwriting them (full record: %v)", keyspace, shard, *si.Shard)
|
||||
}
|
||||
|
||||
s.SourceShards = sourceShards
|
||||
si.SourceShards = sourceShards
|
||||
return nil
|
||||
})
|
||||
return err
|
||||
|
|
|
@ -8,7 +8,6 @@ import (
|
|||
"fmt"
|
||||
|
||||
"github.com/youtube/vitess/go/vt/key"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/topotools"
|
||||
|
@ -40,7 +39,7 @@ func (wr *Wrangler) InitTablet(ctx context.Context, tablet *topodatapb.Tablet, a
|
|||
|
||||
if createShardAndKeyspace {
|
||||
// create the parent keyspace and shard if needed
|
||||
si, err = topotools.GetOrCreateShard(ctx, wr.ts, tablet.Keyspace, tablet.Shard)
|
||||
si, err = wr.ts.GetOrCreateShard(ctx, tablet.Keyspace, tablet.Shard)
|
||||
} else {
|
||||
si, err = wr.ts.GetShard(ctx, tablet.Keyspace, tablet.Shard)
|
||||
if err == topo.ErrNoNode {
|
||||
|
@ -94,7 +93,7 @@ func (wr *Wrangler) InitTablet(ctx context.Context, tablet *topodatapb.Tablet, a
|
|||
// DeleteTablet removes a tablet from a shard.
|
||||
// - if allowMaster is set, we can Delete a master tablet (and clear
|
||||
// its record from the Shard record if it was the master).
|
||||
func (wr *Wrangler) DeleteTablet(ctx context.Context, tabletAlias *topodatapb.TabletAlias, allowMaster bool) error {
|
||||
func (wr *Wrangler) DeleteTablet(ctx context.Context, tabletAlias *topodatapb.TabletAlias, allowMaster bool) (err error) {
|
||||
// load the tablet, see if we'll need to rebuild
|
||||
ti, err := wr.ts.GetTablet(ctx, tabletAlias)
|
||||
if err != nil {
|
||||
|
@ -111,30 +110,24 @@ func (wr *Wrangler) DeleteTablet(ctx context.Context, tabletAlias *topodatapb.Ta
|
|||
}
|
||||
|
||||
// update the Shard object if the master was scrapped.
|
||||
// we lock the shard to not conflict with reparent operations.
|
||||
if wasMaster {
|
||||
actionNode := actionnode.UpdateShard()
|
||||
lockPath, err := wr.lockShard(ctx, ti.Keyspace, ti.Shard, actionNode)
|
||||
if err != nil {
|
||||
return err
|
||||
// We lock the shard to not conflict with reparent operations.
|
||||
ctx, unlock, lockErr := wr.ts.LockShard(ctx, ti.Keyspace, ti.Shard, fmt.Sprintf("DeleteTablet(%v)", topoproto.TabletAliasString(tabletAlias)))
|
||||
if lockErr != nil {
|
||||
return lockErr
|
||||
}
|
||||
defer unlock(&err)
|
||||
|
||||
// update the shard record's master
|
||||
if _, err := wr.ts.UpdateShardFields(ctx, ti.Keyspace, ti.Shard, func(s *topodatapb.Shard) error {
|
||||
if !topoproto.TabletAliasEqual(s.MasterAlias, tabletAlias) {
|
||||
wr.Logger().Warningf("Deleting master %v from shard %v/%v but master in Shard object was %v", topoproto.TabletAliasString(tabletAlias), ti.Keyspace, ti.Shard, topoproto.TabletAliasString(s.MasterAlias))
|
||||
_, err = wr.ts.UpdateShardFields(ctx, ti.Keyspace, ti.Shard, func(si *topo.ShardInfo) error {
|
||||
if !topoproto.TabletAliasEqual(si.MasterAlias, tabletAlias) {
|
||||
wr.Logger().Warningf("Deleting master %v from shard %v/%v but master in Shard object was %v", topoproto.TabletAliasString(tabletAlias), ti.Keyspace, ti.Shard, topoproto.TabletAliasString(si.MasterAlias))
|
||||
return topo.ErrNoUpdateNeeded
|
||||
}
|
||||
s.MasterAlias = nil
|
||||
si.MasterAlias = nil
|
||||
return nil
|
||||
}); err != nil {
|
||||
return wr.unlockShard(ctx, ti.Keyspace, ti.Shard, actionNode, lockPath, err)
|
||||
}
|
||||
|
||||
// and unlock
|
||||
if err := wr.unlockShard(ctx, ti.Keyspace, ti.Shard, actionNode, lockPath, err); err != nil {
|
||||
return err
|
||||
}
|
||||
})
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
|
|
|
@ -14,6 +14,7 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
|
||||
"github.com/youtube/vitess/go/vt/wrangler"
|
||||
|
@ -220,13 +221,14 @@ func TestInitMasterShardOneSlaveFails(t *testing.T) {
|
|||
|
||||
// also change the master alias in the Shard object, to make sure it
|
||||
// is set back.
|
||||
si, err := ts.GetShard(ctx, master.Tablet.Keyspace, master.Tablet.Shard)
|
||||
_, err := ts.UpdateShardFields(ctx, master.Tablet.Keyspace, master.Tablet.Shard, func(si *topo.ShardInfo) error {
|
||||
// note it's OK to retry this and increment mutiple times,
|
||||
// we just want it to be different
|
||||
si.MasterAlias.Uid++
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.MasterAlias.Uid++
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// run InitShardMaster without force, it fails because master is
|
||||
|
@ -244,7 +246,7 @@ func TestInitMasterShardOneSlaveFails(t *testing.T) {
|
|||
if master.FakeMysqlDaemon.ReadOnly {
|
||||
t.Errorf("master was not turned read-write")
|
||||
}
|
||||
si, err = ts.GetShard(ctx, master.Tablet.Keyspace, master.Tablet.Shard)
|
||||
si, err := ts.GetShard(ctx, master.Tablet.Keyspace, master.Tablet.Shard)
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
|
|
|
@ -31,6 +31,17 @@ func checkShardServedTypes(t *testing.T, ts topo.Server, shard string, expected
|
|||
}
|
||||
}
|
||||
|
||||
func checkShardSourceShards(t *testing.T, ts topo.Server, shard string, expected int) {
|
||||
ctx := context.Background()
|
||||
si, err := ts.GetShard(ctx, "ks", shard)
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
if len(si.SourceShards) != expected {
|
||||
t.Fatalf("shard %v has wrong SourceShards: %#v", shard, si.SourceShards)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMigrateServedTypes(t *testing.T) {
|
||||
db := fakesqldb.Register()
|
||||
ts := zktestserver.New(t, []string{"cell1", "cell2"})
|
||||
|
@ -142,12 +153,16 @@ func TestMigrateServedTypes(t *testing.T) {
|
|||
defer dest2Master.StopActionLoop(t)
|
||||
|
||||
// simulate the clone, by fixing the dest shard record
|
||||
checkShardSourceShards(t, ts, "-80", 0)
|
||||
checkShardSourceShards(t, ts, "80-", 0)
|
||||
if err := vp.Run([]string{"SourceShardAdd", "--key_range=-", "ks/-80", "0", "ks/0"}); err != nil {
|
||||
t.Fatalf("SourceShardAdd failed: %v", err)
|
||||
}
|
||||
if err := vp.Run([]string{"SourceShardAdd", "--key_range=-", "ks/80-", "0", "ks/0"}); err != nil {
|
||||
t.Fatalf("SourceShardAdd failed: %v", err)
|
||||
}
|
||||
checkShardSourceShards(t, ts, "-80", 1)
|
||||
checkShardSourceShards(t, ts, "80-", 1)
|
||||
|
||||
// migrate rdonly over
|
||||
if err := vp.Run([]string{"MigrateServedTypes", "ks/0", "rdonly"}); err != nil {
|
||||
|
@ -157,6 +172,8 @@ func TestMigrateServedTypes(t *testing.T) {
|
|||
checkShardServedTypes(t, ts, "0", 2)
|
||||
checkShardServedTypes(t, ts, "-80", 1)
|
||||
checkShardServedTypes(t, ts, "80-", 1)
|
||||
checkShardSourceShards(t, ts, "-80", 1)
|
||||
checkShardSourceShards(t, ts, "80-", 1)
|
||||
|
||||
// migrate replica over
|
||||
if err := vp.Run([]string{"MigrateServedTypes", "ks/0", "replica"}); err != nil {
|
||||
|
@ -166,6 +183,8 @@ func TestMigrateServedTypes(t *testing.T) {
|
|||
checkShardServedTypes(t, ts, "0", 1)
|
||||
checkShardServedTypes(t, ts, "-80", 2)
|
||||
checkShardServedTypes(t, ts, "80-", 2)
|
||||
checkShardSourceShards(t, ts, "-80", 1)
|
||||
checkShardSourceShards(t, ts, "80-", 1)
|
||||
|
||||
// migrate master over
|
||||
if err := vp.Run([]string{"MigrateServedTypes", "ks/0", "master"}); err != nil {
|
||||
|
@ -175,4 +194,6 @@ func TestMigrateServedTypes(t *testing.T) {
|
|||
checkShardServedTypes(t, ts, "0", 0)
|
||||
checkShardServedTypes(t, ts, "-80", 3)
|
||||
checkShardServedTypes(t, ts, "80-", 3)
|
||||
checkShardSourceShards(t, ts, "-80", 0)
|
||||
checkShardSourceShards(t, ts, "80-", 0)
|
||||
}
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"github.com/youtube/vitess/go/sqltypes"
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
|
||||
"github.com/youtube/vitess/go/vt/wrangler"
|
||||
"github.com/youtube/vitess/go/vt/zktopo/zktestserver"
|
||||
|
@ -33,13 +34,12 @@ func TestPermissions(t *testing.T) {
|
|||
replica := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, db)
|
||||
|
||||
// mark the master inside the shard
|
||||
si, err := ts.GetShard(ctx, master.Tablet.Keyspace, master.Tablet.Shard)
|
||||
_, err := ts.UpdateShardFields(ctx, master.Tablet.Keyspace, master.Tablet.Shard, func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// master will be asked for permissions
|
||||
|
|
|
@ -46,13 +46,14 @@ func TestTabletExternallyReparented(t *testing.T) {
|
|||
|
||||
// Add a new Cell to the Shard, that doesn't map to any read topo cell,
|
||||
// to simulate a data center being unreachable.
|
||||
si, err := ts.GetShard(ctx, "test_keyspace", "0")
|
||||
_, err := ts.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
if !si.HasCell("cell666") {
|
||||
si.Cells = append(si.Cells, "cell666")
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.Cells = append(si.Cells, "cell666")
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// Slightly unrelated test: make sure we can find the tablets
|
||||
|
|
|
@ -11,6 +11,7 @@ import (
|
|||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
"github.com/youtube/vitess/go/vt/topo/topoproto"
|
||||
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
|
||||
"github.com/youtube/vitess/go/vt/wrangler"
|
||||
|
@ -27,20 +28,18 @@ func TestShardReplicationStatuses(t *testing.T) {
|
|||
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
|
||||
|
||||
// create shard and tablets
|
||||
if err := ts.CreateShard(ctx, "test_keyspace", "0"); err != nil {
|
||||
t.Fatalf("CreateShard failed: %v", err)
|
||||
if _, err := ts.GetOrCreateShard(ctx, "test_keyspace", "0"); err != nil {
|
||||
t.Fatalf("GetOrCreateShard failed: %v", err)
|
||||
}
|
||||
master := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_MASTER, db)
|
||||
slave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
|
||||
|
||||
// mark the master inside the shard
|
||||
si, err := ts.GetShard(ctx, "test_keyspace", "0")
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
if _, err := ts.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
return nil
|
||||
}); err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// master action loop (to initialize host and port)
|
||||
|
@ -96,20 +95,18 @@ func TestReparentTablet(t *testing.T) {
|
|||
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
|
||||
|
||||
// create shard and tablets
|
||||
if err := ts.CreateShard(ctx, "test_keyspace", "0"); err != nil {
|
||||
if _, err := ts.GetOrCreateShard(ctx, "test_keyspace", "0"); err != nil {
|
||||
t.Fatalf("CreateShard failed: %v", err)
|
||||
}
|
||||
master := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_MASTER, db)
|
||||
slave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
|
||||
|
||||
// mark the master inside the shard
|
||||
si, err := ts.GetShard(ctx, "test_keyspace", "0")
|
||||
if err != nil {
|
||||
t.Fatalf("GetShard failed: %v", err)
|
||||
}
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
if err := ts.UpdateShard(ctx, si); err != nil {
|
||||
t.Fatalf("UpdateShard failed: %v", err)
|
||||
if _, err := ts.UpdateShardFields(ctx, "test_keyspace", "0", func(si *topo.ShardInfo) error {
|
||||
si.MasterAlias = master.Tablet.Alias
|
||||
return nil
|
||||
}); err != nil {
|
||||
t.Fatalf("UpdateShardFields failed: %v", err)
|
||||
}
|
||||
|
||||
// master action loop (to initialize host and port)
|
||||
|
|
|
@ -9,6 +9,7 @@ import (
|
|||
"fmt"
|
||||
"io"
|
||||
"net"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -26,12 +27,7 @@ import (
|
|||
_ "github.com/youtube/vitess/go/vt/vtctl/grpcvtctlclient"
|
||||
)
|
||||
|
||||
var servenvInitialized = false
|
||||
|
||||
func init() {
|
||||
// make sure we use the right protocol
|
||||
flag.Set("vtctl_client_protocol", "grpc")
|
||||
}
|
||||
var servenvInitialized sync.Once
|
||||
|
||||
// VtctlPipe is a vtctl server based on a topo server, and a client that
|
||||
// is connected to it via gRPC.
|
||||
|
@ -44,11 +40,14 @@ type VtctlPipe struct {
|
|||
// NewVtctlPipe creates a new VtctlPipe based on the given topo server.
|
||||
func NewVtctlPipe(t *testing.T, ts topo.Server) *VtctlPipe {
|
||||
// Register all vtctl commands
|
||||
if !servenvInitialized {
|
||||
servenvInitialized.Do(func() {
|
||||
// make sure we use the right protocol
|
||||
flag.Set("vtctl_client_protocol", "grpc")
|
||||
|
||||
// Enable all query groups
|
||||
flag.Set("enable_queries", "true")
|
||||
servenv.FireRunHooks()
|
||||
servenvInitialized = true
|
||||
}
|
||||
})
|
||||
|
||||
// Listen on a random port
|
||||
listener, err := net.Listen("tcp", ":0")
|
||||
|
|
|
@ -108,8 +108,6 @@ func (wr *Wrangler) validateKeyspace(ctx context.Context, keyspace string, pingT
|
|||
}
|
||||
}
|
||||
|
||||
// FIXME(msolomon) This validate presumes the master is up and running.
|
||||
// Even when that isn't true, there are validation processes that might be valuable.
|
||||
func (wr *Wrangler) validateShard(ctx context.Context, keyspace, shard string, pingTablets bool, wg *sync.WaitGroup, results chan<- error) {
|
||||
shardInfo, err := wr.ts.GetShard(ctx, keyspace, shard)
|
||||
if err != nil {
|
||||
|
|
|
@ -8,7 +8,6 @@ package wrangler
|
|||
|
||||
import (
|
||||
"github.com/youtube/vitess/go/vt/logutil"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
|
||||
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
|
||||
"github.com/youtube/vitess/go/vt/topo"
|
||||
)
|
||||
|
@ -18,7 +17,7 @@ var (
|
|||
// remote actions. We usually take a lock then do an action,
|
||||
// so basing this to be greater than DefaultLockTimeout is good.
|
||||
// Use this as the default value for Context that need a deadline.
|
||||
DefaultActionTimeout = actionnode.DefaultLockTimeout * 4
|
||||
DefaultActionTimeout = topo.DefaultLockTimeout * 4
|
||||
)
|
||||
|
||||
// Wrangler manages complex actions on the topology, like reparents,
|
||||
|
|
|
@ -0,0 +1,121 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema {
|
||||
|
||||
class AutoIncrement extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $column = null;
|
||||
|
||||
/** @var string */
|
||||
public $sequence = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.AutoIncrement');
|
||||
|
||||
// OPTIONAL STRING column = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "column";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL STRING sequence = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "sequence";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <column> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasColumn(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <column> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\AutoIncrement
|
||||
*/
|
||||
public function clearColumn(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <column> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getColumn(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <column> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\AutoIncrement
|
||||
*/
|
||||
public function setColumn( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <sequence> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasSequence(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <sequence> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\AutoIncrement
|
||||
*/
|
||||
public function clearSequence(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <sequence> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getSequence(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <sequence> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\AutoIncrement
|
||||
*/
|
||||
public function setSequence( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,121 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema {
|
||||
|
||||
class ColumnVindex extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $column = null;
|
||||
|
||||
/** @var string */
|
||||
public $name = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.ColumnVindex');
|
||||
|
||||
// OPTIONAL STRING column = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "column";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL STRING name = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "name";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <column> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasColumn(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <column> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex
|
||||
*/
|
||||
public function clearColumn(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <column> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getColumn(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <column> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex
|
||||
*/
|
||||
public function setColumn( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <name> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasName(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <name> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex
|
||||
*/
|
||||
public function clearName(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <name> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getName(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <name> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex
|
||||
*/
|
||||
public function setName( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,211 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema {
|
||||
|
||||
class Keyspace extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var boolean */
|
||||
public $sharded = null;
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\Keyspace\VindexesEntry[] */
|
||||
public $vindexes = array();
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\Keyspace\TablesEntry[] */
|
||||
public $tables = array();
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Keyspace');
|
||||
|
||||
// OPTIONAL BOOL sharded = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "sharded";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_BOOL;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// REPEATED MESSAGE vindexes = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "vindexes";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_REPEATED;
|
||||
$f->reference = '\Vitess\Proto\Vschema\Keyspace\VindexesEntry';
|
||||
$descriptor->addField($f);
|
||||
|
||||
// REPEATED MESSAGE tables = 3
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 3;
|
||||
$f->name = "tables";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_REPEATED;
|
||||
$f->reference = '\Vitess\Proto\Vschema\Keyspace\TablesEntry';
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <sharded> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasSharded(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <sharded> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function clearSharded(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <sharded> value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function getSharded(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <sharded> value
|
||||
*
|
||||
* @param boolean $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function setSharded( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <vindexes> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasVindexes(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <vindexes> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function clearVindexes(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <vindexes> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry
|
||||
*/
|
||||
public function getVindexes($idx = NULL){
|
||||
return $this->_get(2, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <vindexes> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Keyspace\VindexesEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function setVindexes(\Vitess\Proto\Vschema\Keyspace\VindexesEntry $value, $idx = NULL){
|
||||
return $this->_set(2, $value, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all elements of <vindexes>
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry[]
|
||||
*/
|
||||
public function getVindexesList(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <vindexes>
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Keyspace\VindexesEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function addVindexes(\Vitess\Proto\Vschema\Keyspace\VindexesEntry $value){
|
||||
return $this->_add(2, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <tables> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasTables(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <tables> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function clearTables(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <tables> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry
|
||||
*/
|
||||
public function getTables($idx = NULL){
|
||||
return $this->_get(3, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <tables> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Keyspace\TablesEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function setTables(\Vitess\Proto\Vschema\Keyspace\TablesEntry $value, $idx = NULL){
|
||||
return $this->_set(3, $value, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all elements of <tables>
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry[]
|
||||
*/
|
||||
public function getTablesList(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <tables>
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Keyspace\TablesEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace
|
||||
*/
|
||||
public function addTables(\Vitess\Proto\Vschema\Keyspace\TablesEntry $value){
|
||||
return $this->_add(3, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,122 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema\Keyspace {
|
||||
|
||||
class TablesEntry extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $key = null;
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\Table */
|
||||
public $value = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Keyspace.TablesEntry');
|
||||
|
||||
// OPTIONAL STRING key = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "key";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL MESSAGE value = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "value";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$f->reference = '\Vitess\Proto\Vschema\Table';
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <key> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKey(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <key> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry
|
||||
*/
|
||||
public function clearKey(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <key> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getKey(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <key> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry
|
||||
*/
|
||||
public function setKey( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <value> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasValue(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry
|
||||
*/
|
||||
public function clearValue(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function getValue(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <value> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Table $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\TablesEntry
|
||||
*/
|
||||
public function setValue(\Vitess\Proto\Vschema\Table $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,122 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema\Keyspace {
|
||||
|
||||
class VindexesEntry extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $key = null;
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\Vindex */
|
||||
public $value = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Keyspace.VindexesEntry');
|
||||
|
||||
// OPTIONAL STRING key = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "key";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL MESSAGE value = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "value";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$f->reference = '\Vitess\Proto\Vschema\Vindex';
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <key> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKey(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <key> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry
|
||||
*/
|
||||
public function clearKey(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <key> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getKey(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <key> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry
|
||||
*/
|
||||
public function setKey( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <value> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasValue(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry
|
||||
*/
|
||||
public function clearValue(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function getValue(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <value> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Vindex $value
|
||||
* @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry
|
||||
*/
|
||||
public function setValue(\Vitess\Proto\Vschema\Vindex $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,191 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema {
|
||||
|
||||
class Table extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $type = null;
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\ColumnVindex[] */
|
||||
public $column_vindexes = array();
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\AutoIncrement */
|
||||
public $auto_increment = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Table');
|
||||
|
||||
// OPTIONAL STRING type = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "type";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// REPEATED MESSAGE column_vindexes = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "column_vindexes";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_REPEATED;
|
||||
$f->reference = '\Vitess\Proto\Vschema\ColumnVindex';
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL MESSAGE auto_increment = 3
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 3;
|
||||
$f->name = "auto_increment";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$f->reference = '\Vitess\Proto\Vschema\AutoIncrement';
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <type> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasType(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <type> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function clearType(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <type> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getType(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <type> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function setType( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <column_vindexes> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasColumnVindexes(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <column_vindexes> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function clearColumnVindexes(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <column_vindexes> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex
|
||||
*/
|
||||
public function getColumnVindexes($idx = NULL){
|
||||
return $this->_get(2, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <column_vindexes> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\ColumnVindex $value
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function setColumnVindexes(\Vitess\Proto\Vschema\ColumnVindex $value, $idx = NULL){
|
||||
return $this->_set(2, $value, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all elements of <column_vindexes>
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\ColumnVindex[]
|
||||
*/
|
||||
public function getColumnVindexesList(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <column_vindexes>
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\ColumnVindex $value
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function addColumnVindexes(\Vitess\Proto\Vschema\ColumnVindex $value){
|
||||
return $this->_add(2, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <auto_increment> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasAutoIncrement(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <auto_increment> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function clearAutoIncrement(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <auto_increment> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\AutoIncrement
|
||||
*/
|
||||
public function getAutoIncrement(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <auto_increment> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\AutoIncrement $value
|
||||
* @return \Vitess\Proto\Vschema\Table
|
||||
*/
|
||||
public function setAutoIncrement(\Vitess\Proto\Vschema\AutoIncrement $value){
|
||||
return $this->_set(3, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,190 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema {
|
||||
|
||||
class Vindex extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $type = null;
|
||||
|
||||
/** @var \Vitess\Proto\Vschema\Vindex\ParamsEntry[] */
|
||||
public $params = array();
|
||||
|
||||
/** @var string */
|
||||
public $owner = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Vindex');
|
||||
|
||||
// OPTIONAL STRING type = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "type";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// REPEATED MESSAGE params = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "params";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_MESSAGE;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_REPEATED;
|
||||
$f->reference = '\Vitess\Proto\Vschema\Vindex\ParamsEntry';
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL STRING owner = 3
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 3;
|
||||
$f->name = "owner";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <type> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasType(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <type> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function clearType(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <type> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getType(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <type> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function setType( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <params> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasParams(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <params> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function clearParams(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <params> value
|
||||
*
|
||||
* @param int $idx
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry
|
||||
*/
|
||||
public function getParams($idx = NULL){
|
||||
return $this->_get(2, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <params> value
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Vindex\ParamsEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function setParams(\Vitess\Proto\Vschema\Vindex\ParamsEntry $value, $idx = NULL){
|
||||
return $this->_set(2, $value, $idx);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all elements of <params>
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry[]
|
||||
*/
|
||||
public function getParamsList(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new element to <params>
|
||||
*
|
||||
* @param \Vitess\Proto\Vschema\Vindex\ParamsEntry $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function addParams(\Vitess\Proto\Vschema\Vindex\ParamsEntry $value){
|
||||
return $this->_add(2, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <owner> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasOwner(){
|
||||
return $this->_has(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <owner> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function clearOwner(){
|
||||
return $this->_clear(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <owner> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getOwner(){
|
||||
return $this->_get(3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <owner> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex
|
||||
*/
|
||||
public function setOwner( $value){
|
||||
return $this->_set(3, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,121 @@
|
|||
<?php
|
||||
// DO NOT EDIT! Generated by Protobuf-PHP protoc plugin 1.0
|
||||
// Source: vschema.proto
|
||||
|
||||
namespace Vitess\Proto\Vschema\Vindex {
|
||||
|
||||
class ParamsEntry extends \DrSlump\Protobuf\Message {
|
||||
|
||||
/** @var string */
|
||||
public $key = null;
|
||||
|
||||
/** @var string */
|
||||
public $value = null;
|
||||
|
||||
|
||||
/** @var \Closure[] */
|
||||
protected static $__extensions = array();
|
||||
|
||||
public static function descriptor()
|
||||
{
|
||||
$descriptor = new \DrSlump\Protobuf\Descriptor(__CLASS__, 'vschema.Vindex.ParamsEntry');
|
||||
|
||||
// OPTIONAL STRING key = 1
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 1;
|
||||
$f->name = "key";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
// OPTIONAL STRING value = 2
|
||||
$f = new \DrSlump\Protobuf\Field();
|
||||
$f->number = 2;
|
||||
$f->name = "value";
|
||||
$f->type = \DrSlump\Protobuf::TYPE_STRING;
|
||||
$f->rule = \DrSlump\Protobuf::RULE_OPTIONAL;
|
||||
$descriptor->addField($f);
|
||||
|
||||
foreach (self::$__extensions as $cb) {
|
||||
$descriptor->addField($cb(), true);
|
||||
}
|
||||
|
||||
return $descriptor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <key> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasKey(){
|
||||
return $this->_has(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <key> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry
|
||||
*/
|
||||
public function clearKey(){
|
||||
return $this->_clear(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <key> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getKey(){
|
||||
return $this->_get(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <key> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry
|
||||
*/
|
||||
public function setKey( $value){
|
||||
return $this->_set(1, $value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if <value> has a value
|
||||
*
|
||||
* @return boolean
|
||||
*/
|
||||
public function hasValue(){
|
||||
return $this->_has(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear <value> value
|
||||
*
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry
|
||||
*/
|
||||
public function clearValue(){
|
||||
return $this->_clear(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get <value> value
|
||||
*
|
||||
* @return string
|
||||
*/
|
||||
public function getValue(){
|
||||
return $this->_get(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set <value> value
|
||||
*
|
||||
* @param string $value
|
||||
* @return \Vitess\Proto\Vschema\Vindex\ParamsEntry
|
||||
*/
|
||||
public function setValue( $value){
|
||||
return $this->_set(2, $value);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -128,6 +128,12 @@ message Tablet {
|
|||
message Shard {
|
||||
// master_alias is the tablet alias of the master for the shard.
|
||||
// If it is unset, then there is no master in this shard yet.
|
||||
|
||||
// No lock is necessary to update this field, when for instance
|
||||
// TabletExternallyReparented updates this. However, we lock the
|
||||
// shard for reparenting operations (InitShardMaster,
|
||||
// PlannedReparentShard,EmergencyReparentShard), to guarantee
|
||||
// exclusive operation.
|
||||
TabletAlias master_alias = 1;
|
||||
|
||||
// key_range is the KeyRange for this shard. It can be unset if:
|
||||
|
@ -135,6 +141,7 @@ message Shard {
|
|||
// - the shard covers the entire keyrange.
|
||||
// This must match the shard name based on our other conventions, but
|
||||
// helpful to have it decomposed here.
|
||||
// Once set at creation time, it is never changed.
|
||||
KeyRange key_range = 2;
|
||||
|
||||
// ServedType is an entry in the served_types
|
||||
|
@ -144,6 +151,7 @@ message Shard {
|
|||
}
|
||||
|
||||
// served_types has at most one entry per TabletType
|
||||
// The keyspace lock is always taken when changing this.
|
||||
repeated ServedType served_types = 3;
|
||||
|
||||
// SourceShard represents a data source for filtered replication
|
||||
|
@ -168,9 +176,11 @@ message Shard {
|
|||
|
||||
// SourceShards is the list of shards we're replicating from,
|
||||
// using filtered replication.
|
||||
// The keyspace lock is always taken when changing this.
|
||||
repeated SourceShard source_shards = 4;
|
||||
|
||||
// Cells is the list of cells that contain tablets for this shard.
|
||||
// No lock is necessary to update this field.
|
||||
repeated string cells = 5;
|
||||
|
||||
// TabletControl controls tablet's behavior
|
||||
|
@ -184,7 +194,8 @@ message Shard {
|
|||
repeated string blacklisted_tables = 4;
|
||||
}
|
||||
|
||||
// tablet_controls has at most one entry per TabletType
|
||||
// tablet_controls has at most one entry per TabletType.
|
||||
// The keyspace lock is always taken when changing this.
|
||||
repeated TabletControl tablet_controls = 6;
|
||||
}
|
||||
|
||||
|
|
Загрузка…
Ссылка в новой задаче