func (scw *SplitCloneWorker) loadVSchema(ctx context.Context) error { var keyspaceSchema *vindexes.KeyspaceSchema if *useV3ReshardingMode { kschema, err := scw.wr.TopoServer().GetVSchema(ctx, scw.destinationKeyspace) if err != nil { return fmt.Errorf("cannot load VSchema for keyspace %v: %v", scw.destinationKeyspace, err) } if kschema == nil { return fmt.Errorf("no VSchema for keyspace %v", scw.destinationKeyspace) } keyspaceSchema, err = vindexes.BuildKeyspaceSchema(kschema, scw.destinationKeyspace) if err != nil { return fmt.Errorf("cannot build vschema for keyspace %v: %v", scw.destinationKeyspace, err) } scw.keyspaceSchema = keyspaceSchema } 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 *LegacySplitCloneWorker) copy(ctx context.Context) error { scw.setState(WorkerStateCloneOffline) start := time.Now() defer func() { statsStateDurationsNs.Set(string(WorkerStateCloneOffline), time.Now().Sub(start).Nanoseconds()) }() // get source schema from the first shard // TODO(alainjobart): for now, we assume the schema is compatible // on all source shards. Furthermore, we estimate the number of rows // in each source shard for each table to be about the same // (rowCount is used to estimate an ETA) shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) sourceSchemaDefinition, err := scw.wr.GetSchema(shortCtx, scw.sourceAliases[0], nil, scw.excludeTables, false /* includeViews */) cancel() if err != nil { return fmt.Errorf("cannot get schema from source %v: %v", topoproto.TabletAliasString(scw.sourceAliases[0]), err) } if len(sourceSchemaDefinition.TableDefinitions) == 0 { return fmt.Errorf("no tables matching the table filter in tablet %v", topoproto.TabletAliasString(scw.sourceAliases[0])) } for _, td := range sourceSchemaDefinition.TableDefinitions { if len(td.Columns) == 0 { return fmt.Errorf("schema for table %v has no columns", td.Name) } } scw.wr.Logger().Infof("Source tablet 0 has %v tables to copy", len(sourceSchemaDefinition.TableDefinitions)) scw.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)) 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) go func(keyspace, shard string, insertChannel chan string) { for j := 0; j < scw.destinationWriterCount; j++ { destinationWaitGroup.Add(1) go func(threadID int) { defer destinationWaitGroup.Done() keyspaceAndShard := topoproto.KeyspaceShardString(keyspace, shard) throttler := scw.destinationThrottlers[keyspaceAndShard] defer throttler.ThreadFinished(threadID) executor := newExecutor(scw.wr, scw.tsc, throttler, keyspace, shard, threadID) if err := executor.fetchLoop(ctx, insertChannel); err != nil { processError("executer.FetchLoop failed: %v", err) } }(j) } }(si.Keyspace(), si.ShardName(), insertChannels[shardIndex]) } // read the vschema if needed var keyspaceSchema *vindexes.KeyspaceSchema if *useV3ReshardingMode { kschema, err := scw.wr.TopoServer().GetVSchema(ctx, scw.keyspace) if err != nil { return fmt.Errorf("cannot load VSchema for keyspace %v: %v", scw.keyspace, err) } if kschema == nil { return fmt.Errorf("no VSchema for keyspace %v", scw.keyspace) } keyspaceSchema, err = vindexes.BuildKeyspaceSchema(kschema, scw.keyspace) if err != nil { return fmt.Errorf("cannot build vschema for keyspace %v: %v", scw.keyspace, err) } } // Now for each table, read data chunks and send them to all // insertChannels sourceWaitGroup := sync.WaitGroup{} for shardIndex := range scw.sourceShards { sema := sync2.NewSemaphore(scw.sourceReaderCount, 0) for tableIndex, td := range sourceSchemaDefinition.TableDefinitions { var keyResolver keyspaceIDResolver if *useV3ReshardingMode { keyResolver, err = newV3ResolverFromTableDefinition(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) } } rowSplitter := NewRowSplitter(scw.destinationShards, keyResolver) chunks, err := generateChunks(ctx, scw.wr, scw.sourceTablets[shardIndex], td, scw.sourceReaderCount, defaultMinRowsPerChunk) if err != nil { return err } scw.tableStatusList.setThreadCount(tableIndex, len(chunks)-1) for _, c := range chunks { sourceWaitGroup.Add(1) go func(td *tabletmanagerdatapb.TableDefinition, tableIndex int, chunk chunk) { defer sourceWaitGroup.Done() sema.Acquire() defer sema.Release() scw.tableStatusList.threadStarted(tableIndex) // Start streaming from the source tablets. tp := newSingleTabletProvider(ctx, scw.wr.TopoServer(), scw.sourceAliases[shardIndex]) rr, err := NewRestartableResultReader(ctx, scw.wr.Logger(), tp, td, chunk, false /* allowMultipleRetries */) if err != nil { processError("NewRestartableResultReader failed: %v", err) return } defer rr.Close(ctx) // process the data dbNames := make([]string, len(scw.destinationShards)) for i, si := range scw.destinationShards { keyspaceAndShard := topoproto.KeyspaceShardString(si.Keyspace(), si.ShardName()) dbNames[i] = scw.destinationDbNames[keyspaceAndShard] } if err := scw.processData(ctx, dbNames, td, tableIndex, rr, rowSplitter, insertChannels, scw.destinationPackCount); err != nil { processError("processData failed: %v", err) } scw.tableStatusList.threadDone(tableIndex) }(td, tableIndex, c) } } } sourceWaitGroup.Wait() for shardIndex := range scw.destinationShards { close(insertChannels[shardIndex]) } destinationWaitGroup.Wait() if firstError != nil { return firstError } // then create and populate the blp_checkpoint table 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.tsc, 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 } } // Now we're done with data copy, update the shard's source info. // 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) } } } err = scw.findRefreshTargets(ctx) if err != nil { return fmt.Errorf("failed before refreshing state on destination tablets: %v", err) } // And force a state refresh (re-read topo) on all destination tablets. // The master tablet will end up starting filtered replication // at this point. 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]) } } destinationWaitGroup.Wait() return firstError }
// init phase: // - read the destination keyspace, make sure it has 'servedFrom' values func (scw *SplitCloneWorker) init(ctx context.Context) error { scw.setState(WorkerStateInit) var err error // read the keyspace and validate it shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) scw.keyspaceInfo, err = scw.wr.TopoServer().GetKeyspace(shortCtx, scw.keyspace) cancel() if err != nil { return fmt.Errorf("cannot read keyspace %v: %v", scw.keyspace, err) } // find the OverlappingShards in the keyspace shortCtx, cancel = context.WithTimeout(ctx, *remoteActionsTimeout) osList, err := topotools.FindOverlappingShards(shortCtx, scw.wr.TopoServer(), scw.keyspace) cancel() if err != nil { return fmt.Errorf("cannot FindOverlappingShards in %v: %v", scw.keyspace, err) } // find the shard we mentioned in there, if any os := topotools.OverlappingShardsForShard(osList, scw.shard) if os == nil { return fmt.Errorf("the specified shard %v/%v is not in any overlapping shard", scw.keyspace, scw.shard) } scw.wr.Logger().Infof("Found overlapping shards: %+v\n", os) // one side should have served types, the other one none, // figure out wich is which, then double check them all if len(os.Left[0].ServedTypes) > 0 { scw.sourceShards = os.Left scw.destinationShards = os.Right } else { scw.sourceShards = os.Right scw.destinationShards = os.Left } // Verify that filtered replication is not already enabled. for _, si := range scw.destinationShards { if len(si.SourceShards) > 0 { return fmt.Errorf("destination shard %v/%v has filtered replication already enabled from a previous resharding (ShardInfo is set)."+ " This requires manual intervention e.g. use vtctl SourceShardDelete to remove it", si.Keyspace(), si.ShardName()) } } // validate all serving types servingTypes := []topodatapb.TabletType{topodatapb.TabletType_MASTER, topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY} for _, st := range servingTypes { for _, si := range scw.sourceShards { if si.GetServedType(st) == nil { return fmt.Errorf("source shard %v/%v is not serving type %v", si.Keyspace(), si.ShardName(), st) } } } for _, si := range scw.destinationShards { if len(si.ServedTypes) > 0 { return fmt.Errorf("destination shard %v/%v is serving some types", si.Keyspace(), si.ShardName()) } } // read the vschema if needed var keyspaceSchema *vindexes.KeyspaceSchema if *useV3ReshardingMode { kschema, err := scw.wr.TopoServer().GetVSchema(ctx, scw.keyspace) if err != nil { return fmt.Errorf("cannot load VSchema for keyspace %v: %v", scw.keyspace, err) } if kschema == nil { return fmt.Errorf("no VSchema for keyspace %v", scw.keyspace) } keyspaceSchema, err = vindexes.BuildKeyspaceSchema(kschema, scw.keyspace) if err != nil { return fmt.Errorf("cannot build vschema for keyspace %v: %v", scw.keyspace, err) } scw.keyspaceSchema = keyspaceSchema } // Initialize healthcheck and add destination shards to it. scw.healthCheck = discovery.NewHealthCheck(*remoteActionsTimeout, *healthcheckRetryDelay, *healthCheckTimeout) allShards := append(scw.sourceShards, scw.destinationShards...) for _, si := range allShards { watcher := discovery.NewShardReplicationWatcher(scw.wr.TopoServer(), scw.healthCheck, scw.cell, si.Keyspace(), si.ShardName(), *healthCheckTopologyRefresh, discovery.DefaultTopoReadConcurrency) scw.shardWatchers = append(scw.shardWatchers, watcher) } return nil }
func (sdw *SplitDiffWorker) diff(ctx context.Context) error { sdw.SetState(WorkerStateDiff) sdw.wr.Logger().Infof("Gathering schema information...") wg := sync.WaitGroup{} rec := &concurrency.AllErrorRecorder{} wg.Add(1) go func() { var err error shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) sdw.destinationSchemaDefinition, err = sdw.wr.GetSchema( shortCtx, sdw.destinationAlias, nil /* tables */, sdw.excludeTables, false /* includeViews */) cancel() rec.RecordError(err) sdw.wr.Logger().Infof("Got schema from destination %v", sdw.destinationAlias) wg.Done() }() wg.Add(1) go func() { var err error shortCtx, cancel := context.WithTimeout(ctx, *remoteActionsTimeout) sdw.sourceSchemaDefinition, err = sdw.wr.GetSchema( shortCtx, sdw.sourceAlias, nil /* tables */, sdw.excludeTables, false /* includeViews */) cancel() rec.RecordError(err) sdw.wr.Logger().Infof("Got schema from source %v", sdw.sourceAlias) wg.Done() }() wg.Wait() if rec.HasErrors() { return rec.Error() } sdw.wr.Logger().Infof("Diffing the schema...") rec = &concurrency.AllErrorRecorder{} tmutils.DiffSchema("destination", sdw.destinationSchemaDefinition, "source", sdw.sourceSchemaDefinition, rec) if rec.HasErrors() { sdw.wr.Logger().Warningf("Different schemas: %v", rec.Error().Error()) } else { sdw.wr.Logger().Infof("Schema match, good.") } // read the vschema if needed var keyspaceSchema *vindexes.KeyspaceSchema if *useV3ReshardingMode { kschema, err := sdw.wr.TopoServer().GetVSchema(ctx, sdw.keyspace) if err != nil { return fmt.Errorf("cannot load VSchema for keyspace %v: %v", sdw.keyspace, err) } if kschema == nil { return fmt.Errorf("no VSchema for keyspace %v", sdw.keyspace) } keyspaceSchema, err = vindexes.BuildKeyspaceSchema(kschema, sdw.keyspace) if err != nil { return fmt.Errorf("cannot build vschema for keyspace %v: %v", sdw.keyspace, err) } } // Compute the overlap keyrange. Later, we'll compare it with // source or destination keyrange. If it matches either, // we'll just ask for all the data. If the overlap is a subset, // we'll filter. overlap, err := key.KeyRangesOverlap(sdw.shardInfo.KeyRange, sdw.shardInfo.SourceShards[sdw.sourceUID].KeyRange) if err != nil { return fmt.Errorf("Source shard doesn't overlap with destination: %v", err) } // run the diffs, 8 at a time sdw.wr.Logger().Infof("Running the diffs...") // TODO(mberlin): Parameterize the hard coded value 8. sem := sync2.NewSemaphore(8, 0) for _, tableDefinition := range sdw.destinationSchemaDefinition.TableDefinitions { wg.Add(1) go func(tableDefinition *tabletmanagerdatapb.TableDefinition) { defer wg.Done() sem.Acquire() defer sem.Release() sdw.wr.Logger().Infof("Starting the diff on table %v", tableDefinition.Name) // On the source, see if we need a full scan // or a filtered scan. var sourceQueryResultReader *QueryResultReader if key.KeyRangeEqual(overlap, sdw.shardInfo.SourceShards[sdw.sourceUID].KeyRange) { sourceQueryResultReader, err = TableScan(ctx, sdw.wr.Logger(), sdw.wr.TopoServer(), sdw.sourceAlias, tableDefinition) } else { sourceQueryResultReader, err = TableScanByKeyRange(ctx, sdw.wr.Logger(), sdw.wr.TopoServer(), sdw.sourceAlias, tableDefinition, overlap, keyspaceSchema, sdw.keyspaceInfo.ShardingColumnName, sdw.keyspaceInfo.ShardingColumnType) } if err != nil { newErr := fmt.Errorf("TableScan(ByKeyRange?)(source) failed: %v", err) rec.RecordError(newErr) sdw.wr.Logger().Errorf("%v", newErr) return } defer sourceQueryResultReader.Close(ctx) // On the destination, see if we need a full scan // or a filtered scan. var destinationQueryResultReader *QueryResultReader if key.KeyRangeEqual(overlap, sdw.shardInfo.KeyRange) { destinationQueryResultReader, err = TableScan(ctx, sdw.wr.Logger(), sdw.wr.TopoServer(), sdw.destinationAlias, tableDefinition) } else { destinationQueryResultReader, err = TableScanByKeyRange(ctx, sdw.wr.Logger(), sdw.wr.TopoServer(), sdw.destinationAlias, tableDefinition, overlap, keyspaceSchema, sdw.keyspaceInfo.ShardingColumnName, sdw.keyspaceInfo.ShardingColumnType) } if err != nil { newErr := fmt.Errorf("TableScan(ByKeyRange?)(destination) failed: %v", err) rec.RecordError(newErr) sdw.wr.Logger().Errorf("%v", newErr) return } defer destinationQueryResultReader.Close(ctx) // Create the row differ. differ, err := NewRowDiffer(sourceQueryResultReader, destinationQueryResultReader, tableDefinition) if err != nil { newErr := fmt.Errorf("NewRowDiffer() failed: %v", err) rec.RecordError(newErr) sdw.wr.Logger().Errorf("%v", newErr) return } // And run the diff. report, err := differ.Go(sdw.wr.Logger()) if err != nil { newErr := fmt.Errorf("Differ.Go failed: %v", err.Error()) rec.RecordError(newErr) sdw.wr.Logger().Errorf("%v", newErr) } else { if report.HasDifferences() { err := fmt.Errorf("Table %v has differences: %v", tableDefinition.Name, report.String()) rec.RecordError(err) sdw.wr.Logger().Warningf(err.Error()) } else { sdw.wr.Logger().Infof("Table %v checks out (%v rows processed, %v qps)", tableDefinition.Name, report.processedRows, report.processingQPS) } } }(tableDefinition) } wg.Wait() return rec.Error() }