func waitForHealthyRdonlyTablets(ctx context.Context, wr *wrangler.Wrangler, healthCheck discovery.HealthCheck, cell, keyspace, shard string, minHealthyRdonlyTablets int, timeout time.Duration) ([]*discovery.TabletStats, error) { busywaitCtx, busywaitCancel := context.WithTimeout(ctx, timeout) defer busywaitCancel() start := time.Now() deadlineForLog, _ := busywaitCtx.Deadline() wr.Logger().Infof("Waiting for enough healthy RDONLY tablets to become available in (%v,%v/%v). required: %v Waiting up to %.1f seconds.", cell, keyspace, shard, minHealthyRdonlyTablets, deadlineForLog.Sub(time.Now()).Seconds()) // Wait for at least one RDONLY tablet initially before checking the list. if err := discovery.WaitForTablets(busywaitCtx, healthCheck, cell, keyspace, shard, []topodatapb.TabletType{topodatapb.TabletType_RDONLY}); err != nil { return nil, fmt.Errorf("error waiting for RDONLY tablets for (%v,%v/%v): %v", cell, keyspace, shard, err) } var healthyTablets []*discovery.TabletStats for { select { case <-busywaitCtx.Done(): return nil, fmt.Errorf("not enough healthy RDONLY tablets to choose from in (%v,%v/%v), have %v healthy ones, need at least %v Context error: %v", cell, keyspace, shard, len(healthyTablets), minHealthyRdonlyTablets, busywaitCtx.Err()) default: } healthyTablets = discovery.RemoveUnhealthyTablets( healthCheck.GetTabletStatsFromTarget(keyspace, shard, topodatapb.TabletType_RDONLY)) if len(healthyTablets) >= minHealthyRdonlyTablets { break } deadlineForLog, _ := busywaitCtx.Deadline() wr.Logger().Infof("Waiting for enough healthy RDONLY tablets to become available (%v,%v/%v). available: %v required: %v Waiting up to %.1f more seconds.", cell, keyspace, shard, len(healthyTablets), minHealthyRdonlyTablets, deadlineForLog.Sub(time.Now()).Seconds()) // Block for 1 second because 2 seconds is the -health_check_interval flag value in integration tests. timer := time.NewTimer(1 * time.Second) select { case <-busywaitCtx.Done(): timer.Stop() case <-timer.C: } } wr.Logger().Infof("At least %v healthy RDONLY tablets are available in (%v,%v/%v) (required: %v). Took %.1f seconds to find this out.", len(healthyTablets), cell, keyspace, shard, minHealthyRdonlyTablets, time.Now().Sub(start).Seconds()) return healthyTablets, nil }
// Iteration is a single iteration for the player: get the current status, // try to play, and plays until interrupted, or until an error occurs. func (bpc *BinlogPlayerController) Iteration() (err error) { defer func() { if x := recover(); x != nil { log.Errorf("%v: caught panic: %v\n%s", bpc, x, tb.Stack(4)) err = fmt.Errorf("panic: %v", x) } }() // Check if the context is still good. select { case <-bpc.ctx.Done(): if bpc.ctx.Err() == context.Canceled { // We were stopped. Break out of Loop(). return nil } return fmt.Errorf("giving up since the context is done: %v", bpc.ctx.Err()) default: } // Apply any special settings necessary for playback of binlogs. // We do it on every iteration to be sure, in case MySQL was restarted. if err := bpc.mysqld.EnableBinlogPlayback(); err != nil { // We failed to apply the required settings, so we shouldn't keep going. return err } // create the db connection, connect it vtClient := bpc.vtClientFactory() if err := vtClient.Connect(); err != nil { return fmt.Errorf("can't connect to database: %v", err) } defer vtClient.Close() // Read the start position startPosition, flags, err := binlogplayer.ReadStartPosition(vtClient, bpc.sourceShard.Uid) if err != nil { return fmt.Errorf("can't read startPosition: %v", err) } // if we shouldn't start, we just error out and try again later if strings.Index(flags, binlogplayer.BlpFlagDontStart) != -1 { return fmt.Errorf("not starting because flag '%v' is set", binlogplayer.BlpFlagDontStart) } // wait for the tablet set (usefull for the first run at least, fast for next runs) if err := discovery.WaitForTablets(bpc.ctx, bpc.healthCheck, bpc.cell, bpc.sourceShard.Keyspace, bpc.sourceShard.Shard, []topodatapb.TabletType{topodatapb.TabletType_REPLICA}); err != nil { return fmt.Errorf("error waiting for tablets for %v %v %v: %v", bpc.cell, bpc.sourceShard.String(), topodatapb.TabletType_REPLICA, err) } // Find the server list from the health check addrs := discovery.RemoveUnhealthyTablets( bpc.healthCheck.GetTabletStatsFromTarget(bpc.sourceShard.Keyspace, bpc.sourceShard.Shard, topodatapb.TabletType_REPLICA)) if len(addrs) == 0 { return fmt.Errorf("can't find any healthy source tablet for %v %v %v", bpc.cell, bpc.sourceShard.String(), topodatapb.TabletType_REPLICA) } newServerIndex := rand.Intn(len(addrs)) tablet := addrs[newServerIndex].Tablet // save our current server bpc.playerMutex.Lock() bpc.sourceTablet = tablet.Alias bpc.lastError = nil bpc.playerMutex.Unlock() // check which kind of replication we're doing, tables or keyrange if len(bpc.sourceShard.Tables) > 0 { // tables, first resolve wildcards tables, err := mysqlctl.ResolveTables(bpc.mysqld, bpc.dbName, bpc.sourceShard.Tables) if err != nil { return fmt.Errorf("failed to resolve table names: %v", err) } // tables, just get them player, err := binlogplayer.NewBinlogPlayerTables(vtClient, tablet, tables, bpc.sourceShard.Uid, startPosition, bpc.stopPosition, bpc.binlogPlayerStats) if err != nil { return fmt.Errorf("NewBinlogPlayerTables failed: %v", err) } return player.ApplyBinlogEvents(bpc.ctx) } // the data we have to replicate is the intersection of the // source keyrange and our keyrange overlap, err := key.KeyRangesOverlap(bpc.sourceShard.KeyRange, bpc.keyRange) if err != nil { return fmt.Errorf("Source shard %v doesn't overlap destination shard %v", bpc.sourceShard.KeyRange, bpc.keyRange) } player, err := binlogplayer.NewBinlogPlayerKeyRange(vtClient, tablet, overlap, bpc.sourceShard.Uid, startPosition, bpc.stopPosition, bpc.binlogPlayerStats) if err != nil { return fmt.Errorf("NewBinlogPlayerKeyRange failed: %v", err) } return player.ApplyBinlogEvents(bpc.ctx) }
func (wr *Wrangler) waitForDrainInCell(ctx context.Context, cell, keyspace, shard string, servedType topodatapb.TabletType, retryDelay, healthCheckTopologyRefresh, healthcheckRetryDelay, healthCheckTimeout time.Duration) error { hc := discovery.NewHealthCheck(healthCheckTimeout /* connectTimeout */, healthcheckRetryDelay, healthCheckTimeout) defer hc.Close() watcher := discovery.NewShardReplicationWatcher(wr.TopoServer(), hc, cell, keyspace, shard, healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) defer watcher.Stop() if err := discovery.WaitForTablets(ctx, hc, cell, keyspace, shard, []topodatapb.TabletType{servedType}); err != nil { return fmt.Errorf("%v: error waiting for initial %v tablets for %v/%v: %v", cell, servedType, keyspace, shard, err) } wr.Logger().Infof("%v: Waiting for %.1f seconds to make sure that the discovery module retrieves healthcheck information from all tablets.", cell, healthCheckTimeout.Seconds()) // Wait at least for -vtctl_healthcheck_timeout to elapse to make sure that we // see all healthy tablets. Otherwise, we might miss some tablets. // It's safe to wait not longer for this because we would only miss slow // tablets and vtgate would not serve from such tablets anyway. time.Sleep(healthCheckTimeout) // Now check the QPS rate of all tablets until the timeout expires. startTime := time.Now() for { // map key: tablet uid drainedHealthyTablets := make(map[uint32]*discovery.TabletStats) notDrainedHealtyTablets := make(map[uint32]*discovery.TabletStats) healthyTablets := discovery.RemoveUnhealthyTablets( hc.GetTabletStatsFromTarget(keyspace, shard, servedType)) for _, ts := range healthyTablets { if ts.Stats.Qps == 0.0 { drainedHealthyTablets[ts.Tablet.Alias.Uid] = ts } else { notDrainedHealtyTablets[ts.Tablet.Alias.Uid] = ts } } if len(drainedHealthyTablets) == len(healthyTablets) { wr.Logger().Infof("%v: All %d healthy tablets were drained after %.1f seconds (not counting %.1f seconds for the initial wait).", cell, len(healthyTablets), time.Now().Sub(startTime).Seconds(), healthCheckTimeout.Seconds()) break } // Continue waiting, sleep in between. deadlineString := "" if d, ok := ctx.Deadline(); ok { deadlineString = fmt.Sprintf(" up to %.1f more seconds", d.Sub(time.Now()).Seconds()) } wr.Logger().Infof("%v: Waiting%v for all healthy tablets to be drained (%d/%d done).", cell, deadlineString, len(drainedHealthyTablets), len(healthyTablets)) timer := time.NewTimer(retryDelay) select { case <-ctx.Done(): timer.Stop() var l []string for _, ts := range notDrainedHealtyTablets { l = append(l, formatTabletStats(ts)) } return fmt.Errorf("%v: WaitForDrain failed for %v tablets in %v/%v. Only %d/%d tablets were drained. err: %v List of tablets which were not drained: %v", cell, servedType, keyspace, shard, len(drainedHealthyTablets), len(healthyTablets), ctx.Err(), strings.Join(l, ";")) case <-timer.C: } } return nil }
// copy phase: // - copy the data from source tablets to destination masters (with replication on) // Assumes that the schema has already been created on each destination tablet // (probably from vtctl's CopySchemaShard) func (scw *SplitCloneWorker) clone(ctx context.Context, state StatusWorkerState) error { if state != WorkerStateCloneOnline && state != WorkerStateCloneOffline { panic(fmt.Sprintf("invalid state passed to clone(): %v", state)) } scw.setState(state) start := time.Now() defer func() { statsStateDurationsNs.Set(string(state), time.Now().Sub(start).Nanoseconds()) }() var firstSourceTablet *topodatapb.Tablet if state == WorkerStateCloneOffline { // Use the first source tablet which we took offline. firstSourceTablet = scw.sourceTablets[0] } else { // Pick any healthy serving source tablet. si := scw.sourceShards[0] tablets := discovery.RemoveUnhealthyTablets( scw.healthCheck.GetTabletStatsFromTarget(si.Keyspace(), si.ShardName(), topodatapb.TabletType_RDONLY)) if len(tablets) == 0 { // We fail fast on this problem and don't retry because at the start all tablets should be healthy. return fmt.Errorf("no healthy RDONLY tablet in source shard (%v) available (required to find out the schema)", topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName())) } firstSourceTablet = tablets[0].Tablet } var statsCounters []*stats.Counters var tableStatusList *tableStatusList switch state { case WorkerStateCloneOnline: statsCounters = []*stats.Counters{statsOnlineInsertsCounters, statsOnlineUpdatesCounters, statsOnlineDeletesCounters} tableStatusList = scw.tableStatusListOnline case WorkerStateCloneOffline: statsCounters = []*stats.Counters{statsOfflineInsertsCounters, statsOfflineUpdatesCounters, statsOfflineDeletesCounters} tableStatusList = scw.tableStatusListOffline } sourceSchemaDefinition, err := scw.getSourceSchema(ctx, firstSourceTablet) if err != nil { return err } scw.wr.Logger().Infof("Source tablet 0 has %v tables to copy", len(sourceSchemaDefinition.TableDefinitions)) tableStatusList.initialize(sourceSchemaDefinition) // In parallel, setup the channels to send SQL data chunks to for each destination tablet: // // mu protects the context for cancelation, and firstError mu := sync.Mutex{} var firstError error ctx, cancelCopy := context.WithCancel(ctx) processError := func(format string, args ...interface{}) { scw.wr.Logger().Errorf(format, args...) mu.Lock() if firstError == nil { firstError = fmt.Errorf(format, args...) cancelCopy() } mu.Unlock() } insertChannels := make([]chan string, len(scw.destinationShards)) destinationThrottlers := make([]*throttler.Throttler, len(scw.destinationShards)) destinationWaitGroup := sync.WaitGroup{} for shardIndex, si := range scw.destinationShards { // we create one channel per destination tablet. It // is sized to have a buffer of a maximum of // destinationWriterCount * 2 items, to hopefully // always have data. We then have // destinationWriterCount go routines reading from it. insertChannels[shardIndex] = make(chan string, scw.destinationWriterCount*2) // Set up the throttler for each destination shard. keyspaceAndShard := topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName()) t, err := throttler.NewThrottler( keyspaceAndShard, "transactions", scw.destinationWriterCount, scw.maxTPS, throttler.ReplicationLagModuleDisabled) if err != nil { return fmt.Errorf("cannot instantiate throttler: %v", err) } destinationThrottlers[shardIndex] = t defer func(i int) { if t := destinationThrottlers[shardIndex]; t != nil { t.Close() } }(shardIndex) go func(keyspace, shard string, insertChannel chan string) { for j := 0; j < scw.destinationWriterCount; j++ { destinationWaitGroup.Add(1) go func(throttler *throttler.Throttler, threadID int) { defer destinationWaitGroup.Done() defer throttler.ThreadFinished(threadID) executor := newExecutor(scw.wr, scw.healthCheck, throttler, keyspace, shard, threadID) if err := executor.fetchLoop(ctx, insertChannel); err != nil { processError("executer.FetchLoop failed: %v", err) } }(t, j) } }(si.Keyspace(), si.ShardName(), insertChannels[shardIndex]) } // Now for each table, read data chunks and send them to all // insertChannels sourceWaitGroup := sync.WaitGroup{} sema := sync2.NewSemaphore(scw.sourceReaderCount, 0) for tableIndex, td := range sourceSchemaDefinition.TableDefinitions { if td.Type == tmutils.TableView { continue } td = reorderColumnsPrimaryKeyFirst(td) var keyResolver keyspaceIDResolver if *useV3ReshardingMode { keyResolver, err = newV3ResolverFromTableDefinition(scw.keyspaceSchema, td) if err != nil { return fmt.Errorf("cannot resolve v3 sharding keys for keyspace %v: %v", scw.keyspace, err) } } else { keyResolver, err = newV2Resolver(scw.keyspaceInfo, td) if err != nil { return fmt.Errorf("cannot resolve sharding keys for keyspace %v: %v", scw.keyspace, err) } } // TODO(mberlin): We're going to chunk *all* source shards based on the MIN // and MAX values of the *first* source shard. Is this going to be a problem? chunks, err := generateChunks(ctx, scw.wr, firstSourceTablet, td, scw.minTableSizeForSplit, scw.sourceReaderCount) if err != nil { return err } tableStatusList.setThreadCount(tableIndex, len(chunks)) for _, c := range chunks { sourceWaitGroup.Add(1) go func(td *tabletmanagerdatapb.TableDefinition, tableIndex int, chunk chunk) { defer sourceWaitGroup.Done() // We need our own error per Go routine to avoid races. var err error sema.Acquire() defer sema.Release() tableStatusList.threadStarted(tableIndex) if state == WorkerStateCloneOnline { // Wait for enough healthy tablets (they might have become unhealthy // and their replication lag might have increased since we started.) if err := scw.waitForTablets(ctx, scw.sourceShards, *retryDuration); err != nil { processError("table=%v chunk=%v: No healthy source tablets found (gave up after %v): ", td.Name, chunk, *retryDuration, err) return } } // Set up readers for the diff. There will be one reader for every // source and destination shard. sourceReaders := make([]ResultReader, len(scw.sourceShards)) destReaders := make([]ResultReader, len(scw.destinationShards)) for shardIndex, si := range scw.sourceShards { var sourceAlias *topodatapb.TabletAlias if state == WorkerStateCloneOffline { // Use the source tablet which we took offline for this phase. sourceAlias = scw.sourceAliases[shardIndex] } else { // Pick any healthy serving source tablet. tablets := discovery.RemoveUnhealthyTablets( scw.healthCheck.GetTabletStatsFromTarget(si.Keyspace(), si.ShardName(), topodatapb.TabletType_RDONLY)) if len(tablets) == 0 { processError("no healthy RDONLY tablets in source shard (%v) available", topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName())) return } sourceAlias = scw.tabletTracker.Track(tablets) defer scw.tabletTracker.Untrack(sourceAlias) } sourceResultReader, err := NewRestartableResultReader(ctx, scw.wr.Logger(), scw.wr.TopoServer(), sourceAlias, td, chunk) if err != nil { processError("NewRestartableResultReader for source tablet: %v failed: %v", sourceAlias, err) return } defer sourceResultReader.Close() sourceReaders[shardIndex] = sourceResultReader } // Wait for enough healthy tablets (they might have become unhealthy // and their replication lag might have increased due to a previous // chunk pipeline.) if err := scw.waitForTablets(ctx, scw.destinationShards, *retryDuration); err != nil { processError("table=%v chunk=%v: No healthy destination tablets found (gave up after %v): ", td.Name, chunk, *retryDuration, err) return } for shardIndex, si := range scw.destinationShards { // Pick any healthy serving destination tablet. tablets := discovery.RemoveUnhealthyTablets( scw.healthCheck.GetTabletStatsFromTarget(si.Keyspace(), si.ShardName(), topodatapb.TabletType_RDONLY)) if len(tablets) == 0 { processError("no healthy RDONLY tablets in destination shard (%v) available", topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName())) return } destAlias := scw.tabletTracker.Track(tablets) defer scw.tabletTracker.Untrack(destAlias) destResultReader, err := NewRestartableResultReader(ctx, scw.wr.Logger(), scw.wr.TopoServer(), destAlias, td, chunk) if err != nil { processError("NewQueryResultReaderForTablet for dest tablet: %v failed: %v", destAlias, err) return } defer destResultReader.Close() destReaders[shardIndex] = destResultReader } var sourceReader ResultReader var destReader ResultReader if len(sourceReaders) >= 2 { sourceReader, err = NewResultMerger(sourceReaders, len(td.PrimaryKeyColumns)) if err != nil { processError("NewResultMerger for source tablets failed: %v", err) return } } else { sourceReader = sourceReaders[0] } if len(destReaders) >= 2 { destReader, err = NewResultMerger(destReaders, len(td.PrimaryKeyColumns)) if err != nil { processError("NewResultMerger for dest tablets failed: %v", err) return } } else { destReader = destReaders[0] } dbNames := make([]string, len(scw.destinationShards)) for i, si := range scw.destinationShards { keyspaceAndShard := topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName()) dbNames[i] = scw.destinationDbNames[keyspaceAndShard] } // Compare the data and reconcile any differences. differ, err := NewRowDiffer2(ctx, sourceReader, destReader, td, tableStatusList, tableIndex, scw.destinationShards, keyResolver, insertChannels, ctx.Done(), dbNames, scw.writeQueryMaxRows, scw.writeQueryMaxSize, scw.writeQueryMaxRowsDelete, statsCounters) if err != nil { processError("NewRowDiffer2 failed: %v", err) return } // Ignore the diff report because all diffs should get reconciled. _ /* DiffReport */, err = differ.Diff() if err != nil { processError("RowDiffer2 failed: %v", err) return } tableStatusList.threadDone(tableIndex) }(td, tableIndex, c) } } sourceWaitGroup.Wait() for shardIndex := range scw.destinationShards { close(insertChannels[shardIndex]) } destinationWaitGroup.Wait() // Stop Throttlers. for i, t := range destinationThrottlers { t.Close() destinationThrottlers[i] = nil } if firstError != nil { return firstError } if state == WorkerStateCloneOffline { // Create and populate the blp_checkpoint table to give filtered replication // a starting point. if scw.strategy.skipPopulateBlpCheckpoint { scw.wr.Logger().Infof("Skipping populating the blp_checkpoint table") } else { queries := make([]string, 0, 4) queries = append(queries, binlogplayer.CreateBlpCheckpoint()...) flags := "" if scw.strategy.dontStartBinlogPlayer { flags = binlogplayer.BlpFlagDontStart } // get the current position from the sources for shardIndex := range scw.sourceShards { shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) status, err := scw.wr.TabletManagerClient().SlaveStatus(shortCtx, scw.sourceTablets[shardIndex]) cancel() if err != nil { return err } queries = append(queries, binlogplayer.PopulateBlpCheckpoint(uint32(shardIndex), status.Position, scw.maxTPS, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), flags)) } for _, si := range scw.destinationShards { destinationWaitGroup.Add(1) go func(keyspace, shard string) { defer destinationWaitGroup.Done() scw.wr.Logger().Infof("Making and populating blp_checkpoint table") keyspaceAndShard := topoproto.KeyspaceShardString(keyspace, shard) if err := runSQLCommands(ctx, scw.wr, scw.healthCheck, keyspace, shard, scw.destinationDbNames[keyspaceAndShard], queries); err != nil { processError("blp_checkpoint queries failed: %v", err) } }(si.Keyspace(), si.ShardName()) } destinationWaitGroup.Wait() if firstError != nil { return firstError } } // Configure filtered replication by setting the SourceShard info. // The master tablets won't enable filtered replication (the binlog player) // until they re-read the topology due to a restart or a reload. // TODO(alainjobart) this is a superset, some shards may not // overlap, have to deal with this better (for N -> M splits // where both N>1 and M>1) if scw.strategy.skipSetSourceShards { scw.wr.Logger().Infof("Skipping setting SourceShard on destination shards.") } else { for _, si := range scw.destinationShards { scw.wr.Logger().Infof("Setting SourceShard on shard %v/%v", si.Keyspace(), si.ShardName()) shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) err := scw.wr.SetSourceShards(shortCtx, si.Keyspace(), si.ShardName(), scw.sourceAliases, nil) cancel() if err != nil { return fmt.Errorf("failed to set source shards: %v", err) } } } // Force a state refresh (re-read topo) on all destination tablets. // The master tablet will end up starting filtered replication at this point. // // Find all tablets first, then refresh the state on each in parallel. err = scw.findRefreshTargets(ctx) if err != nil { return fmt.Errorf("failed before refreshing state on destination tablets: %v", err) } for shardIndex := range scw.destinationShards { for _, tabletAlias := range scw.refreshAliases[shardIndex] { destinationWaitGroup.Add(1) go func(ti *topo.TabletInfo) { defer destinationWaitGroup.Done() scw.wr.Logger().Infof("Refreshing state on tablet %v", ti.AliasString()) shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) err := scw.wr.TabletManagerClient().RefreshState(shortCtx, ti.Tablet) cancel() if err != nil { processError("RefreshState failed on tablet %v: %v", ti.AliasString(), err) } }(scw.refreshTablets[shardIndex][*tabletAlias]) } } } // clonePhase == offline destinationWaitGroup.Wait() return firstError }