diff --git a/go/cmd/vtcombo/tablet_map.go b/go/cmd/vtcombo/tablet_map.go index 0469b27d19..48bbf9e3ac 100644 --- a/go/cmd/vtcombo/tablet_map.go +++ b/go/cmd/vtcombo/tablet_map.go @@ -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") } diff --git a/go/vt/proto/topodata/topodata.pb.go b/go/vt/proto/topodata/topodata.pb.go index f695b160ff..2e8e5e6daf 100644 --- a/go/vt/proto/topodata/topodata.pb.go +++ b/go/vt/proto/topodata/topodata.pb.go @@ -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"` } diff --git a/go/vt/tabletmanager/actionnode/actionnode.go b/go/vt/tabletmanager/actionnode/actionnode.go deleted file mode 100644 index 7cc5675d8f..0000000000 --- a/go/vt/tabletmanager/actionnode/actionnode.go +++ /dev/null @@ -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 -} diff --git a/go/vt/tabletmanager/actionnode/structs.go b/go/vt/tabletmanager/actionnode/structs.go deleted file mode 100644 index a57de1f1da..0000000000 --- a/go/vt/tabletmanager/actionnode/structs.go +++ /dev/null @@ -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() -} diff --git a/go/vt/tabletmanager/actionnode/structs_test.go b/go/vt/tabletmanager/actionnode/structs_test.go deleted file mode 100644 index a60d268e7b..0000000000 --- a/go/vt/tabletmanager/actionnode/structs_test.go +++ /dev/null @@ -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) - } -} diff --git a/go/vt/tabletmanager/actionnode/utils.go b/go/vt/tabletmanager/actionnode/utils.go deleted file mode 100644 index d0226a7957..0000000000 --- a/go/vt/tabletmanager/actionnode/utils.go +++ /dev/null @@ -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 -} diff --git a/go/vt/tabletmanager/agentrpctest/test_agent_rpc.go b/go/vt/tabletmanager/agentrpctest/test_agent_rpc.go index e498f5a013..5ece83a660 100644 --- a/go/vt/tabletmanager/agentrpctest/test_agent_rpc.go +++ b/go/vt/tabletmanager/agentrpctest/test_agent_rpc.go @@ -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) diff --git a/go/vt/tabletmanager/binlog_players_test.go b/go/vt/tabletmanager/binlog_players_test.go index 03bf858d6a..5d83849e5a 100644 --- a/go/vt/tabletmanager/binlog_players_test.go +++ b/go/vt/tabletmanager/binlog_players_test.go @@ -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 diff --git a/go/vt/tabletmanager/faketmclient/fake_client.go b/go/vt/tabletmanager/faketmclient/fake_client.go index 8ef037bf07..9cfc966393 100644 --- a/go/vt/tabletmanager/faketmclient/fake_client.go +++ b/go/vt/tabletmanager/faketmclient/fake_client.go @@ -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 } diff --git a/go/vt/tabletmanager/grpctmclient/client.go b/go/vt/tabletmanager/grpctmclient/client.go index 9198b42add..11774e0ea6 100644 --- a/go/vt/tabletmanager/grpctmclient/client.go +++ b/go/vt/tabletmanager/grpctmclient/client.go @@ -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 } diff --git a/go/vt/tabletmanager/grpctmserver/server.go b/go/vt/tabletmanager/grpctmserver/server.go index 54c9644949..8ae68bfca8 100644 --- a/go/vt/tabletmanager/grpctmserver/server.go +++ b/go/vt/tabletmanager/grpctmserver/server.go @@ -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 diff --git a/go/vt/tabletmanager/healthcheck_test.go b/go/vt/tabletmanager/healthcheck_test.go index 3549ab28ce..e6635865e6 100644 --- a/go/vt/tabletmanager/healthcheck_test.go +++ b/go/vt/tabletmanager/healthcheck_test.go @@ -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 }) diff --git a/go/vt/tabletmanager/init_tablet.go b/go/vt/tabletmanager/init_tablet.go index a206b755bb..08bf93e05f 100644 --- a/go/vt/tabletmanager/init_tablet.go +++ b/go/vt/tabletmanager/init_tablet.go @@ -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 { diff --git a/go/vt/tabletmanager/init_tablet_test.go b/go/vt/tabletmanager/init_tablet_test.go index 81fbb8d2fe..c1e6b1a8e6 100644 --- a/go/vt/tabletmanager/init_tablet_test.go +++ b/go/vt/tabletmanager/init_tablet_test.go @@ -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) diff --git a/go/vt/tabletmanager/rpc_agent.go b/go/vt/tabletmanager/rpc_agent.go index ee3d798964..a2a54aa4c1 100644 --- a/go/vt/tabletmanager/rpc_agent.go +++ b/go/vt/tabletmanager/rpc_agent.go @@ -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 } diff --git a/go/vt/tabletmanager/rpc_external_reparent.go b/go/vt/tabletmanager/rpc_external_reparent.go index 26a59d95d1..00bcd529f0 100644 --- a/go/vt/tabletmanager/rpc_external_reparent.go +++ b/go/vt/tabletmanager/rpc_external_reparent.go @@ -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 { diff --git a/go/vt/tabletmanager/rpc_replication.go b/go/vt/tabletmanager/rpc_replication.go index 0d15bb29cd..9c0ebddce7 100644 --- a/go/vt/tabletmanager/rpc_replication.go +++ b/go/vt/tabletmanager/rpc_replication.go @@ -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. diff --git a/go/vt/tabletmanager/rpc_server.go b/go/vt/tabletmanager/rpc_server.go index b5a599ec96..b1858aa80a 100644 --- a/go/vt/tabletmanager/rpc_server.go +++ b/go/vt/tabletmanager/rpc_server.go @@ -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*/) } diff --git a/go/vt/tabletmanager/tmclient/rpc_client_api.go b/go/vt/tabletmanager/tmclient/rpc_client_api.go index 6bd742b5d0..c11dea9ae8 100644 --- a/go/vt/tabletmanager/tmclient/rpc_client_api.go +++ b/go/vt/tabletmanager/tmclient/rpc_client_api.go @@ -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. diff --git a/go/vt/topo/keyspace.go b/go/vt/topo/keyspace.go index 5f46033234..8488ccc656 100644 --- a/go/vt/topo/keyspace.go +++ b/go/vt/topo/keyspace.go @@ -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 } diff --git a/go/vt/topo/locks.go b/go/vt/topo/locks.go new file mode 100644 index 0000000000..78c7b0c628 --- /dev/null +++ b/go/vt/topo/locks.go @@ -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) +} diff --git a/go/vt/topo/server.go b/go/vt/topo/server.go index 656f72b8b8..ab717652be 100644 --- a/go/vt/topo/server.go +++ b/go/vt/topo/server.go @@ -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. diff --git a/go/vt/topo/shard.go b/go/vt/topo/shard.go index 2d7c642a36..01e45557cb 100644 --- a/go/vt/topo/shard.go +++ b/go/vt/topo/shard.go @@ -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 { diff --git a/go/vt/topo/shard_test.go b/go/vt/topo/shard_test.go index da938e3e53..4d2379dd2b 100644 --- a/go/vt/topo/shard_test.go +++ b/go/vt/topo/shard_test.go @@ -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"}, diff --git a/go/vt/topo/test/shard.go b/go/vt/topo/test/shard.go index b23fa83897..b5d965a4ef 100644 --- a/go/vt/topo/test/shard.go +++ b/go/vt/topo/test/shard.go @@ -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 { diff --git a/go/vt/topotools/rebuild_keyspace.go b/go/vt/topotools/rebuild_keyspace.go index ff0a665434..5cd25eff37 100644 --- a/go/vt/topotools/rebuild_keyspace.go +++ b/go/vt/topotools/rebuild_keyspace.go @@ -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 { diff --git a/go/vt/topotools/reparent.go b/go/vt/topotools/reparent.go index 83e2dc92c3..759fc2ed3c 100644 --- a/go/vt/topotools/reparent.go +++ b/go/vt/topotools/reparent.go @@ -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 diff --git a/go/vt/topotools/shard.go b/go/vt/topotools/shard.go deleted file mode 100644 index 14bd06b75b..0000000000 --- a/go/vt/topotools/shard.go +++ /dev/null @@ -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 -} diff --git a/go/vt/topotools/shard_test.go b/go/vt/topotools/shard_test.go index 96291e152a..4bb4a02e0d 100644 --- a/go/vt/topotools/shard_test.go +++ b/go/vt/topotools/shard_test.go @@ -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) } diff --git a/go/vt/vtctl/vtctl.go b/go/vt/vtctl/vtctl.go index 1f3368d468..d4c6a2c03c 100644 --- a/go/vt/vtctl/vtctl.go +++ b/go/vt/vtctl/vtctl.go @@ -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 diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go index c9ee9e9513..c4e2ec9b5f 100644 --- a/go/vt/wrangler/keyspace.go +++ b/go/vt/wrangler/keyspace.go @@ -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() diff --git a/go/vt/wrangler/reparent.go b/go/vt/wrangler/reparent.go index 002ebb89b7..1d1f704d88 100644 --- a/go/vt/wrangler/reparent.go +++ b/go/vt/wrangler/reparent.go @@ -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() diff --git a/go/vt/wrangler/schema.go b/go/vt/wrangler/schema.go index ed30617265..4003a61615 100644 --- a/go/vt/wrangler/schema.go +++ b/go/vt/wrangler/schema.go @@ -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. diff --git a/go/vt/wrangler/shard.go b/go/vt/wrangler/shard.go index ec6309882d..5a2dd9ec48 100644 --- a/go/vt/wrangler/shard.go +++ b/go/vt/wrangler/shard.go @@ -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 } diff --git a/go/vt/wrangler/split.go b/go/vt/wrangler/split.go index 13253f6d58..cb1d524b8b 100644 --- a/go/vt/wrangler/split.go +++ b/go/vt/wrangler/split.go @@ -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 diff --git a/go/vt/wrangler/tablet.go b/go/vt/wrangler/tablet.go index 258d423aa1..8d9d29ac02 100644 --- a/go/vt/wrangler/tablet.go +++ b/go/vt/wrangler/tablet.go @@ -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 diff --git a/go/vt/wrangler/testlib/init_shard_master_test.go b/go/vt/wrangler/testlib/init_shard_master_test.go index 41926aebed..fc1902e325 100644 --- a/go/vt/wrangler/testlib/init_shard_master_test.go +++ b/go/vt/wrangler/testlib/init_shard_master_test.go @@ -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) } diff --git a/go/vt/wrangler/testlib/migrate_served_types_test.go b/go/vt/wrangler/testlib/migrate_served_types_test.go index b24ac33b00..b12c72ff5d 100644 --- a/go/vt/wrangler/testlib/migrate_served_types_test.go +++ b/go/vt/wrangler/testlib/migrate_served_types_test.go @@ -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) } diff --git a/go/vt/wrangler/testlib/permissions_test.go b/go/vt/wrangler/testlib/permissions_test.go index d115ab6ea8..ffabdf1686 100644 --- a/go/vt/wrangler/testlib/permissions_test.go +++ b/go/vt/wrangler/testlib/permissions_test.go @@ -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 diff --git a/go/vt/wrangler/testlib/reparent_external_test.go b/go/vt/wrangler/testlib/reparent_external_test.go index e77e1e23de..f33b21ed06 100644 --- a/go/vt/wrangler/testlib/reparent_external_test.go +++ b/go/vt/wrangler/testlib/reparent_external_test.go @@ -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 diff --git a/go/vt/wrangler/testlib/reparent_utils_test.go b/go/vt/wrangler/testlib/reparent_utils_test.go index c743a05ee7..049147b004 100644 --- a/go/vt/wrangler/testlib/reparent_utils_test.go +++ b/go/vt/wrangler/testlib/reparent_utils_test.go @@ -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) diff --git a/go/vt/wrangler/testlib/vtctl_pipe.go b/go/vt/wrangler/testlib/vtctl_pipe.go index 80ae4ce53b..d694f151b6 100644 --- a/go/vt/wrangler/testlib/vtctl_pipe.go +++ b/go/vt/wrangler/testlib/vtctl_pipe.go @@ -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") diff --git a/go/vt/wrangler/validator.go b/go/vt/wrangler/validator.go index 3d30a21d1d..c3ef769013 100644 --- a/go/vt/wrangler/validator.go +++ b/go/vt/wrangler/validator.go @@ -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 { diff --git a/go/vt/wrangler/wrangler.go b/go/vt/wrangler/wrangler.go index 99d47fb97c..0fe75cfe15 100644 --- a/go/vt/wrangler/wrangler.go +++ b/go/vt/wrangler/wrangler.go @@ -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, diff --git a/php/src/Vitess/Proto/Vschema/AutoIncrement.php b/php/src/Vitess/Proto/Vschema/AutoIncrement.php new file mode 100644 index 0000000000..9e27f77b79 --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/AutoIncrement.php @@ -0,0 +1,121 @@ +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 has a value + * + * @return boolean + */ + public function hasColumn(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\AutoIncrement + */ + public function clearColumn(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getColumn(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\AutoIncrement + */ + public function setColumn( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasSequence(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\AutoIncrement + */ + public function clearSequence(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return string + */ + public function getSequence(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\AutoIncrement + */ + public function setSequence( $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/ColumnVindex.php b/php/src/Vitess/Proto/Vschema/ColumnVindex.php new file mode 100644 index 0000000000..a7c174ecdd --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/ColumnVindex.php @@ -0,0 +1,121 @@ +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 has a value + * + * @return boolean + */ + public function hasColumn(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\ColumnVindex + */ + public function clearColumn(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getColumn(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\ColumnVindex + */ + public function setColumn( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasName(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\ColumnVindex + */ + public function clearName(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return string + */ + public function getName(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\ColumnVindex + */ + public function setName( $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Keyspace.php b/php/src/Vitess/Proto/Vschema/Keyspace.php new file mode 100644 index 0000000000..2313a6b6ba --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Keyspace.php @@ -0,0 +1,211 @@ +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 has a value + * + * @return boolean + */ + public function hasSharded(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace + */ + public function clearSharded(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return boolean + */ + public function getSharded(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param boolean $value + * @return \Vitess\Proto\Vschema\Keyspace + */ + public function setSharded( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasVindexes(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace + */ + public function clearVindexes(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry + */ + public function getVindexes($idx = NULL){ + return $this->_get(2, $idx); + } + + /** + * Set 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 + * + * @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry[] + */ + public function getVindexesList(){ + return $this->_get(2); + } + + /** + * Add a new element to + * + * @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 has a value + * + * @return boolean + */ + public function hasTables(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace + */ + public function clearTables(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Vschema\Keyspace\TablesEntry + */ + public function getTables($idx = NULL){ + return $this->_get(3, $idx); + } + + /** + * Set 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 + * + * @return \Vitess\Proto\Vschema\Keyspace\TablesEntry[] + */ + public function getTablesList(){ + return $this->_get(3); + } + + /** + * Add a new element to + * + * @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); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Keyspace/TablesEntry.php b/php/src/Vitess/Proto/Vschema/Keyspace/TablesEntry.php new file mode 100644 index 0000000000..e5fbe48674 --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Keyspace/TablesEntry.php @@ -0,0 +1,122 @@ +number = 1; + $f->name = "key"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE value = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "value"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Vschema\Table'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasKey(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace\TablesEntry + */ + public function clearKey(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getKey(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Keyspace\TablesEntry + */ + public function setKey( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasValue(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace\TablesEntry + */ + public function clearValue(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Vschema\Table + */ + public function getValue(){ + return $this->_get(2); + } + + /** + * Set 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); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Keyspace/VindexesEntry.php b/php/src/Vitess/Proto/Vschema/Keyspace/VindexesEntry.php new file mode 100644 index 0000000000..6e62ac43d0 --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Keyspace/VindexesEntry.php @@ -0,0 +1,122 @@ +number = 1; + $f->name = "key"; + $f->type = \DrSlump\Protobuf::TYPE_STRING; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $descriptor->addField($f); + + // OPTIONAL MESSAGE value = 2 + $f = new \DrSlump\Protobuf\Field(); + $f->number = 2; + $f->name = "value"; + $f->type = \DrSlump\Protobuf::TYPE_MESSAGE; + $f->rule = \DrSlump\Protobuf::RULE_OPTIONAL; + $f->reference = '\Vitess\Proto\Vschema\Vindex'; + $descriptor->addField($f); + + foreach (self::$__extensions as $cb) { + $descriptor->addField($cb(), true); + } + + return $descriptor; + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasKey(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry + */ + public function clearKey(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getKey(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry + */ + public function setKey( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasValue(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Keyspace\VindexesEntry + */ + public function clearValue(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return \Vitess\Proto\Vschema\Vindex + */ + public function getValue(){ + return $this->_get(2); + } + + /** + * Set 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); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Table.php b/php/src/Vitess/Proto/Vschema/Table.php new file mode 100644 index 0000000000..74823d07d6 --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Table.php @@ -0,0 +1,191 @@ +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 has a value + * + * @return boolean + */ + public function hasType(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Table + */ + public function clearType(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getType(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Table + */ + public function setType( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasColumnVindexes(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Table + */ + public function clearColumnVindexes(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Vschema\ColumnVindex + */ + public function getColumnVindexes($idx = NULL){ + return $this->_get(2, $idx); + } + + /** + * Set 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 + * + * @return \Vitess\Proto\Vschema\ColumnVindex[] + */ + public function getColumnVindexesList(){ + return $this->_get(2); + } + + /** + * Add a new element to + * + * @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 has a value + * + * @return boolean + */ + public function hasAutoIncrement(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Table + */ + public function clearAutoIncrement(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return \Vitess\Proto\Vschema\AutoIncrement + */ + public function getAutoIncrement(){ + return $this->_get(3); + } + + /** + * Set 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); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Vindex.php b/php/src/Vitess/Proto/Vschema/Vindex.php new file mode 100644 index 0000000000..15535e11e7 --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Vindex.php @@ -0,0 +1,190 @@ +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 has a value + * + * @return boolean + */ + public function hasType(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Vindex + */ + public function clearType(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getType(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Vindex + */ + public function setType( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasParams(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Vindex + */ + public function clearParams(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @param int $idx + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry + */ + public function getParams($idx = NULL){ + return $this->_get(2, $idx); + } + + /** + * Set 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 + * + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry[] + */ + public function getParamsList(){ + return $this->_get(2); + } + + /** + * Add a new element to + * + * @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 has a value + * + * @return boolean + */ + public function hasOwner(){ + return $this->_has(3); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Vindex + */ + public function clearOwner(){ + return $this->_clear(3); + } + + /** + * Get value + * + * @return string + */ + public function getOwner(){ + return $this->_get(3); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Vindex + */ + public function setOwner( $value){ + return $this->_set(3, $value); + } + } +} + diff --git a/php/src/Vitess/Proto/Vschema/Vindex/ParamsEntry.php b/php/src/Vitess/Proto/Vschema/Vindex/ParamsEntry.php new file mode 100644 index 0000000000..c81c9fb0ca --- /dev/null +++ b/php/src/Vitess/Proto/Vschema/Vindex/ParamsEntry.php @@ -0,0 +1,121 @@ +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 has a value + * + * @return boolean + */ + public function hasKey(){ + return $this->_has(1); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry + */ + public function clearKey(){ + return $this->_clear(1); + } + + /** + * Get value + * + * @return string + */ + public function getKey(){ + return $this->_get(1); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry + */ + public function setKey( $value){ + return $this->_set(1, $value); + } + + /** + * Check if has a value + * + * @return boolean + */ + public function hasValue(){ + return $this->_has(2); + } + + /** + * Clear value + * + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry + */ + public function clearValue(){ + return $this->_clear(2); + } + + /** + * Get value + * + * @return string + */ + public function getValue(){ + return $this->_get(2); + } + + /** + * Set value + * + * @param string $value + * @return \Vitess\Proto\Vschema\Vindex\ParamsEntry + */ + public function setValue( $value){ + return $this->_set(2, $value); + } + } +} + diff --git a/proto/topodata.proto b/proto/topodata.proto index d1b4b70ec0..88e21a1613 100644 --- a/proto/topodata.proto +++ b/proto/topodata.proto @@ -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; }