diff --git a/go/vt/vttablet/tabletmanager/action_agent.go b/go/vt/vttablet/tabletmanager/action_agent.go index a1c3f9ad1b0..26ce7433b22 100644 --- a/go/vt/vttablet/tabletmanager/action_agent.go +++ b/go/vt/vttablet/tabletmanager/action_agent.go @@ -165,19 +165,19 @@ type ActionAgent struct { // It's only set once in NewActionAgent() and never modified after that. orc *orcClient - // shardSyncChan is a channel for informing the shard sync goroutine that + // mutex protects all the following fields (that start with '_'), + // only hold the mutex to update the fields, nothing else. + mutex sync.Mutex + + // _shardSyncChan is a channel for informing the shard sync goroutine that // it should wake up and recheck the tablet state, to make sure it and the // shard record are in sync. // // Call agent.notifyShardSync() instead of sending directly to this channel. - shardSyncChan chan struct{} + _shardSyncChan chan struct{} - // shardSyncCancel is the function to stop the background shard sync goroutine. - shardSyncCancel context.CancelFunc - - // mutex protects all the following fields (that start with '_'), - // only hold the mutex to update the fields, nothing else. - mutex sync.Mutex + // _shardSyncCancel is the function to stop the background shard sync goroutine. + _shardSyncCancel context.CancelFunc // _tablet has the Tablet record we last read from the topology server. _tablet *topodatapb.Tablet diff --git a/go/vt/vttablet/tabletmanager/shard_sync.go b/go/vt/vttablet/tabletmanager/shard_sync.go index 914e536efab..31aea282cb9 100644 --- a/go/vt/vttablet/tabletmanager/shard_sync.go +++ b/go/vt/vttablet/tabletmanager/shard_sync.go @@ -50,7 +50,9 @@ var ( // a notification signal from setTablet(). func (agent *ActionAgent) shardSyncLoop(ctx context.Context) { // Make a copy of the channel so we don't race when stopShardSync() clears it. - notifyChan := agent.shardSyncChan + agent.mutex.Lock() + notifyChan := agent._shardSyncChan + agent.mutex.Unlock() // retryChan is how we wake up after going to sleep between retries. // If no retry is pending, this channel will be nil, which means it's fine @@ -224,9 +226,11 @@ func (agent *ActionAgent) startShardSync() { // even if the receiver is busy. We can drop any additional send attempts // if the buffer is full because all we care about is that the receiver will // be told it needs to recheck the state. - agent.shardSyncChan = make(chan struct{}, 1) + agent.mutex.Lock() + agent._shardSyncChan = make(chan struct{}, 1) ctx, cancel := context.WithCancel(context.Background()) - agent.shardSyncCancel = cancel + agent._shardSyncCancel = cancel + agent.mutex.Unlock() // Queue up a pending notification to force the loop to run once at startup. agent.notifyShardSync() @@ -236,23 +240,28 @@ func (agent *ActionAgent) startShardSync() { } func (agent *ActionAgent) stopShardSync() { - if agent.shardSyncCancel != nil { - agent.shardSyncCancel() - agent.shardSyncCancel = nil - agent.shardSyncChan = nil + agent.mutex.Lock() + if agent._shardSyncCancel != nil { + agent._shardSyncCancel() + agent._shardSyncCancel = nil + agent._shardSyncChan = nil } + agent.mutex.Unlock() } func (agent *ActionAgent) notifyShardSync() { // If this is called before the shard sync is started, do nothing. - if agent.shardSyncChan == nil { + agent.mutex.Lock() + defer agent.mutex.Unlock() + + if agent._shardSyncChan == nil { return } // Try to send. If the channel buffer is full, it means a notification is // already pending, so we don't need to do anything. select { - case agent.shardSyncChan <- struct{}{}: + case agent._shardSyncChan <- struct{}{}: default: } } diff --git a/go/vt/wrangler/testlib/external_reparent_test.go b/go/vt/wrangler/testlib/external_reparent_test.go new file mode 100644 index 00000000000..4bf726d223e --- /dev/null +++ b/go/vt/wrangler/testlib/external_reparent_test.go @@ -0,0 +1,468 @@ +/* +Copyright 2017 Google Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package testlib + +import ( + "testing" + "time" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vttablet/tmclient" + "vitess.io/vitess/go/vt/wrangler" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +// The tests in this package test the wrangler version of TabletExternallyReparented +// This is the one that is now called by the vtctl command + +// TestTabletExternallyReparentedBasic tests the base cases for TER +func TestTabletExternallyReparentedBasic(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("cell1") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + vp := NewVtctlPipe(t, ts) + defer vp.Close() + + // Create an old master, a new master, two good replicas, one bad replica + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + + // Build keyspace graph + err := topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted. + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + // First test: reparent to the same master, make sure it works + // as expected. + if err := vp.Run([]string{"TabletExternallyReparented", topoproto.TabletAliasString(oldMaster.Tablet.Alias)}); err != nil { + t.Fatalf("TabletExternallyReparented(same master) should have worked: %v", err) + } + + // check the old master is still master + tablet, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("old master should be MASTER but is: %v", tablet.Type) + } + + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + + // This tests the good case, where everything works as planned + t.Logf("TabletExternallyReparented(new master) expecting success") + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + } + + // check the new master is master + tablet, err = ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + + // We have to wait for shard sync to do its magic in the background + timer := time.NewTimer(10 * time.Second) + defer timer.Stop() + +loop: + for { + select { + case <-timer.C: + // we timed out + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + t.Fatalf("old master (%v) should be replica but is: %v", topoproto.TabletAliasString(oldMaster.Tablet.Alias), tablet.Type) + default: + // check the old master was converted to replica + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_REPLICA { + time.Sleep(100 * time.Millisecond) + } else { + break loop + } + } + } +} + +func TestTabletExternallyReparentedToSlave(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("cell1") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, a new master, two good replicas, one bad replica + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + newMaster.FakeMysqlDaemon.ReadOnly = true + newMaster.FakeMysqlDaemon.Replicating = true + + // Build keyspace graph + err := topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted. + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + // Second test: reparent to a replica, and pretend the old + // master is still good to go. + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + + // This tests a bad case: the new designated master is a slave at mysql level, + // but we should do what we're told anyway. + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) error: %v", err) + } + + // check that newMaster is master + tablet, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + + // We have to wait for shard sync to do its magic in the background + timer := time.NewTimer(10 * time.Second) + defer timer.Stop() + +loop: + for { + select { + case <-timer.C: + // we timed out + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + t.Fatalf("old master (%v) should be replica but is: %v", topoproto.TabletAliasString(oldMaster.Tablet.Alias), tablet.Type) + default: + // check the old master was converted to replica + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_REPLICA { + time.Sleep(100 * time.Millisecond) + } else { + break loop + } + } + } +} + +// TestTabletExternallyReparentedWithDifferentMysqlPort makes sure +// that if mysql is restarted on the master-elect tablet and has a different +// port, we pick it up correctly. +func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("cell1") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, a new master, two good replicas, one bad replica + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + goodReplica := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + // Now we're restarting mysql on a different port, 3301->3303 + // but without updating the Tablet record in topology. + + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted, so we need a MysqlDaemon + // that returns no master, and the new port (as returned by mysql) + newMaster.FakeMysqlDaemon.MysqlPort = 3303 + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted and point to the new mysql port + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + // On the good replicas, we will respond to + // TabletActionSlaveWasRestarted and point to the new mysql port + goodReplica.StartActionLoop(t, wr) + defer goodReplica.StopActionLoop(t) + + // This tests the good case, where everything works as planned + t.Logf("TabletExternallyReparented(new master) expecting success") + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + } + // check the new master is master + tablet, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + + // We have to wait for shard sync to do its magic in the background + timer := time.NewTimer(10 * time.Second) + defer timer.Stop() + +loop: + for { + select { + case <-timer.C: + // we timed out + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + t.Fatalf("old master (%v) should be replica but is: %v", topoproto.TabletAliasString(oldMaster.Tablet.Alias), tablet.Type) + default: + // check the old master was converted to replica + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_REPLICA { + time.Sleep(100 * time.Millisecond) + } else { + break loop + } + } + } +} + +// TestTabletExternallyReparentedContinueOnUnexpectedMaster makes sure +// that we ignore mysql's master if the flag is set +func TestTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("cell1") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, a new master, two good replicas, one bad replica + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + goodReplica := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted, so we need a MysqlDaemon + // that returns no master, and the new port (as returned by mysql) + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted and point to a bad host + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + // On the good replica, we will respond to + // TabletActionSlaveWasRestarted and point to a bad host + goodReplica.StartActionLoop(t, wr) + defer goodReplica.StopActionLoop(t) + + // This tests the good case, where everything works as planned + t.Logf("TabletExternallyReparented(new master) expecting success") + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + } + // check the new master is master + tablet, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + // We have to wait for shard sync to do its magic in the background + timer := time.NewTimer(10 * time.Second) + defer timer.Stop() + +loop: + for { + select { + case <-timer.C: + // we timed out + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + t.Fatalf("old master (%v) should be replica but is: %v", topoproto.TabletAliasString(oldMaster.Tablet.Alias), tablet.Type) + default: + // check the old master was converted to replica + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_REPLICA { + time.Sleep(100 * time.Millisecond) + } else { + break loop + } + } + } +} + +func TestTabletExternallyReparentedRerun(t *testing.T) { + ctx := context.Background() + ts := memorytopo.NewServer("cell1") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, a new master, and a good replica. + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + goodReplica := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted. + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted. + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + goodReplica.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + // On the good replica, we will respond to + // TabletActionSlaveWasRestarted. + goodReplica.StartActionLoop(t, wr) + defer goodReplica.StopActionLoop(t) + + // The reparent should work as expected here + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + } + + // check the new master is master + tablet, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + + // We have to wait for shard sync to do its magic in the background + timer := time.NewTimer(10 * time.Second) + defer timer.Stop() + +loop: + for { + select { + case <-timer.C: + // we timed out + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + t.Fatalf("old master (%v) should be replica but is: %v", topoproto.TabletAliasString(oldMaster.Tablet.Alias), tablet.Type) + default: + // check the old master was converted to replica + tablet, err = ts.GetTablet(ctx, oldMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", oldMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_REPLICA { + time.Sleep(100 * time.Millisecond) + } else { + break loop + } + } + } + + // run TER again and make sure the master is still correct + if err := wr.TabletExternallyReparented(ctx, newMaster.Tablet.Alias); err != nil { + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + } + + // check the new master is still master + tablet, err = ts.GetTablet(ctx, newMaster.Tablet.Alias) + if err != nil { + t.Fatalf("GetTablet(%v) failed: %v", newMaster.Tablet.Alias, err) + } + if tablet.Type != topodatapb.TabletType_MASTER { + t.Fatalf("new master should be MASTER but is: %v", tablet.Type) + } + +} diff --git a/go/vt/wrangler/testlib/find_tablet_test.go b/go/vt/wrangler/testlib/find_tablet_test.go new file mode 100644 index 00000000000..d6835ce053c --- /dev/null +++ b/go/vt/wrangler/testlib/find_tablet_test.go @@ -0,0 +1,92 @@ +/* +Copyright 2017 Google Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package testlib + +import ( + "testing" + "time" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vttablet/tabletmanager" + "vitess.io/vitess/go/vt/vttablet/tmclient" + "vitess.io/vitess/go/vt/wrangler" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +func TestFindTablet(t *testing.T) { + tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) + + ctx := context.Background() + ts := memorytopo.NewServer("cell1", "cell2") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, two good slaves + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + goodSlave1 := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + goodSlave2 := NewFakeTablet(t, wr, "cell2", 3, topodatapb.TabletType_REPLICA, nil) + + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1", "cell2"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + + // make sure we can find the tablets + // even with a datacenter being down. + tabletMap, err := ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell1"}) + if err != nil { + t.Fatalf("GetTabletMapForShardByCell should have worked but got: %v", err) + } + master, err := topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) + if err != nil || !topoproto.TabletAliasEqual(master, oldMaster.Tablet.Alias) { + t.Fatalf("FindTabletByHostAndPort(master) failed: %v %v", err, master) + } + slave1, err := topotools.FindTabletByHostAndPort(tabletMap, goodSlave1.Tablet.Hostname, "vt", goodSlave1.Tablet.PortMap["vt"]) + if err != nil || !topoproto.TabletAliasEqual(slave1, goodSlave1.Tablet.Alias) { + t.Fatalf("FindTabletByHostAndPort(slave1) failed: %v %v", err, master) + } + slave2, err := topotools.FindTabletByHostAndPort(tabletMap, goodSlave2.Tablet.Hostname, "vt", goodSlave2.Tablet.PortMap["vt"]) + if !topo.IsErrType(err, topo.NoNode) { + t.Fatalf("FindTabletByHostAndPort(slave2) worked: %v %v", err, slave2) + } + + // Make sure the master is not exported in other cells + tabletMap, _ = ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell2"}) + master, err = topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) + if !topo.IsErrType(err, topo.NoNode) { + t.Fatalf("FindTabletByHostAndPort(master) worked in cell2: %v %v", err, master) + } + + // Get tablet map for all cells. If there were to be failures talking to local cells, this will return the tablet map + // and forward a partial result error + tabletMap, err = ts.GetTabletMapForShard(ctx, "test_keyspace", "0") + if err != nil { + t.Fatalf("GetTabletMapForShard should nil but got: %v", err) + } + master, err = topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) + if err != nil || !topoproto.TabletAliasEqual(master, oldMaster.Tablet.Alias) { + t.Fatalf("FindTabletByHostAndPort(master) failed: %v %v", err, master) + } + +} diff --git a/go/vt/wrangler/testlib/rpc_reparent_external_test.go b/go/vt/wrangler/testlib/rpc_reparent_external_test.go index 1d871a8dec3..ed9aeb977a2 100644 --- a/go/vt/wrangler/testlib/rpc_reparent_external_test.go +++ b/go/vt/wrangler/testlib/rpc_reparent_external_test.go @@ -26,7 +26,6 @@ import ( "vitess.io/vitess/go/event" "vitess.io/vitess/go/vt/logutil" - "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" @@ -38,19 +37,16 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -func TestTabletExternallyReparented(t *testing.T) { +func TestRPCTabletExternallyReparented(t *testing.T) { tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) ctx := context.Background() ts := memorytopo.NewServer("cell1", "cell2") wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) - // Create an old master, a new master, two good slaves, one bad slave + // Create an old master and a new master oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) - newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) - goodSlave1 := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) - goodSlave2 := NewFakeTablet(t, wr, "cell2", 3, topodatapb.TabletType_REPLICA, nil) - badSlave := NewFakeTablet(t, wr, "cell1", 4, topodatapb.TabletType_REPLICA, nil) + newMaster := NewFakeTablet(t, wr, "cell2", 1, topodatapb.TabletType_REPLICA, nil) // Build keyspace graph err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1", "cell2"}) @@ -58,66 +54,21 @@ func TestTabletExternallyReparented(t *testing.T) { t.Fatalf("RebuildKeyspaceLocked failed: %v", err) } - // Slightly unrelated test: make sure we can find the tablets - // even with a datacenter being down. - tabletMap, err := ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell1"}) - if err != nil { - t.Fatalf("GetTabletMapForShardByCell should have worked but got: %v", err) - } - master, err := topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) - if err != nil || !topoproto.TabletAliasEqual(master, oldMaster.Tablet.Alias) { - t.Fatalf("FindTabletByHostAndPort(master) failed: %v %v", err, master) - } - slave1, err := topotools.FindTabletByHostAndPort(tabletMap, goodSlave1.Tablet.Hostname, "vt", goodSlave1.Tablet.PortMap["vt"]) - if err != nil || !topoproto.TabletAliasEqual(slave1, goodSlave1.Tablet.Alias) { - t.Fatalf("FindTabletByHostAndPort(slave1) failed: %v %v", err, master) - } - slave2, err := topotools.FindTabletByHostAndPort(tabletMap, goodSlave2.Tablet.Hostname, "vt", goodSlave2.Tablet.PortMap["vt"]) - if !topo.IsErrType(err, topo.NoNode) { - t.Fatalf("FindTabletByHostAndPort(slave2) worked: %v %v", err, slave2) - } - - // Make sure the master is not exported in other cells - tabletMap, _ = ts.GetTabletMapForShardByCell(ctx, "test_keyspace", "0", []string{"cell2"}) - master, err = topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) - if !topo.IsErrType(err, topo.NoNode) { - t.Fatalf("FindTabletByHostAndPort(master) worked in cell2: %v %v", err, master) - } - - // Get tablet map for all cells. If there were to be failures talking to local cells, this will return the tablet map - // and forward a partial result error - tabletMap, err = ts.GetTabletMapForShard(ctx, "test_keyspace", "0") - if err != nil { - t.Fatalf("GetTabletMapForShard should nil but got: %v", err) - } - master, err = topotools.FindTabletByHostAndPort(tabletMap, oldMaster.Tablet.Hostname, "vt", oldMaster.Tablet.PortMap["vt"]) - if err != nil || !topoproto.TabletAliasEqual(master, oldMaster.Tablet.Alias) { - t.Fatalf("FindTabletByHostAndPort(master) failed: %v %v", err, master) - } - // On the elected master, we will respond to // TabletActionSlaveWasPromoted newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted. oldMaster.StartActionLoop(t, wr) defer oldMaster.StopActionLoop(t) - // On the good slaves, we will respond to - // TabletActionSlaveWasRestarted. - goodSlave1.StartActionLoop(t, wr) - defer goodSlave1.StopActionLoop(t) - - goodSlave2.StartActionLoop(t, wr) - defer goodSlave2.StopActionLoop(t) - - // On the bad slave, we will respond to - // TabletActionSlaveWasRestarted with bad data. - badSlave.StartActionLoop(t, wr) - defer badSlave.StopActionLoop(t) - // First test: reparent to the same master, make sure it works // as expected. tmc := tmclient.NewTabletManagerClient() @@ -129,39 +80,74 @@ func TestTabletExternallyReparented(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), oldMaster.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(same master) should have worked: %v", err) } + waitForExternalReparent(t, "TestRPCTabletExternallyReparented: same master", waitID) - // Second test: reparent to a replica, and pretend the old - // master is still good to go. - - // This tests a bad case: the new designated master is a slave, - // but we should do what we're told anyway. - ti, err := ts.GetTablet(ctx, goodSlave1.Tablet.Alias) + // This tests the good case, where everything works as planned + t.Logf("TabletExternallyReparented(new master) expecting success") + ti, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) if err != nil { t.Fatalf("GetTablet failed: %v", err) } waitID = makeWaitID() if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { - t.Fatalf("TabletExternallyReparented(slave) error: %v", err) + t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparented: slave designated as master", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparented: good case", waitID) +} - // This tests the good case, where everything works as planned - t.Logf("TabletExternallyReparented(new master) expecting success") - ti, err = ts.GetTablet(ctx, newMaster.Tablet.Alias) +func TestRPCTabletExternallyReparentedSlaveMysql(t *testing.T) { + tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) + + ctx := context.Background() + ts := memorytopo.NewServer("cell1", "cell2") + wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient()) + + // Create an old master, a new master, two good slaves, one bad slave + oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil) + newMaster := NewFakeTablet(t, wr, "cell2", 1, topodatapb.TabletType_REPLICA, nil) + newMaster.FakeMysqlDaemon.ReadOnly = true + newMaster.FakeMysqlDaemon.Replicating = true + + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1", "cell2"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } + + // On the elected master, we will respond to + // TabletActionSlaveWasPromoted + newMaster.StartActionLoop(t, wr) + defer newMaster.StopActionLoop(t) + + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } + // On the old master, we will only respond to + // TabletActionSlaveWasRestarted. + oldMaster.StartActionLoop(t, wr) + defer oldMaster.StopActionLoop(t) + + // This tests a bad case: the new designated master is a slave, + // but we should do what we're told anyway. + tmc := tmclient.NewTabletManagerClient() + ti, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) if err != nil { t.Fatalf("GetTablet failed: %v", err) } - waitID = makeWaitID() + waitID := makeWaitID() if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { - t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) + t.Fatalf("TabletExternallyReparented(slave) error: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparented: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparented: slave designated as master", waitID) + } -// TestTabletExternallyReparentedWithDifferentMysqlPort makes sure +// TestRPCTabletExternallyReparentedWithDifferentMysqlPort makes sure // that if mysql is restarted on the master-elect tablet and has a different // port, we pick it up correctly. -func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { +func TestRPCTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) ctx := context.Background() @@ -173,6 +159,11 @@ func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) goodSlave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // Now we're restarting mysql on a different port, 3301->3303 // but without updating the Tablet record in topology. @@ -183,6 +174,11 @@ func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted and point to the new mysql port oldMaster.StartActionLoop(t, wr) @@ -204,12 +200,12 @@ func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedWithDifferentMysqlPort: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedWithDifferentMysqlPort: good case", waitID) } -// TestTabletExternallyReparentedContinueOnUnexpectedMaster makes sure +// TestRPCTabletExternallyReparentedContinueOnUnexpectedMaster makes sure // that we ignore mysql's master if the flag is set -func TestTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) { +func TestRPCTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) { tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) ctx := context.Background() @@ -221,12 +217,22 @@ func TestTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) { newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) goodSlave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // On the elected master, we will respond to // TabletActionSlaveWasPromoted, so we need a MysqlDaemon // that returns no master, and the new port (as returned by mysql) newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted and point to a bad host oldMaster.StartActionLoop(t, wr) @@ -248,10 +254,10 @@ func TestTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedContinueOnUnexpectedMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedContinueOnUnexpectedMaster: good case", waitID) } -func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) { +func TestRPCTabletExternallyReparentedFailedOldMaster(t *testing.T) { // The 'RefreshState' call on the old master will timeout on // this value, so it has to be smaller than the 10s of the // wait for the 'finished' state of waitForExternalReparent. @@ -266,6 +272,11 @@ func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) { newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) goodSlave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // Reparent to a replica, and pretend the old master is not responding. // On the elected master, we will respond to @@ -275,6 +286,11 @@ func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) { // On the old master, we will only get a RefreshState call, // let's just not respond to it at all, and let it timeout. + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the good slave, we will respond to // TabletActionSlaveWasRestarted. @@ -291,7 +307,7 @@ func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedFailedOldMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedFailedOldMaster: good case", waitID) // check the old master was converted to replica tablet, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) @@ -303,7 +319,7 @@ func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) { } } -func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { +func TestRPCTabletExternallyReparentedImpostorMaster(t *testing.T) { tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) ctx := context.Background() @@ -316,6 +332,11 @@ func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil, ForceInitTablet()) newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // check the old master is really master tablet, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) if err != nil { @@ -331,7 +352,7 @@ func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { t.Fatalf("GetTablet(%v) failed: %v", badSlave.Tablet.Alias, err) } if tablet.Type != topodatapb.TabletType_MASTER { - t.Fatalf("old master should be MASTER but is: %v", tablet.Type) + t.Fatalf("impostor should be MASTER but is: %v", tablet.Type) } // On the elected master, we will respond to @@ -339,11 +360,23 @@ func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted. oldMaster.StartActionLoop(t, wr) defer oldMaster.StopActionLoop(t) + // set this to old master because as soon as badSlave starts, it detects that + // there is a master with a later timestamp and demotes itself + badSlave.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(oldMaster.Tablet) + badSlave.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the bad slave, we will respond to // TabletActionSlaveWasRestarted. badSlave.StartActionLoop(t, wr) @@ -359,7 +392,7 @@ func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedImpostorMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedImpostorMaster: good case", waitID) // check the new master is really master tablet, err = ts.GetTablet(ctx, newMaster.Tablet.Alias) @@ -389,7 +422,7 @@ func TestTabletExternallyReparentedImpostorMaster(t *testing.T) { } } -func TestTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { +func TestRPCTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { tabletmanager.SetReparentFlags(2 * time.Second /* finalizeTimeout */) ctx := context.Background() @@ -402,6 +435,11 @@ func TestTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { oldMaster := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, nil, ForceInitTablet()) newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // check the old master is really master tablet, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) if err != nil { @@ -425,13 +463,16 @@ func TestTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted. oldMaster.StartActionLoop(t, wr) defer oldMaster.StopActionLoop(t) - // Reparent to a replica, and pretend the impostor master is not responding. - // The reparent should work as expected here tmc := tmclient.NewTabletManagerClient() ti, err := ts.GetTablet(ctx, newMaster.Tablet.Alias) @@ -442,7 +483,7 @@ func TestTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedImpostorMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedImpostorMaster: good case", waitID) // check the new master is really master tablet, err = ts.GetTablet(ctx, newMaster.Tablet.Alias) @@ -472,7 +513,7 @@ func TestTabletExternallyReparentedFailedImpostorMaster(t *testing.T) { } } -func TestTabletExternallyReparentedRerun(t *testing.T) { +func TestRPCTabletExternallyReparentedRerun(t *testing.T) { tabletmanager.SetReparentFlags(time.Minute /* finalizeTimeout */) ctx := context.Background() @@ -484,13 +525,21 @@ func TestTabletExternallyReparentedRerun(t *testing.T) { newMaster := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, nil) goodSlave := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, nil) - // Reparent to a replica, and pretend the old master is not responding. - + // Build keyspace graph + err := topotools.RebuildKeyspace(context.Background(), logutil.NewConsoleLogger(), ts, oldMaster.Tablet.Keyspace, []string{"cell1"}) + if err != nil { + t.Fatalf("RebuildKeyspaceLocked failed: %v", err) + } // On the elected master, we will respond to // TabletActionSlaveWasPromoted. newMaster.StartActionLoop(t, wr) defer newMaster.StopActionLoop(t) + oldMaster.FakeMysqlDaemon.SetMasterInput = topoproto.MysqlAddr(newMaster.Tablet) + oldMaster.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{ + "FAKE SET MASTER", + "START SLAVE", + } // On the old master, we will only respond to // TabletActionSlaveWasRestarted. oldMaster.StartActionLoop(t, wr) @@ -511,7 +560,7 @@ func TestTabletExternallyReparentedRerun(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedFailedOldMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedFailedOldMaster: good case", waitID) // check the old master was converted to replica tablet, err := ts.GetTablet(ctx, oldMaster.Tablet.Alias) @@ -527,7 +576,7 @@ func TestTabletExternallyReparentedRerun(t *testing.T) { if err := tmc.TabletExternallyReparented(context.Background(), ti.Tablet, waitID); err != nil { t.Fatalf("TabletExternallyReparented(replica) failed: %v", err) } - waitForExternalReparent(t, "TestTabletExternallyReparentedFailedOldMaster: good case", waitID) + waitForExternalReparent(t, "TestRPCTabletExternallyReparentedFailedOldMaster: good case", waitID) // check the new master is still master tablet, err = ts.GetTablet(ctx, newMaster.Tablet.Alias)