From b364b0cdddbbb122151cf7b1cd0881cb74f57500 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Mon, 28 Oct 2024 23:38:10 +0530 Subject: [PATCH 01/12] refac: Refactor Server.GetWorkflows() Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/server.go | 544 +---------------------- go/vt/vtctl/workflow/server_test.go | 60 +++ go/vt/vtctl/workflow/workflows.go | 642 ++++++++++++++++++++++++++++ 3 files changed, 713 insertions(+), 533 deletions(-) create mode 100644 go/vt/vtctl/workflow/workflows.go diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 1828b0af814..afd01e12814 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -18,7 +18,6 @@ package workflow import ( "context" - "encoding/json" "errors" "fmt" "math" @@ -41,11 +40,9 @@ import ( "vitess.io/vitess/go/constants/sidecar" "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/ptr" - "vitess.io/vitess/go/sets" "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/trace" - "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/key" @@ -58,7 +55,6 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vtctl/schematools" - "vitess.io/vitess/go/vt/vtctl/workflow/common" "vitess.io/vitess/go/vt/vtctl/workflow/vexec" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vterrors" @@ -406,546 +402,28 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows span.Annotate("include_logs", req.IncludeLogs) span.Annotate("shards", req.Shards) - readReq := &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{} - if req.Workflow != "" { - readReq.IncludeWorkflows = []string{req.Workflow} + wf := &workflowFetcher{ + ts: s.ts, + tmc: s.tmc, + parser: s.SQLParser(), + logger: s.Logger(), } - if req.ActiveOnly { - readReq.ExcludeStates = []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped} - } - - // Guards access to the maps used throughout. - m := sync.Mutex{} - shards, err := common.GetShards(ctx, s.ts, req.Keyspace, req.Shards) + workflowsByShard, err := wf.fetchWorkflowsByShard(ctx, req) if err != nil { return nil, err } - results := make(map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, len(shards)) - readWorkflowsEg, readWorkflowsCtx := errgroup.WithContext(ctx) - for _, shard := range shards { - readWorkflowsEg.Go(func() error { - si, err := s.ts.GetShard(readWorkflowsCtx, req.Keyspace, shard) - if err != nil { - return err - } - if si.PrimaryAlias == nil { - return fmt.Errorf("%w %s/%s", vexec.ErrNoShardPrimary, req.Keyspace, shard) - } - primary, err := s.ts.GetTablet(readWorkflowsCtx, si.PrimaryAlias) - if err != nil { - return err - } - if primary == nil { - return fmt.Errorf("%w %s/%s: tablet %v not found", vexec.ErrNoShardPrimary, req.Keyspace, shard, topoproto.TabletAliasString(si.PrimaryAlias)) - } - // Clone the request so that we can set the correct DB name for tablet. - req := readReq.CloneVT() - wres, err := s.tmc.ReadVReplicationWorkflows(readWorkflowsCtx, primary.Tablet, req) - if err != nil { - return err - } - m.Lock() - defer m.Unlock() - results[primary] = wres - return nil - }) - } - if readWorkflowsEg.Wait() != nil { - return nil, err - } - - copyStatesByShardStreamId := make(map[string][]*vtctldatapb.Workflow_Stream_CopyState, len(results)) - - fetchCopyStates := func(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) error { - span, ctx := trace.NewSpan(ctx, "workflow.Server.fetchCopyStates") - defer span.Finish() - span.Annotate("keyspace", req.Keyspace) - span.Annotate("shard", tablet.Shard) - span.Annotate("tablet_alias", tablet.AliasString()) - - copyStates, err := s.getWorkflowCopyStates(ctx, tablet, streamIds) - if err != nil { - return err - } - - m.Lock() - defer m.Unlock() - - for _, copyState := range copyStates { - shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, copyState.StreamId) - copyStatesByShardStreamId[shardStreamId] = append( - copyStatesByShardStreamId[shardStreamId], - copyState, - ) - } - - return nil - } - - fetchCopyStatesEg, fetchCopyStatesCtx := errgroup.WithContext(ctx) - for tablet, result := range results { - tablet := tablet // loop closure - - streamIds := make([]int32, 0, len(result.Workflows)) - for _, wf := range result.Workflows { - for _, stream := range wf.Streams { - streamIds = append(streamIds, stream.Id) - } - } - - if len(streamIds) == 0 { - continue - } - - fetchCopyStatesEg.Go(func() error { - return fetchCopyStates(fetchCopyStatesCtx, tablet, streamIds) - }) - } - - if err := fetchCopyStatesEg.Wait(); err != nil { + copyStatesByShardStreamId, err := wf.fetchCopyStatesByShardStream(ctx, workflowsByShard) + if err != nil { return nil, err } - workflowsMap := make(map[string]*vtctldatapb.Workflow, len(results)) - sourceKeyspaceByWorkflow := make(map[string]string, len(results)) - sourceShardsByWorkflow := make(map[string]sets.Set[string], len(results)) - targetKeyspaceByWorkflow := make(map[string]string, len(results)) - targetShardsByWorkflow := make(map[string]sets.Set[string], len(results)) - maxVReplicationLagByWorkflow := make(map[string]float64, len(results)) - maxVReplicationTransactionLagByWorkflow := make(map[string]float64, len(results)) - - // We guarantee the following invariants when this function is called for a - // given workflow: - // - workflow.Name != "" (more precisely, ".Name is set 'properly'") - // - workflowsMap[workflow.Name] == workflow - // - sourceShardsByWorkflow[workflow.Name] != nil - // - targetShardsByWorkflow[workflow.Name] != nil - // - workflow.ShardStatuses != nil - scanWorkflow := func(ctx context.Context, workflow *vtctldatapb.Workflow, res *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, tablet *topo.TabletInfo) error { - // This is not called concurrently, but we still protect the maps to ensure - // that we're concurrency-safe in the face of future changes (e.g. where other - // things are running concurrently with this which also access these maps). - m.Lock() - defer m.Unlock() - for _, rstream := range res.Streams { - // The value in the pos column can be compressed and thus not - // have a valid GTID consisting of valid UTF-8 characters so we - // have to decode it so that it's properly decompressed first - // when needed. - pos := rstream.Pos - if pos != "" { - mpos, err := binlogplayer.DecodePosition(pos) - if err != nil { - return err - } - pos = mpos.String() - } - - cells := strings.Split(res.Cells, ",") - for i := range cells { - cells[i] = strings.TrimSpace(cells[i]) - } - options := res.Options - if options != "" { - if err := json.Unmarshal([]byte(options), &workflow.Options); err != nil { - return err - } - } - stream := &vtctldatapb.Workflow_Stream{ - Id: int64(rstream.Id), - Shard: tablet.Shard, - Tablet: tablet.Alias, - BinlogSource: rstream.Bls, - Position: pos, - StopPosition: rstream.StopPos, - State: rstream.State.String(), - DbName: tablet.DbName(), - TabletTypes: res.TabletTypes, - TabletSelectionPreference: res.TabletSelectionPreference, - Cells: cells, - TransactionTimestamp: rstream.TransactionTimestamp, - TimeUpdated: rstream.TimeUpdated, - Message: rstream.Message, - Tags: strings.Split(res.Tags, ","), - RowsCopied: rstream.RowsCopied, - ThrottlerStatus: &vtctldatapb.Workflow_Stream_ThrottlerStatus{ - ComponentThrottled: rstream.ComponentThrottled, - TimeThrottled: rstream.TimeThrottled, - }, - } - - // Merge in copy states, which we've already fetched. - shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, stream.Id) - if copyState, ok := copyStatesByShardStreamId[shardStreamId]; ok { - stream.CopyStates = copyState - } - - if rstream.TimeUpdated == nil { - rstream.TimeUpdated = &vttimepb.Time{} - } - - switch { - case strings.Contains(strings.ToLower(stream.Message), "error"): - stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: - stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-rstream.TimeUpdated.Seconds > 10: - stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() - } - - shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) - shardStream, ok := workflow.ShardStreams[shardStreamKey] - if !ok { - ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) - defer cancel() - - si, err := s.ts.GetShard(ctx, req.Keyspace, tablet.Shard) - if err != nil { - return err - } - - shardStream = &vtctldatapb.Workflow_ShardStream{ - Streams: nil, - TabletControls: si.TabletControls, - IsPrimaryServing: si.IsPrimaryServing, - } - - workflow.ShardStreams[shardStreamKey] = shardStream - } - - shardStream.Streams = append(shardStream.Streams, stream) - sourceShardsByWorkflow[workflow.Name].Insert(stream.BinlogSource.Shard) - targetShardsByWorkflow[workflow.Name].Insert(tablet.Shard) - - if ks, ok := sourceKeyspaceByWorkflow[workflow.Name]; ok && ks != stream.BinlogSource.Keyspace { - return vterrors.Wrapf(ErrMultipleSourceKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, stream.BinlogSource.Keyspace) - } - - sourceKeyspaceByWorkflow[workflow.Name] = stream.BinlogSource.Keyspace - - if ks, ok := targetKeyspaceByWorkflow[workflow.Name]; ok && ks != tablet.Keyspace { - return vterrors.Wrapf(ErrMultipleTargetKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, tablet.Keyspace) - } - - targetKeyspaceByWorkflow[workflow.Name] = tablet.Keyspace - - if stream.TimeUpdated == nil { - stream.TimeUpdated = &vttimepb.Time{} - } - timeUpdated := time.Unix(stream.TimeUpdated.Seconds, 0) - vreplicationLag := time.Since(timeUpdated) - - // MaxVReplicationLag represents the time since we last processed any event - // in the workflow. - if currentMaxLag, ok := maxVReplicationLagByWorkflow[workflow.Name]; ok { - if vreplicationLag.Seconds() > currentMaxLag { - maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() - } - } else { - maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() - } - - workflow.WorkflowType = res.WorkflowType.String() - workflow.WorkflowSubType = res.WorkflowSubType.String() - workflow.DeferSecondaryKeys = res.DeferSecondaryKeys - - // MaxVReplicationTransactionLag estimates the actual statement processing lag - // between the source and the target. If we are still processing source events it - // is the difference b/w current time and the timestamp of the last event. If - // heartbeats are more recent than the last event, then the lag is the time since - // the last heartbeat as there can be an actual event immediately after the - // heartbeat, but which has not yet been processed on the target. - // We don't allow switching during the copy phase, so in that case we just return - // a large lag. All timestamps are in seconds since epoch. - if _, ok := maxVReplicationTransactionLagByWorkflow[workflow.Name]; !ok { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = 0 - } - if rstream.TransactionTimestamp == nil { - rstream.TransactionTimestamp = &vttimepb.Time{} - } - lastTransactionTime := rstream.TransactionTimestamp.Seconds - if rstream.TimeHeartbeat == nil { - rstream.TimeHeartbeat = &vttimepb.Time{} - } - lastHeartbeatTime := rstream.TimeHeartbeat.Seconds - if stream.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = math.MaxInt64 - } else { - if lastTransactionTime == 0 /* no new events after copy */ || - lastHeartbeatTime > lastTransactionTime /* no recent transactions, so all caught up */ { - - lastTransactionTime = lastHeartbeatTime - } - now := time.Now().Unix() /* seconds since epoch */ - transactionReplicationLag := float64(now - lastTransactionTime) - if transactionReplicationLag > maxVReplicationTransactionLagByWorkflow[workflow.Name] { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = transactionReplicationLag - } - } - } - - return nil - } - - for tablet, result := range results { - // In the old implementation, we knew we had at most one (0 <= N <= 1) - // workflow for each shard primary we queried. There might be multiple - // rows (streams) comprising that workflow, so we would aggregate the - // rows for a given primary into a single value ("the workflow", - // ReplicationStatusResult in the old types). - // - // In this version, we have many (N >= 0) workflows for each shard - // primary we queried, so we need to determine if each row corresponds - // to a workflow we're already aggregating, or if it's a workflow we - // haven't seen yet for that shard primary. We use the workflow name to - // dedupe for this. - for _, wfres := range result.Workflows { - workflowName := wfres.Workflow - workflow, ok := workflowsMap[workflowName] - if !ok { - workflow = &vtctldatapb.Workflow{ - Name: workflowName, - ShardStreams: map[string]*vtctldatapb.Workflow_ShardStream{}, - } - - workflowsMap[workflowName] = workflow - sourceShardsByWorkflow[workflowName] = sets.New[string]() - targetShardsByWorkflow[workflowName] = sets.New[string]() - } - - if err := scanWorkflow(ctx, workflow, wfres, tablet); err != nil { - return nil, err - } - } - } - - var ( - fetchLogsWG sync.WaitGroup - vrepLogQuery = strings.TrimSpace(` -SELECT - id, - vrepl_id, - type, - state, - message, - created_at, - updated_at, - count -FROM - _vt.vreplication_log -WHERE vrepl_id IN %a -ORDER BY - vrepl_id ASC, - id ASC -`) - ) - - fetchStreamLogs := func(ctx context.Context, workflow *vtctldatapb.Workflow) { - span, ctx := trace.NewSpan(ctx, "workflow.Server.fetchStreamLogs") - defer span.Finish() - - span.Annotate("keyspace", req.Keyspace) - span.Annotate("workflow", workflow.Name) - - vreplIDs := make([]int64, 0, len(workflow.ShardStreams)) - for _, shardStream := range maps.Values(workflow.ShardStreams) { - for _, stream := range shardStream.Streams { - vreplIDs = append(vreplIDs, stream.Id) - } - } - idsBV, err := sqltypes.BuildBindVariable(vreplIDs) - if err != nil { - return - } - - query, err := sqlparser.ParseAndBind(vrepLogQuery, idsBV) - if err != nil { - return - } - - vx := vexec.NewVExec(req.Keyspace, workflow.Name, s.ts, s.tmc, s.SQLParser()) - results, err := vx.QueryContext(ctx, query) - if err != nil { - // Note that we do not return here. If there are any query results - // in the map (i.e. some tablets returned successfully), we will - // still try to read log rows from them on a best-effort basis. But, - // we will also pre-emptively record the top-level fetch error on - // every stream in every shard in the workflow. Further processing - // below may override the error message for certain streams. - for _, streams := range workflow.ShardStreams { - for _, stream := range streams.Streams { - stream.LogFetchError = err.Error() - } - } - } - - for target, p3qr := range results { - qr := sqltypes.Proto3ToResult(p3qr) - shardStreamKey := fmt.Sprintf("%s/%s", target.Shard, target.AliasString()) - - ss, ok := workflow.ShardStreams[shardStreamKey] - if !ok || ss == nil { - continue - } - - streams := ss.Streams - streamIdx := 0 - markErrors := func(err error) { - if streamIdx >= len(streams) { - return - } - - streams[streamIdx].LogFetchError = err.Error() - } - - for _, row := range qr.Rows { - id, err := row[0].ToCastInt64() - if err != nil { - markErrors(err) - continue - } - - streamID, err := row[1].ToCastInt64() - if err != nil { - markErrors(err) - continue - } - - typ := row[2].ToString() - state := row[3].ToString() - message := row[4].ToString() - - createdAt, err := time.Parse("2006-01-02 15:04:05", row[5].ToString()) - if err != nil { - markErrors(err) - continue - } - - updatedAt, err := time.Parse("2006-01-02 15:04:05", row[6].ToString()) - if err != nil { - markErrors(err) - continue - } - - count, err := row[7].ToCastInt64() - if err != nil { - markErrors(err) - continue - } - - streamLog := &vtctldatapb.Workflow_Stream_Log{ - Id: id, - StreamId: streamID, - Type: typ, - State: state, - CreatedAt: &vttimepb.Time{ - Seconds: createdAt.Unix(), - }, - UpdatedAt: &vttimepb.Time{ - Seconds: updatedAt.Unix(), - }, - Message: message, - Count: count, - } - - // Earlier, in the main loop where we called scanWorkflow for - // each _vt.vreplication row, we also sorted each ShardStreams - // slice by ascending id, and our _vt.vreplication_log query - // ordered by (stream_id ASC, id ASC), so we can walk the - // streams in index order in O(n) amortized over all the rows - // for this tablet. - for streamIdx < len(streams) { - stream := streams[streamIdx] - if stream.Id < streamLog.StreamId { - streamIdx++ - continue - } - - if stream.Id > streamLog.StreamId { - s.Logger().Warningf("Found stream log for nonexistent stream: %+v", streamLog) - // This can happen on manual/failed workflow cleanup so move to the next log. - break - } - - // stream.Id == streamLog.StreamId - stream.Logs = append(stream.Logs, streamLog) - break - } - } - } - } - - workflows := make([]*vtctldatapb.Workflow, 0, len(workflowsMap)) - - for name, workflow := range workflowsMap { - sourceShards, ok := sourceShardsByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no source shards", name) - } - - sourceKeyspace, ok := sourceKeyspaceByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no source keyspace", name) - } - - targetShards, ok := targetShardsByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no target shards", name) - } - - targetKeyspace, ok := targetKeyspaceByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no target keyspace", name) - } - - maxVReplicationLag, ok := maxVReplicationLagByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no tracked vreplication lag", name) - } - - maxVReplicationTransactionLag, ok := maxVReplicationTransactionLagByWorkflow[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no tracked vreplication transaction lag", name) - } - - workflow.Source = &vtctldatapb.Workflow_ReplicationLocation{ - Keyspace: sourceKeyspace, - Shards: sets.List(sourceShards), - } - - workflow.Target = &vtctldatapb.Workflow_ReplicationLocation{ - Keyspace: targetKeyspace, - Shards: sets.List(targetShards), - } - - workflow.MaxVReplicationLag = int64(maxVReplicationLag) - workflow.MaxVReplicationTransactionLag = int64(maxVReplicationTransactionLag) - - // Sort shard streams by stream_id ASC, to support an optimization - // in fetchStreamLogs below. - for _, shardStreams := range workflow.ShardStreams { - sort.Slice(shardStreams.Streams, func(i, j int) bool { - return shardStreams.Streams[i].Id < shardStreams.Streams[j].Id - }) - } - - workflows = append(workflows, workflow) - - if req.IncludeLogs { - // Fetch logs for all streams associated with this workflow in the background. - fetchLogsWG.Add(1) - go func(ctx context.Context, workflow *vtctldatapb.Workflow) { - defer fetchLogsWG.Done() - fetchStreamLogs(ctx, workflow) - }(ctx, workflow) - } + workflows, err := wf.buildWorkflows(ctx, workflowsByShard, copyStatesByShardStreamId, req) + if err != nil { + return nil, err } - // Wait for all the log fetchers to finish. - fetchLogsWG.Wait() - return &vtctldatapb.GetWorkflowsResponse{ Workflows: workflows, }, nil diff --git a/go/vt/vtctl/workflow/server_test.go b/go/vt/vtctl/workflow/server_test.go index b7783fc2945..870a4ae96c3 100644 --- a/go/vt/vtctl/workflow/server_test.go +++ b/go/vt/vtctl/workflow/server_test.go @@ -2256,3 +2256,63 @@ func TestGetWorkflowsStreamLogs(t *testing.T) { assert.Equal(t, gotLogs[0].State, "Running") assert.Equal(t, gotLogs[0].Id, int64(3)) } + +func TestWorkflowStatus(t *testing.T) { + ctx := context.Background() + + sourceKeyspace := "source_keyspace" + targetKeyspace := "target_keyspace" + workflow := "test_workflow" + + sourceShards := []string{"-"} + targetShards := []string{"-"} + + te := newTestMaterializerEnv(t, ctx, &vtctldatapb.MaterializeSettings{ + SourceKeyspace: sourceKeyspace, + TargetKeyspace: targetKeyspace, + Workflow: workflow, + TableSettings: []*vtctldatapb.TableMaterializeSettings{ + { + TargetTable: "table1", + SourceExpression: fmt.Sprintf("select * from %s", "table1"), + }, + { + TargetTable: "table2", + SourceExpression: fmt.Sprintf("select * from %s", "table2"), + }, + }, + }, sourceShards, targetShards) + + tablesResult := sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name", "varchar"), "table1", "table2") + te.tmc.expectVRQuery(200, "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1", tablesResult) + + tablesTargetCopyResult := sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|table_rows|data_length", "varchar|int64|int64"), "table1|50|500", "table2|100|250") + te.tmc.expectVRQuery(200, "select table_name, table_rows, data_length from information_schema.tables where table_schema = 'vt_target_keyspace' and table_name in ('table1','table2')", tablesTargetCopyResult) + + tablesSourceCopyResult := sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|table_rows|data_length", "varchar|int64|int64"), "table1|100|1000", "table2|200|500") + te.tmc.expectVRQuery(100, "select table_name, table_rows, data_length from information_schema.tables where table_schema = 'vt_source_keyspace' and table_name in ('table1','table2')", tablesSourceCopyResult) + + te.tmc.expectVRQuery(200, "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)", &sqltypes.Result{}) + + res, err := te.ws.WorkflowStatus(ctx, &vtctldatapb.WorkflowStatusRequest{ + Keyspace: targetKeyspace, + Workflow: workflow, + Shards: targetShards, + }) + + assert.NoError(t, err) + + require.NotNil(t, res.TableCopyState) + + stateTable1 := res.TableCopyState["table1"] + stateTable2 := res.TableCopyState["table2"] + require.NotNil(t, stateTable1) + require.NotNil(t, stateTable2) + + assert.Equal(t, int64(100), stateTable1.RowsTotal) + assert.Equal(t, int64(200), stateTable2.RowsTotal) + assert.Equal(t, int64(50), stateTable1.RowsCopied) + assert.Equal(t, int64(100), stateTable2.RowsCopied) + assert.Equal(t, float32(50), stateTable1.RowsPercentage) + assert.Equal(t, float32(50), stateTable2.RowsPercentage) +} diff --git a/go/vt/vtctl/workflow/workflows.go b/go/vt/vtctl/workflow/workflows.go new file mode 100644 index 00000000000..8d757db33f1 --- /dev/null +++ b/go/vt/vtctl/workflow/workflows.go @@ -0,0 +1,642 @@ +/* +Copyright 2024 The Vitess Authors. + +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 workflow + +import ( + "context" + "encoding/json" + "fmt" + "math" + "sort" + "strings" + "sync" + "time" + + "golang.org/x/exp/maps" + "golang.org/x/sync/errgroup" + + "vitess.io/vitess/go/sets" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/trace" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" + "vitess.io/vitess/go/vt/vtctl/workflow/common" + "vitess.io/vitess/go/vt/vtctl/workflow/vexec" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vttablet/tmclient" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" + vttimepb "vitess.io/vitess/go/vt/proto/vttime" +) + +// workflowFetcher is responsible for fetching and retrieving information +// about VReplication workflows. +type workflowFetcher struct { + ts *topo.Server + tmc tmclient.TabletManagerClient + + logger logutil.Logger + parser *sqlparser.Parser +} + +type workflowMetadata struct { + sourceKeyspace string + sourceShards sets.Set[string] + targetKeyspace string + targetShards sets.Set[string] + maxVReplicationLag float64 + maxVReplicationTransactionLag float64 +} + +var vrepLogQuery = strings.TrimSpace(` +SELECT + id, + vrepl_id, + type, + state, + message, + created_at, + updated_at, + count +FROM + _vt.vreplication_log +WHERE vrepl_id IN %a +ORDER BY + vrepl_id ASC, + id ASC +`) + +func (wf *workflowFetcher) fetchWorkflowsByShard( + ctx context.Context, + req *vtctldatapb.GetWorkflowsRequest, +) (map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + readReq := &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{} + if req.Workflow != "" { + readReq.IncludeWorkflows = []string{req.Workflow} + } + if req.ActiveOnly { + readReq.ExcludeStates = []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped} + } + + m := sync.Mutex{} + + shards, err := common.GetShards(ctx, wf.ts, req.Keyspace, req.Shards) + if err != nil { + return nil, err + } + + results := make(map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, len(shards)) + + readWorkflowsEg, readWorkflowsCtx := errgroup.WithContext(ctx) + for _, shard := range shards { + readWorkflowsEg.Go(func() error { + si, err := wf.ts.GetShard(readWorkflowsCtx, req.Keyspace, shard) + if err != nil { + return err + } + if si.PrimaryAlias == nil { + return fmt.Errorf("%w %s/%s", vexec.ErrNoShardPrimary, req.Keyspace, shard) + } + primary, err := wf.ts.GetTablet(readWorkflowsCtx, si.PrimaryAlias) + if err != nil { + return err + } + if primary == nil { + return fmt.Errorf("%w %s/%s: tablet %v not found", vexec.ErrNoShardPrimary, req.Keyspace, shard, topoproto.TabletAliasString(si.PrimaryAlias)) + } + // Clone the request so that we can set the correct DB name for tablet. + req := readReq.CloneVT() + wres, err := wf.tmc.ReadVReplicationWorkflows(readWorkflowsCtx, primary.Tablet, req) + if err != nil { + return err + } + m.Lock() + defer m.Unlock() + results[primary] = wres + return nil + }) + } + if readWorkflowsEg.Wait() != nil { + return nil, err + } + + return results, nil +} + +func (wf *workflowFetcher) fetchCopyStatesByShardStream( + ctx context.Context, + workflowsByShard map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, +) (map[string][]*vtctldatapb.Workflow_Stream_CopyState, error) { + m := sync.Mutex{} + + copyStatesByShardStreamId := make(map[string][]*vtctldatapb.Workflow_Stream_CopyState, len(workflowsByShard)) + + fetchCopyStates := func(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) error { + span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.fetchCopyStates") + defer span.Finish() + + span.Annotate("shard", tablet.Shard) + span.Annotate("tablet_alias", tablet.AliasString()) + + copyStates, err := wf.getWorkflowCopyStates(ctx, tablet, streamIds) + if err != nil { + return err + } + + m.Lock() + defer m.Unlock() + + for _, copyState := range copyStates { + shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, copyState.StreamId) + copyStatesByShardStreamId[shardStreamId] = append( + copyStatesByShardStreamId[shardStreamId], + copyState, + ) + } + + return nil + } + + fetchCopyStatesEg, fetchCopyStatesCtx := errgroup.WithContext(ctx) + for tablet, result := range workflowsByShard { + tablet := tablet // loop closure + + streamIds := make([]int32, 0, len(result.Workflows)) + for _, wf := range result.Workflows { + for _, stream := range wf.Streams { + streamIds = append(streamIds, stream.Id) + } + } + + if len(streamIds) == 0 { + continue + } + + fetchCopyStatesEg.Go(func() error { + return fetchCopyStates(fetchCopyStatesCtx, tablet, streamIds) + }) + } + if err := fetchCopyStatesEg.Wait(); err != nil { + return nil, err + } + + return copyStatesByShardStreamId, nil +} + +func (wf *workflowFetcher) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { + span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.getWorkflowCopyStates") + defer span.Finish() + + span.Annotate("keyspace", tablet.Keyspace) + span.Annotate("shard", tablet.Shard) + span.Annotate("tablet_alias", tablet.AliasString()) + span.Annotate("stream_ids", fmt.Sprintf("%#v", streamIds)) + + idsBV, err := sqltypes.BuildBindVariable(streamIds) + if err != nil { + return nil, err + } + query, err := sqlparser.ParseAndBind("select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in %a and id in (select max(id) from _vt.copy_state where vrepl_id in %a group by vrepl_id, table_name)", + idsBV, idsBV) + if err != nil { + return nil, err + } + qr, err := wf.tmc.VReplicationExec(ctx, tablet.Tablet, query) + if err != nil { + return nil, err + } + + result := sqltypes.Proto3ToResult(qr) + if result == nil { + return nil, nil + } + + copyStates := make([]*vtctldatapb.Workflow_Stream_CopyState, len(result.Rows)) + for i, row := range result.Rows { + streamId, err := row[0].ToInt64() + if err != nil { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to cast vrepl_id to int64: %v", err) + } + // These string fields are technically varbinary, but this is close enough. + copyStates[i] = &vtctldatapb.Workflow_Stream_CopyState{ + StreamId: streamId, + Table: row[1].ToString(), + LastPk: row[2].ToString(), + } + } + + return copyStates, nil +} + +func (wf *workflowFetcher) buildWorkflows( + ctx context.Context, + results map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, + copyStatesByShardStreamId map[string][]*vtctldatapb.Workflow_Stream_CopyState, + req *vtctldatapb.GetWorkflowsRequest, +) ([]*vtctldatapb.Workflow, error) { + workflowsMap := make(map[string]*vtctldatapb.Workflow, len(results)) + workflowMetadataMap := make(map[string]*workflowMetadata, len(results)) + + for tablet, result := range results { + // In the old implementation, we knew we had at most one (0 <= N <= 1) + // workflow for each shard primary we queried. There might be multiple + // rows (streams) comprising that workflow, so we would aggregate the + // rows for a given primary into a single value ("the workflow", + // ReplicationStatusResult in the old types). + // + // In this version, we have many (N >= 0) workflows for each shard + // primary we queried, so we need to determine if each row corresponds + // to a workflow we're already aggregating, or if it's a workflow we + // haven't seen yet for that shard primary. We use the workflow name to + // dedupe for this. + for _, wfres := range result.Workflows { + workflowName := wfres.Workflow + workflow, ok := workflowsMap[workflowName] + if !ok { + workflow = &vtctldatapb.Workflow{ + Name: workflowName, + ShardStreams: map[string]*vtctldatapb.Workflow_ShardStream{}, + } + + workflowsMap[workflowName] = workflow + workflowMetadataMap[workflowName] = &workflowMetadata{ + sourceShards: sets.New[string](), + targetShards: sets.New[string](), + } + } + + err := wf.scanWorkflow(ctx, workflow, wfres, tablet, workflowMetadataMap, copyStatesByShardStreamId, req.Keyspace) + if err != nil { + return nil, err + } + } + } + + for name, workflow := range workflowsMap { + meta, ok := workflowMetadataMap[name] + if !ok { + return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no workflow metadata", name) + } + updateWorkflowWithMetadata(workflow, meta) + + // Sort shard streams by stream_id ASC, to support an optimization + // in fetchStreamLogs below. + for _, shardStreams := range workflow.ShardStreams { + sort.Slice(shardStreams.Streams, func(i, j int) bool { + return shardStreams.Streams[i].Id < shardStreams.Streams[j].Id + }) + } + } + + if req.IncludeLogs { + var fetchLogsWG sync.WaitGroup + + for _, workflow := range workflowsMap { + // Fetch logs for all streams associated with this workflow in the background. + fetchLogsWG.Add(1) + go func(ctx context.Context, workflow *vtctldatapb.Workflow) { + defer fetchLogsWG.Done() + wf.fetchStreamLogs(ctx, req.Keyspace, workflow) + }(ctx, workflow) + } + + // Wait for all the log fetchers to finish. + fetchLogsWG.Wait() + } + + return maps.Values(workflowsMap), nil +} + +func (wf *workflowFetcher) scanWorkflow( + ctx context.Context, + workflow *vtctldatapb.Workflow, + res *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, + tablet *topo.TabletInfo, + workflowMetadataMap map[string]*workflowMetadata, + copyStatesByShardStreamId map[string][]*vtctldatapb.Workflow_Stream_CopyState, + keyspace string, +) error { + for _, rstream := range res.Streams { + // The value in the pos column can be compressed and thus not + // have a valid GTID consisting of valid UTF-8 characters so we + // have to decode it so that it's properly decompressed first + // when needed. + pos := rstream.Pos + if pos != "" { + mpos, err := binlogplayer.DecodePosition(pos) + if err != nil { + return err + } + pos = mpos.String() + } + + cells := strings.Split(res.Cells, ",") + for i := range cells { + cells[i] = strings.TrimSpace(cells[i]) + } + options := res.Options + if options != "" { + if err := json.Unmarshal([]byte(options), &workflow.Options); err != nil { + return err + } + } + + stream := &vtctldatapb.Workflow_Stream{ + Id: int64(rstream.Id), + Shard: tablet.Shard, + Tablet: tablet.Alias, + BinlogSource: rstream.Bls, + Position: pos, + StopPosition: rstream.StopPos, + State: rstream.State.String(), + DbName: tablet.DbName(), + TabletTypes: res.TabletTypes, + TabletSelectionPreference: res.TabletSelectionPreference, + Cells: cells, + TransactionTimestamp: rstream.TransactionTimestamp, + TimeUpdated: rstream.TimeUpdated, + Message: rstream.Message, + Tags: strings.Split(res.Tags, ","), + RowsCopied: rstream.RowsCopied, + ThrottlerStatus: &vtctldatapb.Workflow_Stream_ThrottlerStatus{ + ComponentThrottled: rstream.ComponentThrottled, + TimeThrottled: rstream.TimeThrottled, + }, + } + + // Merge in copy states, which we've already fetched. + shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, stream.Id) + if copyState, ok := copyStatesByShardStreamId[shardStreamId]; ok { + stream.CopyStates = copyState + } + + if rstream.TimeUpdated == nil { + rstream.TimeUpdated = &vttimepb.Time{} + } + + switch { + case strings.Contains(strings.ToLower(stream.Message), "error"): + stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: + stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-rstream.TimeUpdated.Seconds > 10: + stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() + } + + shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) + shardStream, ok := workflow.ShardStreams[shardStreamKey] + if !ok { + ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) + defer cancel() + + si, err := wf.ts.GetShard(ctx, keyspace, tablet.Shard) + if err != nil { + return err + } + + shardStream = &vtctldatapb.Workflow_ShardStream{ + Streams: nil, + TabletControls: si.TabletControls, + IsPrimaryServing: si.IsPrimaryServing, + } + + workflow.ShardStreams[shardStreamKey] = shardStream + } + + shardStream.Streams = append(shardStream.Streams, stream) + + meta := workflowMetadataMap[workflow.Name] + meta.sourceShards.Insert(stream.BinlogSource.Shard) + meta.targetShards.Insert(tablet.Shard) + + if meta.sourceKeyspace != "" && meta.sourceKeyspace != stream.BinlogSource.Keyspace { + return vterrors.Wrapf(ErrMultipleSourceKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, meta.sourceKeyspace, stream.BinlogSource.Keyspace) + } + + meta.sourceKeyspace = stream.BinlogSource.Keyspace + + if meta.targetKeyspace != "" && meta.targetKeyspace != tablet.Keyspace { + return vterrors.Wrapf(ErrMultipleTargetKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, meta.targetKeyspace, tablet.Keyspace) + } + + meta.targetKeyspace = tablet.Keyspace + + if stream.TimeUpdated == nil { + stream.TimeUpdated = &vttimepb.Time{} + } + timeUpdated := time.Unix(stream.TimeUpdated.Seconds, 0) + vreplicationLag := time.Since(timeUpdated) + + // MaxVReplicationLag represents the time since we last processed any event + // in the workflow. + if vreplicationLag.Seconds() > meta.maxVReplicationLag { + meta.maxVReplicationLag = vreplicationLag.Seconds() + } + + workflow.WorkflowType = res.WorkflowType.String() + workflow.WorkflowSubType = res.WorkflowSubType.String() + workflow.DeferSecondaryKeys = res.DeferSecondaryKeys + + // MaxVReplicationTransactionLag estimates the actual statement processing lag + // between the source and the target. If we are still processing source events it + // is the difference b/w current time and the timestamp of the last event. If + // heartbeats are more recent than the last event, then the lag is the time since + // the last heartbeat as there can be an actual event immediately after the + // heartbeat, but which has not yet been processed on the target. + // We don't allow switching during the copy phase, so in that case we just return + // a large lag. All timestamps are in seconds since epoch. + if rstream.TransactionTimestamp == nil { + rstream.TransactionTimestamp = &vttimepb.Time{} + } + lastTransactionTime := rstream.TransactionTimestamp.Seconds + if rstream.TimeHeartbeat == nil { + rstream.TimeHeartbeat = &vttimepb.Time{} + } + lastHeartbeatTime := rstream.TimeHeartbeat.Seconds + if stream.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { + meta.maxVReplicationTransactionLag = math.MaxInt64 + } else { + if lastTransactionTime == 0 /* no new events after copy */ || + lastHeartbeatTime > lastTransactionTime /* no recent transactions, so all caught up */ { + + lastTransactionTime = lastHeartbeatTime + } + now := time.Now().Unix() /* seconds since epoch */ + transactionReplicationLag := float64(now - lastTransactionTime) + if transactionReplicationLag > meta.maxVReplicationTransactionLag { + meta.maxVReplicationTransactionLag = transactionReplicationLag + } + } + } + + return nil +} + +func updateWorkflowWithMetadata(workflow *vtctldatapb.Workflow, meta *workflowMetadata) { + workflow.Source = &vtctldatapb.Workflow_ReplicationLocation{ + Keyspace: meta.sourceKeyspace, + Shards: sets.List(meta.sourceShards), + } + + workflow.Target = &vtctldatapb.Workflow_ReplicationLocation{ + Keyspace: meta.targetKeyspace, + Shards: sets.List(meta.targetShards), + } + + workflow.MaxVReplicationLag = int64(meta.maxVReplicationLag) + workflow.MaxVReplicationTransactionLag = int64(meta.maxVReplicationTransactionLag) +} + +func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, workflow *vtctldatapb.Workflow) { + span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.fetchStreamLogs") + defer span.Finish() + + span.Annotate("keyspace", keyspace) + span.Annotate("workflow", workflow.Name) + + vreplIDs := make([]int64, 0, len(workflow.ShardStreams)) + for _, shardStream := range maps.Values(workflow.ShardStreams) { + for _, stream := range shardStream.Streams { + vreplIDs = append(vreplIDs, stream.Id) + } + } + idsBV, err := sqltypes.BuildBindVariable(vreplIDs) + if err != nil { + return + } + + query, err := sqlparser.ParseAndBind(vrepLogQuery, idsBV) + if err != nil { + return + } + + vx := vexec.NewVExec(keyspace, workflow.Name, wf.ts, wf.tmc, wf.parser) + results, err := vx.QueryContext(ctx, query) + if err != nil { + // Note that we do not return here. If there are any query results + // in the map (i.e. some tablets returned successfully), we will + // still try to read log rows from them on a best-effort basis. But, + // we will also pre-emptively record the top-level fetch error on + // every stream in every shard in the workflow. Further processing + // below may override the error message for certain streams. + for _, streams := range workflow.ShardStreams { + for _, stream := range streams.Streams { + stream.LogFetchError = err.Error() + } + } + } + + for target, p3qr := range results { + qr := sqltypes.Proto3ToResult(p3qr) + shardStreamKey := fmt.Sprintf("%s/%s", target.Shard, target.AliasString()) + + ss, ok := workflow.ShardStreams[shardStreamKey] + if !ok || ss == nil { + continue + } + + streams := ss.Streams + streamIdx := 0 + markErrors := func(err error) { + if streamIdx >= len(streams) { + return + } + + streams[streamIdx].LogFetchError = err.Error() + } + + for _, row := range qr.Rows { + id, err := row[0].ToCastInt64() + if err != nil { + markErrors(err) + continue + } + + streamID, err := row[1].ToCastInt64() + if err != nil { + markErrors(err) + continue + } + + typ := row[2].ToString() + state := row[3].ToString() + message := row[4].ToString() + + createdAt, err := time.Parse("2006-01-02 15:04:05", row[5].ToString()) + if err != nil { + markErrors(err) + continue + } + + updatedAt, err := time.Parse("2006-01-02 15:04:05", row[6].ToString()) + if err != nil { + markErrors(err) + continue + } + + count, err := row[7].ToCastInt64() + if err != nil { + markErrors(err) + continue + } + + streamLog := &vtctldatapb.Workflow_Stream_Log{ + Id: id, + StreamId: streamID, + Type: typ, + State: state, + CreatedAt: &vttimepb.Time{ + Seconds: createdAt.Unix(), + }, + UpdatedAt: &vttimepb.Time{ + Seconds: updatedAt.Unix(), + }, + Message: message, + Count: count, + } + + // Earlier, in buildWorkflows, we sorted each ShardStreams + // slice by ascending id, and our _vt.vreplication_log query + // ordered by (stream_id ASC, id ASC), so we can walk the + // streams in index order in O(n) amortized over all the rows + // for this tablet. + for streamIdx < len(streams) { + stream := streams[streamIdx] + if stream.Id < streamLog.StreamId { + streamIdx++ + continue + } + + if stream.Id > streamLog.StreamId { + wf.logger.Warningf("Found stream log for nonexistent stream: %+v", streamLog) + // This can happen on manual/failed workflow cleanup so move to the next log. + break + } + + // stream.Id == streamLog.StreamId + stream.Logs = append(stream.Logs, streamLog) + break + } + } + } +} From c4cfd04aff997814edcf2d9b2f3c78e31299e006 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Tue, 29 Oct 2024 22:47:45 +0530 Subject: [PATCH 02/12] refac: Add more utility functions Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/workflows.go | 140 +++++++++++++++++------------- 1 file changed, 82 insertions(+), 58 deletions(-) diff --git a/go/vt/vtctl/workflow/workflows.go b/go/vt/vtctl/workflow/workflows.go index 8d757db33f1..21be2b4719a 100644 --- a/go/vt/vtctl/workflow/workflows.go +++ b/go/vt/vtctl/workflow/workflows.go @@ -107,36 +107,26 @@ func (wf *workflowFetcher) fetchWorkflowsByShard( results := make(map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, len(shards)) - readWorkflowsEg, readWorkflowsCtx := errgroup.WithContext(ctx) - for _, shard := range shards { - readWorkflowsEg.Go(func() error { - si, err := wf.ts.GetShard(readWorkflowsCtx, req.Keyspace, shard) - if err != nil { - return err - } - if si.PrimaryAlias == nil { - return fmt.Errorf("%w %s/%s", vexec.ErrNoShardPrimary, req.Keyspace, shard) - } - primary, err := wf.ts.GetTablet(readWorkflowsCtx, si.PrimaryAlias) - if err != nil { - return err - } - if primary == nil { - return fmt.Errorf("%w %s/%s: tablet %v not found", vexec.ErrNoShardPrimary, req.Keyspace, shard, topoproto.TabletAliasString(si.PrimaryAlias)) - } - // Clone the request so that we can set the correct DB name for tablet. - req := readReq.CloneVT() - wres, err := wf.tmc.ReadVReplicationWorkflows(readWorkflowsCtx, primary.Tablet, req) - if err != nil { - return err - } - m.Lock() - defer m.Unlock() - results[primary] = wres - return nil - }) - } - if readWorkflowsEg.Wait() != nil { + err = wf.forAllShards(ctx, req.Keyspace, shards, func(ctx context.Context, si *topo.ShardInfo) error { + primary, err := wf.ts.GetTablet(ctx, si.PrimaryAlias) + if err != nil { + return err + } + if primary == nil { + return fmt.Errorf("%w %s/%s: tablet %v not found", vexec.ErrNoShardPrimary, req.Keyspace, si.ShardName(), topoproto.TabletAliasString(si.PrimaryAlias)) + } + // Clone the request so that we can set the correct DB name for tablet. + req := readReq.CloneVT() + wres, err := wf.tmc.ReadVReplicationWorkflows(ctx, primary.Tablet, req) + if err != nil { + return err + } + m.Lock() + defer m.Unlock() + results[primary] = wres + return nil + }) + if err != nil { return nil, err } @@ -336,6 +326,26 @@ func (wf *workflowFetcher) scanWorkflow( copyStatesByShardStreamId map[string][]*vtctldatapb.Workflow_Stream_CopyState, keyspace string, ) error { + shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) + shardStream, ok := workflow.ShardStreams[shardStreamKey] + if !ok { + ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) + defer cancel() + + si, err := wf.ts.GetShard(ctx, keyspace, tablet.Shard) + if err != nil { + return err + } + + shardStream = &vtctldatapb.Workflow_ShardStream{ + Streams: nil, + TabletControls: si.TabletControls, + IsPrimaryServing: si.IsPrimaryServing, + } + + workflow.ShardStreams[shardStreamKey] = shardStream + } + for _, rstream := range res.Streams { // The value in the pos column can be compressed and thus not // have a valid GTID consisting of valid UTF-8 characters so we @@ -394,34 +404,7 @@ func (wf *workflowFetcher) scanWorkflow( rstream.TimeUpdated = &vttimepb.Time{} } - switch { - case strings.Contains(strings.ToLower(stream.Message), "error"): - stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: - stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-rstream.TimeUpdated.Seconds > 10: - stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() - } - - shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) - shardStream, ok := workflow.ShardStreams[shardStreamKey] - if !ok { - ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) - defer cancel() - - si, err := wf.ts.GetShard(ctx, keyspace, tablet.Shard) - if err != nil { - return err - } - - shardStream = &vtctldatapb.Workflow_ShardStream{ - Streams: nil, - TabletControls: si.TabletControls, - IsPrimaryServing: si.IsPrimaryServing, - } - - workflow.ShardStreams[shardStreamKey] = shardStream - } + stream.State = getStreamState(stream, rstream) shardStream.Streams = append(shardStream.Streams, stream) @@ -640,3 +623,44 @@ func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, } } } + +func (wf *workflowFetcher) forAllShards( + ctx context.Context, + keyspace string, + shards []string, + f func(ctx context.Context, shard *topo.ShardInfo) error, +) error { + eg, egCtx := errgroup.WithContext(ctx) + for _, shard := range shards { + eg.Go(func() error { + si, err := wf.ts.GetShard(ctx, keyspace, shard) + if err != nil { + return err + } + if si.PrimaryAlias == nil { + return fmt.Errorf("%w %s/%s", vexec.ErrNoShardPrimary, keyspace, shard) + } + + if err := f(egCtx, si); err != nil { + return err + } + return nil + }) + } + if err := eg.Wait(); err != nil { + return err + } + return nil +} + +func getStreamState(stream *vtctldatapb.Workflow_Stream, rstream *tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream) string { + switch { + case strings.Contains(strings.ToLower(stream.Message), "error"): + return binlogdatapb.VReplicationWorkflowState_Error.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: + return binlogdatapb.VReplicationWorkflowState_Copying.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-rstream.TimeUpdated.Seconds > 10: + return binlogdatapb.VReplicationWorkflowState_Lagging.String() + } + return rstream.State.String() +} From aedf96558a97f0ec0484302bc832be300219daf9 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Wed, 6 Nov 2024 19:09:47 +0530 Subject: [PATCH 03/12] refac: More cleanup and unit tests Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/server.go | 8 +-- go/vt/vtctl/workflow/workflows.go | 57 ++++++++--------- go/vt/vtctl/workflow/workflows_test.go | 87 ++++++++++++++++++++++++++ 3 files changed, 117 insertions(+), 35 deletions(-) create mode 100644 go/vt/vtctl/workflow/workflows_test.go diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index afd01e12814..7858dbe10e4 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -402,24 +402,24 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows span.Annotate("include_logs", req.IncludeLogs) span.Annotate("shards", req.Shards) - wf := &workflowFetcher{ + w := &workflow{ ts: s.ts, tmc: s.tmc, parser: s.SQLParser(), logger: s.Logger(), } - workflowsByShard, err := wf.fetchWorkflowsByShard(ctx, req) + workflowsByShard, err := w.fetchWorkflowsByShard(ctx, req) if err != nil { return nil, err } - copyStatesByShardStreamId, err := wf.fetchCopyStatesByShardStream(ctx, workflowsByShard) + copyStatesByShardStreamId, err := w.fetchCopyStatesByShardStream(ctx, workflowsByShard) if err != nil { return nil, err } - workflows, err := wf.buildWorkflows(ctx, workflowsByShard, copyStatesByShardStreamId, req) + workflows, err := w.buildWorkflows(ctx, workflowsByShard, copyStatesByShardStreamId, req) if err != nil { return nil, err } diff --git a/go/vt/vtctl/workflow/workflows.go b/go/vt/vtctl/workflow/workflows.go index 21be2b4719a..37b17f4ed79 100644 --- a/go/vt/vtctl/workflow/workflows.go +++ b/go/vt/vtctl/workflow/workflows.go @@ -49,9 +49,9 @@ import ( vttimepb "vitess.io/vitess/go/vt/proto/vttime" ) -// workflowFetcher is responsible for fetching and retrieving information +// workflow is responsible for fetching and retrieving information // about VReplication workflows. -type workflowFetcher struct { +type workflow struct { ts *topo.Server tmc tmclient.TabletManagerClient @@ -86,7 +86,7 @@ ORDER BY id ASC `) -func (wf *workflowFetcher) fetchWorkflowsByShard( +func (w *workflow) fetchWorkflowsByShard( ctx context.Context, req *vtctldatapb.GetWorkflowsRequest, ) (map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { @@ -100,15 +100,15 @@ func (wf *workflowFetcher) fetchWorkflowsByShard( m := sync.Mutex{} - shards, err := common.GetShards(ctx, wf.ts, req.Keyspace, req.Shards) + shards, err := common.GetShards(ctx, w.ts, req.Keyspace, req.Shards) if err != nil { return nil, err } results := make(map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, len(shards)) - err = wf.forAllShards(ctx, req.Keyspace, shards, func(ctx context.Context, si *topo.ShardInfo) error { - primary, err := wf.ts.GetTablet(ctx, si.PrimaryAlias) + err = w.forAllShards(ctx, req.Keyspace, shards, func(ctx context.Context, si *topo.ShardInfo) error { + primary, err := w.ts.GetTablet(ctx, si.PrimaryAlias) if err != nil { return err } @@ -117,7 +117,7 @@ func (wf *workflowFetcher) fetchWorkflowsByShard( } // Clone the request so that we can set the correct DB name for tablet. req := readReq.CloneVT() - wres, err := wf.tmc.ReadVReplicationWorkflows(ctx, primary.Tablet, req) + wres, err := w.tmc.ReadVReplicationWorkflows(ctx, primary.Tablet, req) if err != nil { return err } @@ -133,7 +133,7 @@ func (wf *workflowFetcher) fetchWorkflowsByShard( return results, nil } -func (wf *workflowFetcher) fetchCopyStatesByShardStream( +func (w *workflow) fetchCopyStatesByShardStream( ctx context.Context, workflowsByShard map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, ) (map[string][]*vtctldatapb.Workflow_Stream_CopyState, error) { @@ -148,7 +148,7 @@ func (wf *workflowFetcher) fetchCopyStatesByShardStream( span.Annotate("shard", tablet.Shard) span.Annotate("tablet_alias", tablet.AliasString()) - copyStates, err := wf.getWorkflowCopyStates(ctx, tablet, streamIds) + copyStates, err := w.getWorkflowCopyStates(ctx, tablet, streamIds) if err != nil { return err } @@ -169,8 +169,6 @@ func (wf *workflowFetcher) fetchCopyStatesByShardStream( fetchCopyStatesEg, fetchCopyStatesCtx := errgroup.WithContext(ctx) for tablet, result := range workflowsByShard { - tablet := tablet // loop closure - streamIds := make([]int32, 0, len(result.Workflows)) for _, wf := range result.Workflows { for _, stream := range wf.Streams { @@ -193,7 +191,7 @@ func (wf *workflowFetcher) fetchCopyStatesByShardStream( return copyStatesByShardStreamId, nil } -func (wf *workflowFetcher) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { +func (w *workflow) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.getWorkflowCopyStates") defer span.Finish() @@ -211,7 +209,7 @@ func (wf *workflowFetcher) getWorkflowCopyStates(ctx context.Context, tablet *to if err != nil { return nil, err } - qr, err := wf.tmc.VReplicationExec(ctx, tablet.Tablet, query) + qr, err := w.tmc.VReplicationExec(ctx, tablet.Tablet, query) if err != nil { return nil, err } @@ -238,7 +236,7 @@ func (wf *workflowFetcher) getWorkflowCopyStates(ctx context.Context, tablet *to return copyStates, nil } -func (wf *workflowFetcher) buildWorkflows( +func (w *workflow) buildWorkflows( ctx context.Context, results map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, copyStatesByShardStreamId map[string][]*vtctldatapb.Workflow_Stream_CopyState, @@ -275,7 +273,8 @@ func (wf *workflowFetcher) buildWorkflows( } } - err := wf.scanWorkflow(ctx, workflow, wfres, tablet, workflowMetadataMap, copyStatesByShardStreamId, req.Keyspace) + metadata := workflowMetadataMap[workflowName] + err := w.scanWorkflow(ctx, workflow, wfres, tablet, metadata, copyStatesByShardStreamId, req.Keyspace) if err != nil { return nil, err } @@ -283,10 +282,7 @@ func (wf *workflowFetcher) buildWorkflows( } for name, workflow := range workflowsMap { - meta, ok := workflowMetadataMap[name] - if !ok { - return nil, vterrors.Wrapf(ErrInvalidWorkflow, "%s has no workflow metadata", name) - } + meta := workflowMetadataMap[name] updateWorkflowWithMetadata(workflow, meta) // Sort shard streams by stream_id ASC, to support an optimization @@ -306,7 +302,7 @@ func (wf *workflowFetcher) buildWorkflows( fetchLogsWG.Add(1) go func(ctx context.Context, workflow *vtctldatapb.Workflow) { defer fetchLogsWG.Done() - wf.fetchStreamLogs(ctx, req.Keyspace, workflow) + w.fetchStreamLogs(ctx, req.Keyspace, workflow) }(ctx, workflow) } @@ -317,12 +313,12 @@ func (wf *workflowFetcher) buildWorkflows( return maps.Values(workflowsMap), nil } -func (wf *workflowFetcher) scanWorkflow( +func (w *workflow) scanWorkflow( ctx context.Context, workflow *vtctldatapb.Workflow, res *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, tablet *topo.TabletInfo, - workflowMetadataMap map[string]*workflowMetadata, + meta *workflowMetadata, copyStatesByShardStreamId map[string][]*vtctldatapb.Workflow_Stream_CopyState, keyspace string, ) error { @@ -332,7 +328,7 @@ func (wf *workflowFetcher) scanWorkflow( ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) defer cancel() - si, err := wf.ts.GetShard(ctx, keyspace, tablet.Shard) + si, err := w.ts.GetShard(ctx, keyspace, tablet.Shard) if err != nil { return err } @@ -396,8 +392,8 @@ func (wf *workflowFetcher) scanWorkflow( // Merge in copy states, which we've already fetched. shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, stream.Id) - if copyState, ok := copyStatesByShardStreamId[shardStreamId]; ok { - stream.CopyStates = copyState + if copyStates, ok := copyStatesByShardStreamId[shardStreamId]; ok { + stream.CopyStates = copyStates } if rstream.TimeUpdated == nil { @@ -408,7 +404,6 @@ func (wf *workflowFetcher) scanWorkflow( shardStream.Streams = append(shardStream.Streams, stream) - meta := workflowMetadataMap[workflow.Name] meta.sourceShards.Insert(stream.BinlogSource.Shard) meta.targetShards.Insert(tablet.Shard) @@ -490,7 +485,7 @@ func updateWorkflowWithMetadata(workflow *vtctldatapb.Workflow, meta *workflowMe workflow.MaxVReplicationTransactionLag = int64(meta.maxVReplicationTransactionLag) } -func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, workflow *vtctldatapb.Workflow) { +func (w *workflow) fetchStreamLogs(ctx context.Context, keyspace string, workflow *vtctldatapb.Workflow) { span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.fetchStreamLogs") defer span.Finish() @@ -513,7 +508,7 @@ func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, return } - vx := vexec.NewVExec(keyspace, workflow.Name, wf.ts, wf.tmc, wf.parser) + vx := vexec.NewVExec(keyspace, workflow.Name, w.ts, w.tmc, w.parser) results, err := vx.QueryContext(ctx, query) if err != nil { // Note that we do not return here. If there are any query results @@ -611,7 +606,7 @@ func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, } if stream.Id > streamLog.StreamId { - wf.logger.Warningf("Found stream log for nonexistent stream: %+v", streamLog) + w.logger.Warningf("Found stream log for nonexistent stream: %+v", streamLog) // This can happen on manual/failed workflow cleanup so move to the next log. break } @@ -624,7 +619,7 @@ func (wf *workflowFetcher) fetchStreamLogs(ctx context.Context, keyspace string, } } -func (wf *workflowFetcher) forAllShards( +func (w *workflow) forAllShards( ctx context.Context, keyspace string, shards []string, @@ -633,7 +628,7 @@ func (wf *workflowFetcher) forAllShards( eg, egCtx := errgroup.WithContext(ctx) for _, shard := range shards { eg.Go(func() error { - si, err := wf.ts.GetShard(ctx, keyspace, shard) + si, err := w.ts.GetShard(ctx, keyspace, shard) if err != nil { return err } diff --git a/go/vt/vtctl/workflow/workflows_test.go b/go/vt/vtctl/workflow/workflows_test.go new file mode 100644 index 00000000000..f1f4c6babc2 --- /dev/null +++ b/go/vt/vtctl/workflow/workflows_test.go @@ -0,0 +1,87 @@ +/* +Copyright 2024 The Vitess Authors. + +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 workflow + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + "vitess.io/vitess/go/vt/proto/vttime" +) + +func TestGetStreamState(t *testing.T) { + testCases := []struct { + name string + stream *vtctldatapb.Workflow_Stream + rstream *tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream + want string + }{ + { + name: "error state", + stream: &vtctldatapb.Workflow_Stream{ + Message: "test error", + }, + want: "Error", + }, + { + name: "copying state", + stream: &vtctldatapb.Workflow_Stream{ + State: "Running", + CopyStates: []*vtctldatapb.Workflow_Stream_CopyState{ + { + Table: "table1", + }, + }, + }, + want: "Copying", + }, + { + name: "lagging state", + stream: &vtctldatapb.Workflow_Stream{ + State: "Running", + }, + rstream: &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + TimeUpdated: &vttime.Time{ + Seconds: int64(time.Now().Second()) - 11, + }, + }, + want: "Lagging", + }, + { + name: "non-running and error free", + stream: &vtctldatapb.Workflow_Stream{ + State: "Stopped", + }, + rstream: &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + State: binlogdata.VReplicationWorkflowState_Stopped, + }, + want: "Stopped", + }, + } + + for _, tt := range testCases { + t.Run(tt.name, func(t *testing.T) { + state := getStreamState(tt.stream, tt.rstream) + assert.Equal(t, tt.want, state) + }) + } +} From 7e963fffea865ddc8de97f243559b2b9d1920b8f Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Thu, 7 Nov 2024 12:51:37 +0530 Subject: [PATCH 04/12] refac: Named rows and unit tests for copyState Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/workflows.go | 32 ++++++------ go/vt/vtctl/workflow/workflows_test.go | 68 +++++++++++++++++++++++++- 2 files changed, 83 insertions(+), 17 deletions(-) diff --git a/go/vt/vtctl/workflow/workflows.go b/go/vt/vtctl/workflow/workflows.go index 37b17f4ed79..cd0b437edb0 100644 --- a/go/vt/vtctl/workflow/workflows.go +++ b/go/vt/vtctl/workflow/workflows.go @@ -142,7 +142,7 @@ func (w *workflow) fetchCopyStatesByShardStream( copyStatesByShardStreamId := make(map[string][]*vtctldatapb.Workflow_Stream_CopyState, len(workflowsByShard)) fetchCopyStates := func(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) error { - span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.fetchCopyStates") + span, ctx := trace.NewSpan(ctx, "workflow.workflow.fetchCopyStates") defer span.Finish() span.Annotate("shard", tablet.Shard) @@ -192,7 +192,7 @@ func (w *workflow) fetchCopyStatesByShardStream( } func (w *workflow) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { - span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.getWorkflowCopyStates") + span, ctx := trace.NewSpan(ctx, "workflow.workflow.getWorkflowCopyStates") defer span.Finish() span.Annotate("keyspace", tablet.Keyspace) @@ -220,16 +220,16 @@ func (w *workflow) getWorkflowCopyStates(ctx context.Context, tablet *topo.Table } copyStates := make([]*vtctldatapb.Workflow_Stream_CopyState, len(result.Rows)) - for i, row := range result.Rows { - streamId, err := row[0].ToInt64() + for i, row := range result.Named().Rows { + streamId, err := row["vrepl_id"].ToInt64() if err != nil { return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to cast vrepl_id to int64: %v", err) } // These string fields are technically varbinary, but this is close enough. copyStates[i] = &vtctldatapb.Workflow_Stream_CopyState{ StreamId: streamId, - Table: row[1].ToString(), - LastPk: row[2].ToString(), + Table: row["table_name"].ToString(), + LastPk: row["lastpk"].ToString(), } } @@ -486,7 +486,7 @@ func updateWorkflowWithMetadata(workflow *vtctldatapb.Workflow, meta *workflowMe } func (w *workflow) fetchStreamLogs(ctx context.Context, keyspace string, workflow *vtctldatapb.Workflow) { - span, ctx := trace.NewSpan(ctx, "workflow.workflowFetcher.fetchStreamLogs") + span, ctx := trace.NewSpan(ctx, "workflow.workflow.fetchStreamLogs") defer span.Finish() span.Annotate("keyspace", keyspace) @@ -543,36 +543,36 @@ func (w *workflow) fetchStreamLogs(ctx context.Context, keyspace string, workflo streams[streamIdx].LogFetchError = err.Error() } - for _, row := range qr.Rows { - id, err := row[0].ToCastInt64() + for _, row := range qr.Named().Rows { + id, err := row["id"].ToCastInt64() if err != nil { markErrors(err) continue } - streamID, err := row[1].ToCastInt64() + streamID, err := row["vrepl_id"].ToCastInt64() if err != nil { markErrors(err) continue } - typ := row[2].ToString() - state := row[3].ToString() - message := row[4].ToString() + typ := row["type"].ToString() + state := row["state"].ToString() + message := row["message"].ToString() - createdAt, err := time.Parse("2006-01-02 15:04:05", row[5].ToString()) + createdAt, err := time.Parse("2006-01-02 15:04:05", row["created_at"].ToString()) if err != nil { markErrors(err) continue } - updatedAt, err := time.Parse("2006-01-02 15:04:05", row[6].ToString()) + updatedAt, err := time.Parse("2006-01-02 15:04:05", row["updated_at"].ToString()) if err != nil { markErrors(err) continue } - count, err := row[7].ToCastInt64() + count, err := row["count"].ToCastInt64() if err != nil { markErrors(err) continue diff --git a/go/vt/vtctl/workflow/workflows_test.go b/go/vt/vtctl/workflow/workflows_test.go index f1f4c6babc2..6e64fbb3ce3 100644 --- a/go/vt/vtctl/workflow/workflows_test.go +++ b/go/vt/vtctl/workflow/workflows_test.go @@ -17,15 +17,21 @@ limitations under the License. package workflow import ( + "context" + "fmt" "testing" "time" "github.com/stretchr/testify/assert" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/proto/binlogdata" + "vitess.io/vitess/go/vt/proto/vttime" + "vitess.io/vitess/go/vt/topo" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" - "vitess.io/vitess/go/vt/proto/vttime" ) func TestGetStreamState(t *testing.T) { @@ -85,3 +91,63 @@ func TestGetStreamState(t *testing.T) { }) } } + +func TestGetWorkflowCopyStates(t *testing.T) { + ctx := context.Background() + + sourceShards := []string{"-"} + targetShards := []string{"-"} + + te := newTestMaterializerEnv(t, ctx, &vtctldatapb.MaterializeSettings{ + SourceKeyspace: "source_keyspace", + TargetKeyspace: "target_keyspace", + Workflow: "test_workflow", + TableSettings: []*vtctldatapb.TableMaterializeSettings{ + { + TargetTable: "table1", + SourceExpression: fmt.Sprintf("select * from %s", "table1"), + }, + { + TargetTable: "table2", + SourceExpression: fmt.Sprintf("select * from %s", "table2"), + }, + }, + }, sourceShards, targetShards) + + w := workflow{ + ts: te.ws.ts, + tmc: te.tmc, + } + + tablet := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + } + + query := "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)" + te.tmc.expectVRQuery(100, query, sqltypes.MakeTestResult( + sqltypes.MakeTestFields("vrepl_id|table_name|lastpk", "int64|varchar|varchar"), + "1|table1|2", "1|table2|1", + )) + + copyStates, err := w.getWorkflowCopyStates(ctx, &topo.TabletInfo{ + Tablet: tablet, + }, []int32{1}) + assert.NoError(t, err) + assert.Len(t, copyStates, 2) + + state1 := &vtctldatapb.Workflow_Stream_CopyState{ + Table: "table1", + LastPk: "2", + StreamId: 1, + } + state2 := &vtctldatapb.Workflow_Stream_CopyState{ + Table: "table2", + LastPk: "1", + StreamId: 1, + } + assert.Contains(t, copyStates, state1) + assert.Contains(t, copyStates, state2) +} From 60a1ce949755354e32e0239d19133cfd2fb532d3 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Thu, 7 Nov 2024 13:58:05 +0530 Subject: [PATCH 05/12] test: Add more unit tests and fix failing test Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/server_test.go | 2 +- go/vt/vtctl/workflow/workflows_test.go | 107 +++++++++++++++++++++++++ 2 files changed, 108 insertions(+), 1 deletion(-) diff --git a/go/vt/vtctl/workflow/server_test.go b/go/vt/vtctl/workflow/server_test.go index 870a4ae96c3..fa027670b2b 100644 --- a/go/vt/vtctl/workflow/server_test.go +++ b/go/vt/vtctl/workflow/server_test.go @@ -2227,7 +2227,7 @@ func TestGetWorkflowsStreamLogs(t *testing.T) { }, sourceShards, targetShards) logResult := sqltypes.MakeTestResult( - sqltypes.MakeTestFields("id|vrepl_id|type|state|message|created_at|updated_at|`count`", "int64|int64|varchar|varchar|varchar|varchar|varchar|int64"), + sqltypes.MakeTestFields("id|vrepl_id|type|state|message|created_at|updated_at|count", "int64|int64|varchar|varchar|varchar|varchar|varchar|int64"), "1|0|State Change|Running|test message for non-existent 1|2006-01-02 15:04:05|2006-01-02 15:04:05|1", "2|0|State Change|Stopped|test message for non-existent 2|2006-01-02 15:04:06|2006-01-02 15:04:06|1", "3|1|State Change|Running|log message|2006-01-02 15:04:07|2006-01-02 15:04:07|1", diff --git a/go/vt/vtctl/workflow/workflows_test.go b/go/vt/vtctl/workflow/workflows_test.go index 6e64fbb3ce3..1834d8b794b 100644 --- a/go/vt/vtctl/workflow/workflows_test.go +++ b/go/vt/vtctl/workflow/workflows_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/proto/binlogdata" @@ -151,3 +152,109 @@ func TestGetWorkflowCopyStates(t *testing.T) { assert.Contains(t, copyStates, state1) assert.Contains(t, copyStates, state2) } + +func TestFetchCopyStatesByShardStream(t *testing.T) { + ctx := context.Background() + + sourceShards := []string{"-"} + targetShards := []string{"-"} + + te := newTestMaterializerEnv(t, ctx, &vtctldatapb.MaterializeSettings{ + SourceKeyspace: "source_keyspace", + TargetKeyspace: "target_keyspace", + Workflow: "test_workflow", + TableSettings: []*vtctldatapb.TableMaterializeSettings{ + { + TargetTable: "table1", + SourceExpression: fmt.Sprintf("select * from %s", "table1"), + }, + { + TargetTable: "table2", + SourceExpression: fmt.Sprintf("select * from %s", "table2"), + }, + }, + }, sourceShards, targetShards) + + w := workflow{ + ts: te.ws.ts, + tmc: te.tmc, + } + + tablet := &topodatapb.Tablet{ + Shard: "-80", + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 100, + }, + } + tablet2 := &topodatapb.Tablet{ + Shard: "80-", + Alias: &topodatapb.TabletAlias{ + Cell: "zone1", + Uid: 101, + }, + } + + query := "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1, 2) and id in (select max(id) from _vt.copy_state where vrepl_id in (1, 2) group by vrepl_id, table_name)" + te.tmc.expectVRQuery(100, query, sqltypes.MakeTestResult( + sqltypes.MakeTestFields("vrepl_id|table_name|lastpk", "int64|varchar|varchar"), + "1|table1|2", "2|table2|1", "2|table1|1", + )) + + te.tmc.expectVRQuery(101, query, sqltypes.MakeTestResult( + sqltypes.MakeTestFields("vrepl_id|table_name|lastpk", "int64|varchar|varchar"), + "1|table1|2", "1|table2|1", + )) + + ti := &topo.TabletInfo{ + Tablet: tablet, + } + ti2 := &topo.TabletInfo{ + Tablet: tablet2, + } + + readVReplicationResponse := map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse{ + ti: { + Workflows: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ + { + Streams: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + { + Id: 1, + }, { + Id: 2, + }, + }, + }, + }, + }, + ti2: { + Workflows: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ + { + Streams: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + { + Id: 1, + }, { + Id: 2, + }, + }, + }, + }, + }, + } + copyStatesByStreamId, err := w.fetchCopyStatesByShardStream(ctx, readVReplicationResponse) + assert.NoError(t, err) + + copyStates1 := copyStatesByStreamId["-80/1"] + copyStates2 := copyStatesByStreamId["-80/2"] + copyStates3 := copyStatesByStreamId["80-/1"] + + require.NotNil(t, copyStates1) + require.NotNil(t, copyStates2) + require.NotNil(t, copyStates3) + + assert.Len(t, copyStates1, 1) + assert.Len(t, copyStates2, 2) + assert.Len(t, copyStates3, 2) + + assert.Nil(t, copyStatesByStreamId["80-/2"]) +} From fc4f0c7209d6c0ff3af864e902065ca88eb1c062 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Mon, 11 Nov 2024 13:05:17 +0530 Subject: [PATCH 06/12] refac: Remove duplicate function Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/server.go | 53 +++------------------------------- 1 file changed, 4 insertions(+), 49 deletions(-) diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index f9682237d57..8b272333890 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -558,51 +558,6 @@ func (s *Server) getWorkflowState(ctx context.Context, targetKeyspace, workflowN return ts, state, nil } -func (s *Server) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { - span, ctx := trace.NewSpan(ctx, "workflow.Server.getWorkflowCopyStates") - defer span.Finish() - - span.Annotate("keyspace", tablet.Keyspace) - span.Annotate("shard", tablet.Shard) - span.Annotate("tablet_alias", tablet.AliasString()) - span.Annotate("stream_ids", fmt.Sprintf("%#v", streamIds)) - - idsBV, err := sqltypes.BuildBindVariable(streamIds) - if err != nil { - return nil, err - } - query, err := sqlparser.ParseAndBind("select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in %a and id in (select max(id) from _vt.copy_state where vrepl_id in %a group by vrepl_id, table_name)", - idsBV, idsBV) - if err != nil { - return nil, err - } - qr, err := s.tmc.VReplicationExec(ctx, tablet.Tablet, query) - if err != nil { - return nil, err - } - - result := sqltypes.Proto3ToResult(qr) - if result == nil { - return nil, nil - } - - copyStates := make([]*vtctldatapb.Workflow_Stream_CopyState, len(result.Rows)) - for i, row := range result.Rows { - streamId, err := row[0].ToInt64() - if err != nil { - return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to cast vrepl_id to int64: %v", err) - } - // These string fields are technically varbinary, but this is close enough. - copyStates[i] = &vtctldatapb.Workflow_Stream_CopyState{ - StreamId: streamId, - Table: row[1].ToString(), - LastPk: row[2].ToString(), - } - } - - return copyStates, nil -} - // LookupVindexCreate creates the lookup vindex in the specified // keyspace and creates a VReplication workflow to backfill that // vindex from the keyspace to the target/lookup table specified. @@ -1023,7 +978,7 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl } } if isStandardMoveTables() { // Non-standard ones do not use shard scoped mechanisms - if err := s.setupInitialDeniedTables(ctx, ts); err != nil { + if err := setupInitialDeniedTables(ctx, ts); err != nil { return nil, vterrors.Wrapf(err, "failed to put initial denied tables entries in place on the target shards") } } @@ -1078,7 +1033,7 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl }) } -func (s *Server) validateRoutingRuleFlags(req *vtctldatapb.MoveTablesCreateRequest, mz *materializer) error { +func validateRoutingRuleFlags(req *vtctldatapb.MoveTablesCreateRequest, mz *materializer) error { if mz.IsMultiTenantMigration() { switch { case req.NoRoutingRules: @@ -1090,7 +1045,7 @@ func (s *Server) validateRoutingRuleFlags(req *vtctldatapb.MoveTablesCreateReque return nil } -func (s *Server) setupInitialDeniedTables(ctx context.Context, ts *trafficSwitcher) error { +func setupInitialDeniedTables(ctx context.Context, ts *trafficSwitcher) error { if ts.MigrationType() != binlogdatapb.MigrationType_TABLES { return nil } @@ -1108,7 +1063,7 @@ func (s *Server) setupInitialDeniedTables(ctx context.Context, ts *trafficSwitch } func (s *Server) setupInitialRoutingRules(ctx context.Context, req *vtctldatapb.MoveTablesCreateRequest, mz *materializer, tables []string) error { - if err := s.validateRoutingRuleFlags(req, mz); err != nil { + if err := validateRoutingRuleFlags(req, mz); err != nil { return err } From 569a17d1547926fee61a62e68ed9cb32b35093b3 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Tue, 12 Nov 2024 12:24:23 +0530 Subject: [PATCH 07/12] refac: Initiate lookup.go for lookup related functions Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/lookup.go | 450 +++++++++++++++++++++++++++++++++ go/vt/vtctl/workflow/server.go | 60 +++-- 2 files changed, 484 insertions(+), 26 deletions(-) create mode 100644 go/vt/vtctl/workflow/lookup.go diff --git a/go/vt/vtctl/workflow/lookup.go b/go/vt/vtctl/workflow/lookup.go new file mode 100644 index 00000000000..96842b07e3f --- /dev/null +++ b/go/vt/vtctl/workflow/lookup.go @@ -0,0 +1,450 @@ +/* +Copyright 2024 The Vitess Authors. + +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 workflow + +import ( + "context" + "fmt" + "slices" + "strings" + + "golang.org/x/exp/maps" + "google.golang.org/protobuf/proto" + + "vitess.io/vitess/go/sqlescape" + "vitess.io/vitess/go/vt/schema" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtctl/schematools" + "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/vtgate/vindexes" + + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" +) + +// prepareCreateLookup performs the preparatory steps for creating a +// Lookup Vindex. +func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( + ms *vtctldatapb.MaterializeSettings, sourceVSchema, targetVSchema *vschemapb.Keyspace, cancelFunc func() error, err error) { + // Important variables are pulled out here. + var ( + vindexName string + vindex *vschemapb.Vindex + targetKeyspace string + targetTableName string + vindexFromCols []string + vindexToCol string + vindexIgnoreNulls bool + + sourceTableName string + // sourceTable is the supplied table info. + sourceTable *vschemapb.Table + // sourceVSchemaTable is the table info present in the vschema. + sourceVSchemaTable *vschemapb.Table + // sourceVindexColumns are computed from the input sourceTable. + sourceVindexColumns []string + + // Target table info. + createDDL string + materializeQuery string + ) + + // Validate input vindex. + if specs == nil { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") + } + if len(specs.Vindexes) != 1 { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "only one vindex must be specified") + } + vindexName = maps.Keys(specs.Vindexes)[0] + vindex = maps.Values(specs.Vindexes)[0] + if !strings.Contains(vindex.Type, "lookup") { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex %s is not a lookup type", vindex.Type) + } + targetKeyspace, targetTableName, err = w.parser.ParseTable(vindex.Params["table"]) + if err != nil || targetKeyspace == "" { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex table name (%s) must be in the form .", vindex.Params["table"]) + } + vindexFromCols = strings.Split(vindex.Params["from"], ",") + for i, col := range vindexFromCols { + vindexFromCols[i] = strings.TrimSpace(col) + } + if strings.Contains(vindex.Type, "unique") { + if len(vindexFromCols) != 1 { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unique vindex 'from' should have only one column") + } + } else { + if len(vindexFromCols) < 2 { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "non-unique vindex 'from' should have more than one column") + } + } + vindexToCol = vindex.Params["to"] + // Make the vindex write_only. If one exists already in the vschema, + // it will need to match this vindex exactly, including the write_only setting. + vindex.Params["write_only"] = "true" + // See if we can create the vindex without errors. + if _, err := vindexes.CreateVindex(vindex.Type, vindexName, vindex.Params); err != nil { + return nil, nil, nil, nil, err + } + if ignoreNullsStr, ok := vindex.Params["ignore_nulls"]; ok { + // This mirrors the behavior of vindexes.boolFromMap(). + switch ignoreNullsStr { + case "true": + vindexIgnoreNulls = true + case "false": + vindexIgnoreNulls = false + default: + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ignore_nulls (%s) value must be 'true' or 'false'", + ignoreNullsStr) + } + } + + // Validate input table. + if len(specs.Tables) < 1 || len(specs.Tables) > 2 { + return nil, nil, nil, nil, fmt.Errorf("one or two tables must be specified") + } + // Loop executes once or twice. + for tableName, table := range specs.Tables { + if len(table.ColumnVindexes) != 1 { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "exactly one ColumnVindex must be specified for the %s table", + tableName) + } + if tableName != targetTableName { // This is the source table. + sourceTableName = tableName + sourceTable = table + continue + } + // This is a primary vindex definition for the target table + // which allows you to override the vindex type used. + var vindexCols []string + if len(table.ColumnVindexes[0].Columns) != 0 { + vindexCols = table.ColumnVindexes[0].Columns + } else { + if table.ColumnVindexes[0].Column == "" { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", + tableName) + } + vindexCols = []string{table.ColumnVindexes[0].Column} + } + if !slices.Equal(vindexCols, vindexFromCols) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "columns in the lookup table %s primary vindex (%s) don't match the 'from' columns specified (%s)", + tableName, strings.Join(vindexCols, ","), strings.Join(vindexFromCols, ",")) + } + } + + // Validate input table and vindex consistency. + if sourceTable == nil || len(sourceTable.ColumnVindexes) != 1 { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No ColumnVindex found for the owner table (%s) in the %s keyspace", + sourceTable, keyspace) + } + if sourceTable.ColumnVindexes[0].Name != vindexName { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ColumnVindex name (%s) must match vindex name (%s)", + sourceTable.ColumnVindexes[0].Name, vindexName) + } + if vindex.Owner != "" && vindex.Owner != sourceTableName { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex owner (%s) must match table name (%s)", + vindex.Owner, sourceTableName) + } + if len(sourceTable.ColumnVindexes[0].Columns) != 0 { + sourceVindexColumns = sourceTable.ColumnVindexes[0].Columns + } else { + if sourceTable.ColumnVindexes[0].Column == "" { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", + sourceTableName) + } + sourceVindexColumns = []string{sourceTable.ColumnVindexes[0].Column} + } + if len(sourceVindexColumns) != len(vindexFromCols) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "length of table columns (%d) differs from length of vindex columns (%d)", + len(sourceVindexColumns), len(vindexFromCols)) + } + + // Validate against source vschema. + sourceVSchema, err = w.ts.GetVSchema(ctx, keyspace) + if err != nil { + return nil, nil, nil, nil, err + } + if sourceVSchema.Vindexes == nil { + sourceVSchema.Vindexes = make(map[string]*vschemapb.Vindex) + } + // If source and target keyspaces are the same, make vschemas point + // to the same object. + if keyspace == targetKeyspace { + targetVSchema = sourceVSchema + } else { + targetVSchema, err = w.ts.GetVSchema(ctx, targetKeyspace) + if err != nil { + return nil, nil, nil, nil, err + } + } + if targetVSchema.Vindexes == nil { + targetVSchema.Vindexes = make(map[string]*vschemapb.Vindex) + } + if targetVSchema.Tables == nil { + targetVSchema.Tables = make(map[string]*vschemapb.Table) + } + if existing, ok := sourceVSchema.Vindexes[vindexName]; ok { + if !proto.Equal(existing, vindex) { // If the exact same vindex already exists then we can re-use it + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "a conflicting vindex named %s already exists in the %s keyspace", + vindexName, keyspace) + } + } + sourceVSchemaTable = sourceVSchema.Tables[sourceTableName] + if sourceVSchemaTable == nil && !schema.IsInternalOperationTableName(sourceTableName) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "table %s not found in the %s keyspace", sourceTableName, keyspace) + } + for _, colVindex := range sourceVSchemaTable.ColumnVindexes { + // For a conflict, the vindex name and column should match. + if colVindex.Name != vindexName { + continue + } + var colNames []string + if len(colVindex.Columns) == 0 { + colNames = []string{colVindex.Column} + } else { + colNames = colVindex.Columns + } + // If this is the exact same definition then we can use the existing one. If they + // are not the same then they are two distinct conflicting vindexes and we should + // not proceed. + if !slices.Equal(colNames, sourceVindexColumns) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting ColumnVindex on column(s) %s in table %s already exists in the %s keyspace", + strings.Join(colNames, ","), sourceTableName, keyspace) + } + } + + // Validate against source schema. + sourceShards, err := w.ts.GetServingShards(ctx, keyspace) + if err != nil { + return nil, nil, nil, nil, err + } + onesource := sourceShards[0] + if onesource.PrimaryAlias == nil { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "source shard %s has no primary", onesource.ShardName()) + } + req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{sourceTableName}} + tableSchema, err := schematools.GetSchema(ctx, w.ts, w.tmc, onesource.PrimaryAlias, req) + if err != nil { + return nil, nil, nil, nil, err + } + if len(tableSchema.TableDefinitions) != 1 { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected number of tables (%d) returned from %s schema", + len(tableSchema.TableDefinitions), keyspace) + } + + // Generate "create table" statement. + lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") + if len(lines) < 3 { + // Should never happen. + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "schema looks incorrect: %s, expecting at least four lines", + tableSchema.TableDefinitions[0].Schema) + } + var modified []string + modified = append(modified, strings.Replace(lines[0], sourceTableName, targetTableName, 1)) + for i := range sourceVindexColumns { + line, err := generateColDef(lines, sourceVindexColumns[i], vindexFromCols[i]) + if err != nil { + return nil, nil, nil, nil, err + } + modified = append(modified, line) + } + + if vindex.Params["data_type"] == "" || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { + modified = append(modified, fmt.Sprintf(" %s varbinary(128),", sqlescape.EscapeID(vindexToCol))) + } else { + modified = append(modified, fmt.Sprintf(" %s %s,", sqlescape.EscapeID(vindexToCol), sqlescape.EscapeID(vindex.Params["data_type"]))) + } + buf := sqlparser.NewTrackedBuffer(nil) + fmt.Fprintf(buf, " PRIMARY KEY (") + prefix := "" + for _, col := range vindexFromCols { + fmt.Fprintf(buf, "%s%s", prefix, sqlescape.EscapeID(col)) + prefix = ", " + } + fmt.Fprintf(buf, ")") + modified = append(modified, buf.String()) + modified = append(modified, ")") + createDDL = strings.Join(modified, "\n") + // Confirm that our DDL is valid before we create anything. + if _, err = w.parser.ParseStrictDDL(createDDL); err != nil { + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error: %v; invalid lookup table definition generated: %s", + err, createDDL) + } + + // Generate vreplication query. + buf = sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select ") + for i := range vindexFromCols { + buf.Myprintf("%s as %s, ", sqlparser.String(sqlparser.NewIdentifierCI(sourceVindexColumns[i])), sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) + } + if strings.EqualFold(vindexToCol, "keyspace_id") || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { + buf.Myprintf("keyspace_id() as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) + } else { + buf.Myprintf("%s as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol)), sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) + } + buf.Myprintf("from %s", sqlparser.String(sqlparser.NewIdentifierCS(sourceTableName))) + if vindexIgnoreNulls { + buf.Myprintf(" where ") + lastValIdx := len(vindexFromCols) - 1 + for i := range vindexFromCols { + buf.Myprintf("%s is not null", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) + if i != lastValIdx { + buf.Myprintf(" and ") + } + } + } + if vindex.Owner != "" { + // Only backfill. + buf.Myprintf(" group by ") + for i := range vindexFromCols { + buf.Myprintf("%s, ", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) + } + buf.Myprintf("%s", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) + } + materializeQuery = buf.String() + + // Save a copy of the original vschema if we modify it and need to provide + // a cancelFunc. + ogTargetVSchema := targetVSchema.CloneVT() + targetChanged := false + + // Update targetVSchema. + targetTable := specs.Tables[targetTableName] + if targetVSchema.Sharded { + // Choose a primary vindex type for the lookup table based on the source + // definition if one was not explicitly specified. + var targetVindexType string + var targetVindex *vschemapb.Vindex + for _, field := range tableSchema.TableDefinitions[0].Fields { + if sourceVindexColumns[0] == field.Name { + if targetTable != nil && len(targetTable.ColumnVindexes) > 0 { + targetVindexType = targetTable.ColumnVindexes[0].Name + } + if targetVindexType == "" { + targetVindexType, err = vindexes.ChooseVindexForType(field.Type) + if err != nil { + return nil, nil, nil, nil, err + } + } + targetVindex = &vschemapb.Vindex{ + Type: targetVindexType, + } + break + } + } + if targetVindex == nil { + // Unreachable. We validated column names when generating the DDL. + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "column %s not found in target schema %s", + sourceVindexColumns[0], tableSchema.TableDefinitions[0].Schema) + } + if existing, ok := targetVSchema.Vindexes[targetVindexType]; ok { + if !proto.Equal(existing, targetVindex) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting vindex named %v already exists in the %s keyspace", + targetVindexType, targetKeyspace) + } + } else { + targetVSchema.Vindexes[targetVindexType] = targetVindex + targetChanged = true + } + + targetTable = &vschemapb.Table{ + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: vindexFromCols[0], + Name: targetVindexType, + }}, + } + } else { + targetTable = &vschemapb.Table{} + } + if existing, ok := targetVSchema.Tables[targetTableName]; ok { + if !proto.Equal(existing, targetTable) { + return nil, nil, nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting table named %s already exists in the %s vschema", + targetTableName, targetKeyspace) + } + } else { + targetVSchema.Tables[targetTableName] = targetTable + targetChanged = true + } + + if targetChanged { + cancelFunc = func() error { + // Restore the original target vschema. + return w.ts.SaveVSchema(ctx, targetKeyspace, ogTargetVSchema) + } + } + + ms = &vtctldatapb.MaterializeSettings{ + Workflow: workflow, + MaterializationIntent: vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX, + SourceKeyspace: keyspace, + TargetKeyspace: targetKeyspace, + StopAfterCopy: vindex.Owner != "" && !continueAfterCopyWithOwner, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: targetTableName, + SourceExpression: materializeQuery, + CreateDdl: createDDL, + }}, + } + + // Update sourceVSchema + sourceVSchema.Vindexes[vindexName] = vindex + sourceVSchemaTable.ColumnVindexes = append(sourceVSchemaTable.ColumnVindexes, sourceTable.ColumnVindexes[0]) + + return ms, sourceVSchema, targetVSchema, cancelFunc, nil +} + +func generateColDef(lines []string, sourceVindexCol, vindexFromCol string) (string, error) { + source := sqlescape.EscapeID(sourceVindexCol) + target := sqlescape.EscapeID(vindexFromCol) + + for _, line := range lines[1:] { + if strings.Contains(line, source) { + line = strings.Replace(line, source, target, 1) + line = strings.Replace(line, " AUTO_INCREMENT", "", 1) + line = strings.Replace(line, " DEFAULT NULL", "", 1) + // Ensure that the column definition ends with a comma as we will + // be appending the TO column and PRIMARY KEY definitions. If the + // souce column here was the last entity defined in the source + // table's definition then it will not already have the comma. + if !strings.HasSuffix(strings.TrimSpace(line), ",") { + line += "," + } + return line, nil + } + } + return "", fmt.Errorf("column %s not found in schema %v", sourceVindexCol, lines) +} diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 8b272333890..1f1fc18b691 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -715,11 +715,8 @@ func (s *Server) Materialize(ctx context.Context, ms *vtctldatapb.MaterializeSet cells[i] = strings.TrimSpace(cells[i]) } - switch { - case len(ms.ReferenceTables) == 0 && len(ms.TableSettings) == 0: - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "either --table-settings or --reference-tables must be specified") - case len(ms.ReferenceTables) > 0 && len(ms.TableSettings) > 0: - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot specify both --table-settings and --reference-tables") + if err := validateMaterializeSettings(ms); err != nil { + return err } for _, table := range ms.ReferenceTables { @@ -746,6 +743,17 @@ func (s *Server) Materialize(ctx context.Context, ms *vtctldatapb.MaterializeSet return mz.startStreams(ctx) } +func validateMaterializeSettings(ms *vtctldatapb.MaterializeSettings) error { + switch { + case len(ms.ReferenceTables) == 0 && len(ms.TableSettings) == 0: + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "either --table-settings or --reference-tables must be specified") + case len(ms.ReferenceTables) > 0 && len(ms.TableSettings) > 0: + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot specify both --table-settings and --reference-tables") + } + + return nil +} + // MoveTablesCreate is part of the vtctlservicepb.VtctldServer interface. // It passes the embedded TabletRequest object to the given keyspace's // target primary tablets that will be executing the workflow. @@ -3797,27 +3805,27 @@ func (s *Server) prepareCreateLookup(ctx context.Context, workflow, keyspace str return ms, sourceVSchema, targetVSchema, cancelFunc, nil } -func generateColDef(lines []string, sourceVindexCol, vindexFromCol string) (string, error) { - source := sqlescape.EscapeID(sourceVindexCol) - target := sqlescape.EscapeID(vindexFromCol) - - for _, line := range lines[1:] { - if strings.Contains(line, source) { - line = strings.Replace(line, source, target, 1) - line = strings.Replace(line, " AUTO_INCREMENT", "", 1) - line = strings.Replace(line, " DEFAULT NULL", "", 1) - // Ensure that the column definition ends with a comma as we will - // be appending the TO column and PRIMARY KEY definitions. If the - // souce column here was the last entity defined in the source - // table's definition then it will not already have the comma. - if !strings.HasSuffix(strings.TrimSpace(line), ",") { - line += "," - } - return line, nil - } - } - return "", fmt.Errorf("column %s not found in schema %v", sourceVindexCol, lines) -} +// func generateColDef(lines []string, sourceVindexCol, vindexFromCol string) (string, error) { +// source := sqlescape.EscapeID(sourceVindexCol) +// target := sqlescape.EscapeID(vindexFromCol) + +// for _, line := range lines[1:] { +// if strings.Contains(line, source) { +// line = strings.Replace(line, source, target, 1) +// line = strings.Replace(line, " AUTO_INCREMENT", "", 1) +// line = strings.Replace(line, " DEFAULT NULL", "", 1) +// // Ensure that the column definition ends with a comma as we will +// // be appending the TO column and PRIMARY KEY definitions. If the +// // souce column here was the last entity defined in the source +// // table's definition then it will not already have the comma. +// if !strings.HasSuffix(strings.TrimSpace(line), ",") { +// line += "," +// } +// return line, nil +// } +// } +// return "", fmt.Errorf("column %s not found in schema %v", sourceVindexCol, lines) +// } func (s *Server) MigrateCreate(ctx context.Context, req *vtctldatapb.MigrateCreateRequest) (*vtctldatapb.WorkflowStatusResponse, error) { moveTablesCreateRequest := &vtctldatapb.MoveTablesCreateRequest{ From d914e1c81cb6879e214bcff41becb2b047086243 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Sat, 16 Nov 2024 16:29:17 +0530 Subject: [PATCH 08/12] refac: Split prepareCreateLookup into individual funcs Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/lookup.go | 572 ++++++++++++---------- go/vt/vtctl/workflow/materializer_test.go | 104 ++-- go/vt/vtctl/workflow/server.go | 423 +--------------- 3 files changed, 404 insertions(+), 695 deletions(-) diff --git a/go/vt/vtctl/workflow/lookup.go b/go/vt/vtctl/workflow/lookup.go index 96842b07e3f..271984504b1 100644 --- a/go/vt/vtctl/workflow/lookup.go +++ b/go/vt/vtctl/workflow/lookup.go @@ -42,19 +42,7 @@ import ( // Lookup Vindex. func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( ms *vtctldatapb.MaterializeSettings, sourceVSchema, targetVSchema *vschemapb.Keyspace, cancelFunc func() error, err error) { - // Important variables are pulled out here. var ( - vindexName string - vindex *vschemapb.Vindex - targetKeyspace string - targetTableName string - vindexFromCols []string - vindexToCol string - vindexIgnoreNulls bool - - sourceTableName string - // sourceTable is the supplied table info. - sourceTable *vschemapb.Table // sourceVSchemaTable is the table info present in the vschema. sourceVSchemaTable *vschemapb.Table // sourceVindexColumns are computed from the input sourceTable. @@ -66,179 +54,40 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s ) // Validate input vindex. - if specs == nil { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") - } - if len(specs.Vindexes) != 1 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "only one vindex must be specified") - } - vindexName = maps.Keys(specs.Vindexes)[0] - vindex = maps.Values(specs.Vindexes)[0] - if !strings.Contains(vindex.Type, "lookup") { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex %s is not a lookup type", vindex.Type) - } - targetKeyspace, targetTableName, err = w.parser.ParseTable(vindex.Params["table"]) - if err != nil || targetKeyspace == "" { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex table name (%s) must be in the form .
", vindex.Params["table"]) - } - vindexFromCols = strings.Split(vindex.Params["from"], ",") - for i, col := range vindexFromCols { - vindexFromCols[i] = strings.TrimSpace(col) - } - if strings.Contains(vindex.Type, "unique") { - if len(vindexFromCols) != 1 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unique vindex 'from' should have only one column") - } - } else { - if len(vindexFromCols) < 2 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "non-unique vindex 'from' should have more than one column") - } - } - vindexToCol = vindex.Params["to"] - // Make the vindex write_only. If one exists already in the vschema, - // it will need to match this vindex exactly, including the write_only setting. - vindex.Params["write_only"] = "true" - // See if we can create the vindex without errors. - if _, err := vindexes.CreateVindex(vindex.Type, vindexName, vindex.Params); err != nil { + vindex, vInfo, err := w.validateAndGetVindex(specs) + if err != nil { return nil, nil, nil, nil, err } - if ignoreNullsStr, ok := vindex.Params["ignore_nulls"]; ok { - // This mirrors the behavior of vindexes.boolFromMap(). - switch ignoreNullsStr { - case "true": - vindexIgnoreNulls = true - case "false": - vindexIgnoreNulls = false - default: - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ignore_nulls (%s) value must be 'true' or 'false'", - ignoreNullsStr) - } - } - // Validate input table. - if len(specs.Tables) < 1 || len(specs.Tables) > 2 { - return nil, nil, nil, nil, fmt.Errorf("one or two tables must be specified") - } - // Loop executes once or twice. - for tableName, table := range specs.Tables { - if len(table.ColumnVindexes) != 1 { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "exactly one ColumnVindex must be specified for the %s table", - tableName) - } - if tableName != targetTableName { // This is the source table. - sourceTableName = tableName - sourceTable = table - continue - } - // This is a primary vindex definition for the target table - // which allows you to override the vindex type used. - var vindexCols []string - if len(table.ColumnVindexes[0].Columns) != 0 { - vindexCols = table.ColumnVindexes[0].Columns - } else { - if table.ColumnVindexes[0].Column == "" { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", - tableName) - } - vindexCols = []string{table.ColumnVindexes[0].Column} - } - if !slices.Equal(vindexCols, vindexFromCols) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "columns in the lookup table %s primary vindex (%s) don't match the 'from' columns specified (%s)", - tableName, strings.Join(vindexCols, ","), strings.Join(vindexFromCols, ",")) - } + vInfo.sourceTable, vInfo.sourceTableName, err = getSourceTable(specs, vInfo.targetTableName, vInfo.fromCols) + if err != nil { + return nil, nil, nil, nil, err } - // Validate input table and vindex consistency. - if sourceTable == nil || len(sourceTable.ColumnVindexes) != 1 { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No ColumnVindex found for the owner table (%s) in the %s keyspace", - sourceTable, keyspace) - } - if sourceTable.ColumnVindexes[0].Name != vindexName { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ColumnVindex name (%s) must match vindex name (%s)", - sourceTable.ColumnVindexes[0].Name, vindexName) - } - if vindex.Owner != "" && vindex.Owner != sourceTableName { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex owner (%s) must match table name (%s)", - vindex.Owner, sourceTableName) - } - if len(sourceTable.ColumnVindexes[0].Columns) != 0 { - sourceVindexColumns = sourceTable.ColumnVindexes[0].Columns - } else { - if sourceTable.ColumnVindexes[0].Column == "" { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", - sourceTableName) - } - sourceVindexColumns = []string{sourceTable.ColumnVindexes[0].Column} - } - if len(sourceVindexColumns) != len(vindexFromCols) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "length of table columns (%d) differs from length of vindex columns (%d)", - len(sourceVindexColumns), len(vindexFromCols)) + sourceVindexColumns, err = validateSourceTableAndGetVindexColumns(vInfo, vindex, keyspace) + if err != nil { + return nil, nil, nil, nil, err } - // Validate against source vschema. - sourceVSchema, err = w.ts.GetVSchema(ctx, keyspace) + sourceVSchema, targetVSchema, err = w.getTargetAndSourceVSchema(ctx, keyspace, vInfo.targetKeyspace) if err != nil { return nil, nil, nil, nil, err } - if sourceVSchema.Vindexes == nil { - sourceVSchema.Vindexes = make(map[string]*vschemapb.Vindex) - } - // If source and target keyspaces are the same, make vschemas point - // to the same object. - if keyspace == targetKeyspace { - targetVSchema = sourceVSchema - } else { - targetVSchema, err = w.ts.GetVSchema(ctx, targetKeyspace) - if err != nil { - return nil, nil, nil, nil, err - } - } - if targetVSchema.Vindexes == nil { - targetVSchema.Vindexes = make(map[string]*vschemapb.Vindex) - } - if targetVSchema.Tables == nil { - targetVSchema.Tables = make(map[string]*vschemapb.Table) - } - if existing, ok := sourceVSchema.Vindexes[vindexName]; ok { + + if existing, ok := sourceVSchema.Vindexes[vInfo.name]; ok { if !proto.Equal(existing, vindex) { // If the exact same vindex already exists then we can re-use it - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "a conflicting vindex named %s already exists in the %s keyspace", - vindexName, keyspace) + return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "a conflicting vindex named %s already exists in the %s keyspace", + vInfo.name, keyspace) } } - sourceVSchemaTable = sourceVSchema.Tables[sourceTableName] - if sourceVSchemaTable == nil && !schema.IsInternalOperationTableName(sourceTableName) { + + sourceVSchemaTable = sourceVSchema.Tables[vInfo.sourceTableName] + if sourceVSchemaTable == nil && !schema.IsInternalOperationTableName(vInfo.sourceTableName) { return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "table %s not found in the %s keyspace", sourceTableName, keyspace) + vterrors.Errorf(vtrpcpb.Code_INTERNAL, "table %s not found in the %s keyspace", vInfo.sourceTableName, keyspace) } - for _, colVindex := range sourceVSchemaTable.ColumnVindexes { - // For a conflict, the vindex name and column should match. - if colVindex.Name != vindexName { - continue - } - var colNames []string - if len(colVindex.Columns) == 0 { - colNames = []string{colVindex.Column} - } else { - colNames = colVindex.Columns - } - // If this is the exact same definition then we can use the existing one. If they - // are not the same then they are two distinct conflicting vindexes and we should - // not proceed. - if !slices.Equal(colNames, sourceVindexColumns) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting ColumnVindex on column(s) %s in table %s already exists in the %s keyspace", - strings.Join(colNames, ","), sourceTableName, keyspace) - } + if err := validateNonConflictingColumnVindex(sourceVSchemaTable, vInfo, sourceVindexColumns, keyspace); err != nil { + return nil, nil, nil, nil, err } // Validate against source schema. @@ -251,7 +100,8 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "source shard %s has no primary", onesource.ShardName()) } - req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{sourceTableName}} + + req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{vInfo.sourceTableName}} tableSchema, err := schematools.GetSchema(ctx, w.ts, w.tmc, onesource.PrimaryAlias, req) if err != nil { return nil, nil, nil, nil, err @@ -263,76 +113,13 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s } // Generate "create table" statement. - lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") - if len(lines) < 3 { - // Should never happen. - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "schema looks incorrect: %s, expecting at least four lines", - tableSchema.TableDefinitions[0].Schema) - } - var modified []string - modified = append(modified, strings.Replace(lines[0], sourceTableName, targetTableName, 1)) - for i := range sourceVindexColumns { - line, err := generateColDef(lines, sourceVindexColumns[i], vindexFromCols[i]) - if err != nil { - return nil, nil, nil, nil, err - } - modified = append(modified, line) - } - - if vindex.Params["data_type"] == "" || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { - modified = append(modified, fmt.Sprintf(" %s varbinary(128),", sqlescape.EscapeID(vindexToCol))) - } else { - modified = append(modified, fmt.Sprintf(" %s %s,", sqlescape.EscapeID(vindexToCol), sqlescape.EscapeID(vindex.Params["data_type"]))) - } - buf := sqlparser.NewTrackedBuffer(nil) - fmt.Fprintf(buf, " PRIMARY KEY (") - prefix := "" - for _, col := range vindexFromCols { - fmt.Fprintf(buf, "%s%s", prefix, sqlescape.EscapeID(col)) - prefix = ", " - } - fmt.Fprintf(buf, ")") - modified = append(modified, buf.String()) - modified = append(modified, ")") - createDDL = strings.Join(modified, "\n") - // Confirm that our DDL is valid before we create anything. - if _, err = w.parser.ParseStrictDDL(createDDL); err != nil { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error: %v; invalid lookup table definition generated: %s", - err, createDDL) + createDDL, err = w.generateCreateDDLStatement(tableSchema, sourceVindexColumns, vInfo, vindex) + if err != nil { + return nil, nil, nil, nil, err } // Generate vreplication query. - buf = sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("select ") - for i := range vindexFromCols { - buf.Myprintf("%s as %s, ", sqlparser.String(sqlparser.NewIdentifierCI(sourceVindexColumns[i])), sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - } - if strings.EqualFold(vindexToCol, "keyspace_id") || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { - buf.Myprintf("keyspace_id() as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } else { - buf.Myprintf("%s as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol)), sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } - buf.Myprintf("from %s", sqlparser.String(sqlparser.NewIdentifierCS(sourceTableName))) - if vindexIgnoreNulls { - buf.Myprintf(" where ") - lastValIdx := len(vindexFromCols) - 1 - for i := range vindexFromCols { - buf.Myprintf("%s is not null", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - if i != lastValIdx { - buf.Myprintf(" and ") - } - } - } - if vindex.Owner != "" { - // Only backfill. - buf.Myprintf(" group by ") - for i := range vindexFromCols { - buf.Myprintf("%s, ", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - } - buf.Myprintf("%s", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } - materializeQuery = buf.String() + materializeQuery = generateMaterializeQuery(vInfo, vindex, sourceVindexColumns) // Save a copy of the original vschema if we modify it and need to provide // a cancelFunc. @@ -340,7 +127,7 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s targetChanged := false // Update targetVSchema. - targetTable := specs.Tables[targetTableName] + targetTable := specs.Tables[vInfo.targetTableName] if targetVSchema.Sharded { // Choose a primary vindex type for the lookup table based on the source // definition if one was not explicitly specified. @@ -369,11 +156,12 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s vterrors.Errorf(vtrpcpb.Code_INTERNAL, "column %s not found in target schema %s", sourceVindexColumns[0], tableSchema.TableDefinitions[0].Schema) } + if existing, ok := targetVSchema.Vindexes[targetVindexType]; ok { if !proto.Equal(existing, targetVindex) { return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting vindex named %v already exists in the %s keyspace", - targetVindexType, targetKeyspace) + targetVindexType, vInfo.targetKeyspace) } } else { targetVSchema.Vindexes[targetVindexType] = targetVindex @@ -382,28 +170,28 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s targetTable = &vschemapb.Table{ ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: vindexFromCols[0], + Column: vInfo.fromCols[0], Name: targetVindexType, }}, } } else { targetTable = &vschemapb.Table{} } - if existing, ok := targetVSchema.Tables[targetTableName]; ok { + if existing, ok := targetVSchema.Tables[vInfo.targetTableName]; ok { if !proto.Equal(existing, targetTable) { return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting table named %s already exists in the %s vschema", - targetTableName, targetKeyspace) + vInfo.targetTableName, vInfo.targetKeyspace) } } else { - targetVSchema.Tables[targetTableName] = targetTable + targetVSchema.Tables[vInfo.targetTableName] = targetTable targetChanged = true } if targetChanged { cancelFunc = func() error { // Restore the original target vschema. - return w.ts.SaveVSchema(ctx, targetKeyspace, ogTargetVSchema) + return w.ts.SaveVSchema(ctx, vInfo.targetKeyspace, ogTargetVSchema) } } @@ -411,22 +199,308 @@ func (w *workflow) prepareCreateLookup(ctx context.Context, workflow, keyspace s Workflow: workflow, MaterializationIntent: vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX, SourceKeyspace: keyspace, - TargetKeyspace: targetKeyspace, + TargetKeyspace: vInfo.targetKeyspace, StopAfterCopy: vindex.Owner != "" && !continueAfterCopyWithOwner, TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: targetTableName, + TargetTable: vInfo.targetTableName, SourceExpression: materializeQuery, CreateDdl: createDDL, }}, } // Update sourceVSchema - sourceVSchema.Vindexes[vindexName] = vindex - sourceVSchemaTable.ColumnVindexes = append(sourceVSchemaTable.ColumnVindexes, sourceTable.ColumnVindexes[0]) + sourceVSchema.Vindexes[vInfo.name] = vindex + sourceVSchemaTable.ColumnVindexes = append(sourceVSchemaTable.ColumnVindexes, vInfo.sourceTable.ColumnVindexes[0]) return ms, sourceVSchema, targetVSchema, cancelFunc, nil } +// vindexInfo holds the validated vindex configuration +type vindexInfo struct { + name string + targetKeyspace string + targetTableName string + fromCols []string + toCol string + ignoreNulls bool + + // sourceTable is the supplied table info. + sourceTable *vschemapb.Table + sourceTableName string +} + +// validateAndGetVindex validates and extracts vindex configuration +func (w *workflow) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vindex, *vindexInfo, error) { + if specs == nil { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") + } + if len(specs.Vindexes) != 1 { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "only one vindex must be specified") + } + + vindexName := maps.Keys(specs.Vindexes)[0] + vindex := maps.Values(specs.Vindexes)[0] + + if !strings.Contains(vindex.Type, "lookup") { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex %s is not a lookup type", vindex.Type) + } + + targetKeyspace, targetTableName, err := w.parser.ParseTable(vindex.Params["table"]) + if err != nil || targetKeyspace == "" { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, + "vindex table name (%s) must be in the form .
", vindex.Params["table"]) + } + + vindexFromCols := strings.Split(vindex.Params["from"], ",") + for i, col := range vindexFromCols { + vindexFromCols[i] = strings.TrimSpace(col) + } + + if strings.Contains(vindex.Type, "unique") { + if len(vindexFromCols) != 1 { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unique vindex 'from' should have only one column") + } + } else { + if len(vindexFromCols) < 2 { + return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "non-unique vindex 'from' should have more than one column") + } + } + + vindexToCol := vindex.Params["to"] + // Make the vindex write_only. If one exists already in the vschema, + // it will need to match this vindex exactly, including the write_only setting. + vindex.Params["write_only"] = "true" + + // See if we can create the vindex without errors. + if _, err := vindexes.CreateVindex(vindex.Type, vindexName, vindex.Params); err != nil { + return nil, nil, err + } + + ignoreNulls := false + if ignoreNullsStr, ok := vindex.Params["ignore_nulls"]; ok { + // This mirrors the behavior of vindexes.boolFromMap(). + switch ignoreNullsStr { + case "true": + ignoreNulls = true + case "false": + ignoreNulls = false + default: + return nil, nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ignore_nulls (%s) value must be 'true' or 'false'", + ignoreNullsStr) + } + } + + // Validate input table. + if len(specs.Tables) < 1 || len(specs.Tables) > 2 { + return nil, nil, fmt.Errorf("one or two tables must be specified") + } + + return vindex, &vindexInfo{ + name: vindexName, + targetKeyspace: targetKeyspace, + targetTableName: targetTableName, + fromCols: vindexFromCols, + toCol: vindexToCol, + ignoreNulls: ignoreNulls, + }, nil +} + +func (w *workflow) getTargetAndSourceVSchema(ctx context.Context, sourceKeyspace string, targetKeyspace string) (sourceVSchema *vschemapb.Keyspace, targetVSchema *vschemapb.Keyspace, err error) { + sourceVSchema, err = w.ts.GetVSchema(ctx, sourceKeyspace) + if err != nil { + return nil, nil, err + } + if sourceVSchema.Vindexes == nil { + sourceVSchema.Vindexes = make(map[string]*vschemapb.Vindex) + } + // If source and target keyspaces are the same, make vschemas point + // to the same object. + if sourceKeyspace == targetKeyspace { + targetVSchema = sourceVSchema + } else { + targetVSchema, err = w.ts.GetVSchema(ctx, targetKeyspace) + if err != nil { + return nil, nil, err + } + } + if targetVSchema.Vindexes == nil { + targetVSchema.Vindexes = make(map[string]*vschemapb.Vindex) + } + if targetVSchema.Tables == nil { + targetVSchema.Tables = make(map[string]*vschemapb.Table) + } + + return sourceVSchema, targetVSchema, nil +} + +func getSourceTable(specs *vschemapb.Keyspace, targetTableName string, fromCols []string) (sourceTable *vschemapb.Table, sourceTableName string, err error) { + // Loop executes once or twice. + for tableName, table := range specs.Tables { + if len(table.ColumnVindexes) != 1 { + return nil, "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "exactly one ColumnVindex must be specified for the %s table", + tableName) + } + + if tableName != targetTableName { // This is the source table. + sourceTableName = tableName + sourceTable = table + continue + } + // This is a primary vindex definition for the target table + // which allows you to override the vindex type used. + var vindexCols []string + if len(table.ColumnVindexes[0].Columns) != 0 { + vindexCols = table.ColumnVindexes[0].Columns + } else { + if table.ColumnVindexes[0].Column == "" { + return nil, "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", + tableName) + } + vindexCols = []string{table.ColumnVindexes[0].Column} + } + if !slices.Equal(vindexCols, fromCols) { + return nil, "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "columns in the lookup table %s primary vindex (%s) don't match the 'from' columns specified (%s)", + tableName, strings.Join(vindexCols, ","), strings.Join(fromCols, ",")) + } + } + return sourceTable, sourceTableName, nil +} + +func (w *workflow) generateCreateDDLStatement(tableSchema *tabletmanagerdatapb.SchemaDefinition, sourceVindexColumns []string, vInfo *vindexInfo, vindex *vschemapb.Vindex) (string, error) { + lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") + if len(lines) < 3 { + // Should never happen. + return "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "schema looks incorrect: %s, expecting at least four lines", + tableSchema.TableDefinitions[0].Schema) + } + + var modified []string + modified = append(modified, strings.Replace(lines[0], vInfo.sourceTableName, vInfo.targetTableName, 1)) + for i := range sourceVindexColumns { + line, err := generateColDef(lines, sourceVindexColumns[i], vInfo.fromCols[i]) + if err != nil { + return "", err + } + modified = append(modified, line) + } + + if vindex.Params["data_type"] == "" || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { + modified = append(modified, fmt.Sprintf(" %s varbinary(128),", sqlescape.EscapeID(vInfo.toCol))) + } else { + modified = append(modified, fmt.Sprintf(" %s %s,", sqlescape.EscapeID(vInfo.toCol), sqlescape.EscapeID(vindex.Params["data_type"]))) + } + + buf := sqlparser.NewTrackedBuffer(nil) + fmt.Fprintf(buf, " PRIMARY KEY (") + prefix := "" + for _, col := range vInfo.fromCols { + fmt.Fprintf(buf, "%s%s", prefix, sqlescape.EscapeID(col)) + prefix = ", " + } + fmt.Fprintf(buf, ")") + + modified = append(modified, buf.String()) + modified = append(modified, ")") + createDDL := strings.Join(modified, "\n") + + // Confirm that our DDL is valid before we create anything. + if _, err := w.parser.ParseStrictDDL(createDDL); err != nil { + return "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error: %v; invalid lookup table definition generated: %s", + err, createDDL) + } + + return createDDL, nil +} + +func generateMaterializeQuery(vInfo *vindexInfo, vindex *vschemapb.Vindex, sourceVindexColumns []string) string { + buf := sqlparser.NewTrackedBuffer(nil) + buf.Myprintf("select ") + for i := range vInfo.fromCols { + buf.Myprintf("%s as %s, ", sqlparser.String(sqlparser.NewIdentifierCI(sourceVindexColumns[i])), sqlparser.String(sqlparser.NewIdentifierCI(vInfo.fromCols[i]))) + } + if strings.EqualFold(vInfo.toCol, "keyspace_id") || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { + buf.Myprintf("keyspace_id() as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vInfo.toCol))) + } else { + buf.Myprintf("%s as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vInfo.toCol)), sqlparser.String(sqlparser.NewIdentifierCI(vInfo.toCol))) + } + buf.Myprintf("from %s", sqlparser.String(sqlparser.NewIdentifierCS(vInfo.sourceTableName))) + if vInfo.ignoreNulls { + buf.Myprintf(" where ") + lastValIdx := len(vInfo.fromCols) - 1 + for i := range vInfo.fromCols { + buf.Myprintf("%s is not null", sqlparser.String(sqlparser.NewIdentifierCI(vInfo.fromCols[i]))) + if i != lastValIdx { + buf.Myprintf(" and ") + } + } + } + if vindex.Owner != "" { + // Only backfill. + buf.Myprintf(" group by ") + for i := range vInfo.fromCols { + buf.Myprintf("%s, ", sqlparser.String(sqlparser.NewIdentifierCI(vInfo.fromCols[i]))) + } + buf.Myprintf("%s", sqlparser.String(sqlparser.NewIdentifierCI(vInfo.toCol))) + } + return buf.String() +} + +// validateSourceTableAndGetVindexColumns validates input table and vindex consistency, and returns sourceVindexColumns. +func validateSourceTableAndGetVindexColumns(vInfo *vindexInfo, vindex *vschemapb.Vindex, keyspace string) (sourceVindexColumns []string, err error) { + if vInfo.sourceTable == nil || len(vInfo.sourceTable.ColumnVindexes) != 1 { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No ColumnVindex found for the owner table (%s) in the %s keyspace", + vInfo.sourceTable, keyspace) + } + if vInfo.sourceTable.ColumnVindexes[0].Name != vInfo.name { + return nil, + vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ColumnVindex name (%s) must match vindex name (%s)", + vInfo.sourceTable.ColumnVindexes[0].Name, vInfo.name) + } + if vindex.Owner != "" && vindex.Owner != vInfo.sourceTableName { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex owner (%s) must match table name (%s)", + vindex.Owner, vInfo.sourceTableName) + } + if len(vInfo.sourceTable.ColumnVindexes[0].Columns) != 0 { + sourceVindexColumns = vInfo.sourceTable.ColumnVindexes[0].Columns + } else { + if vInfo.sourceTable.ColumnVindexes[0].Column == "" { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", + vInfo.sourceTableName) + } + sourceVindexColumns = []string{vInfo.sourceTable.ColumnVindexes[0].Column} + } + if len(sourceVindexColumns) != len(vInfo.fromCols) { + return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "length of table columns (%d) differs from length of vindex columns (%d)", + len(sourceVindexColumns), len(vInfo.fromCols)) + } + + return sourceVindexColumns, nil +} + +func validateNonConflictingColumnVindex(sourceVSchemaTable *vschemapb.Table, vInfo *vindexInfo, sourceVindexColumns []string, keyspace string) error { + for _, colVindex := range sourceVSchemaTable.ColumnVindexes { + // For a conflict, the vindex name and column should match. + if colVindex.Name != vInfo.name { + continue + } + var colNames []string + if len(colVindex.Columns) == 0 { + colNames = []string{colVindex.Column} + } else { + colNames = colVindex.Columns + } + // If this is the exact same definition then we can use the existing one. If they + // are not the same then they are two distinct conflicting vindexes and we should + // not proceed. + if !slices.Equal(colNames, sourceVindexColumns) { + return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting ColumnVindex on column(s) %s in table %s already exists in the %s keyspace", + strings.Join(colNames, ","), vInfo.sourceTableName, keyspace) + } + } + return nil +} + func generateColDef(lines []string, sourceVindexCol, vindexFromCol string) (string, error) { source := sqlescape.EscapeID(sourceVindexCol) target := sqlescape.EscapeID(vindexFromCol) diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go index 746c5fe2bae..bb31fdde436 100644 --- a/go/vt/vtctl/workflow/materializer_test.go +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -1515,7 +1515,13 @@ func TestCreateLookupVindexCreateDDL(t *testing.T) { setStartingVschema() }() } - outms, _, _, cancelFunc, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + outms, _, _, cancelFunc, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) if tcase.err != "" { require.Error(t, err) require.Contains(t, err.Error(), tcase.err, "prepareCreateLookup(%s) err: %v, does not contain %v", tcase.description, err, tcase.err) @@ -1763,7 +1769,13 @@ func TestCreateLookupVindexSourceVSchema(t *testing.T) { t.Fatal(err) } - _, got, _, _, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + _, got, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, tcase.out) { t.Errorf("%s: got:\n%v, want\n%v", tcase.description, got, tcase.out) @@ -1984,32 +1996,40 @@ func TestCreateLookupVindexTargetVSchema(t *testing.T) { err: "type SET is not recommended for a vindex", }} for _, tcase := range testcases { - env.tmc.schema[ms.SourceKeyspace+".t1"] = &tabletmanagerdatapb.SchemaDefinition{ - TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{ - Fields: []*querypb.Field{{ - Name: "col2", - Type: tcase.sourceFieldType, + t.Run(tcase.description, func(t *testing.T) { + env.tmc.schema[ms.SourceKeyspace+".t1"] = &tabletmanagerdatapb.SchemaDefinition{ + TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{ + Fields: []*querypb.Field{{ + Name: "col2", + Type: tcase.sourceFieldType, + }}, + Schema: sourceSchema, }}, - Schema: sourceSchema, - }}, - } - specs.Vindexes["v"].Params["table"] = fmt.Sprintf("%s.%s", ms.TargetKeyspace, tcase.targetTable) - if err := env.topoServ.SaveVSchema(ctx, ms.TargetKeyspace, tcase.targetVSchema); err != nil { - t.Fatal(err) - } + } + specs.Vindexes["v"].Params["table"] = fmt.Sprintf("%s.%s", ms.TargetKeyspace, tcase.targetTable) + if err := env.topoServ.SaveVSchema(ctx, ms.TargetKeyspace, tcase.targetVSchema); err != nil { + t.Fatal(err) + } - _, _, got, cancelFunc, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) - if tcase.err != "" { - if err == nil || !strings.Contains(err.Error(), tcase.err) { - t.Errorf("prepareCreateLookup(%s) err: %v, must contain %v", tcase.description, err, tcase.err) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), } - continue - } - require.NoError(t, err) - // withTable is a vschema that already contains the table and thus - // we don't make any vschema changes and there's nothing to cancel. - require.True(t, (cancelFunc != nil) == (tcase.targetVSchema != withTable)) - utils.MustMatch(t, tcase.out, got, tcase.description) + _, _, got, cancelFunc, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + if tcase.err != "" { + if err == nil || !strings.Contains(err.Error(), tcase.err) { + t.Errorf("prepareCreateLookup(%s) err: %v, must contain %v", tcase.description, err, tcase.err) + } + return + } + require.NoError(t, err) + // withTable is a vschema that already contains the table and thus + // we don't make any vschema changes and there's nothing to cancel. + require.True(t, (cancelFunc != nil) == (tcase.targetVSchema != withTable)) + utils.MustMatch(t, tcase.out, got, tcase.description) + }) } } @@ -2119,7 +2139,13 @@ func TestCreateLookupVindexSameKeyspace(t *testing.T) { t.Fatal(err) } - _, got, _, _, err := env.ws.prepareCreateLookup(ctx, "keyspace", ms.TargetKeyspace, specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + _, got, _, _, err := w.prepareCreateLookup(ctx, "keyspace", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("same keyspace: got:\n%v, want\n%v", got, want) @@ -2245,7 +2271,13 @@ func TestCreateCustomizedVindex(t *testing.T) { t.Fatal(err) } - _, got, _, _, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + _, got, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("customize create lookup error same: got:\n%v, want\n%v", got, want) @@ -2363,7 +2395,13 @@ func TestCreateLookupVindexIgnoreNulls(t *testing.T) { t.Fatal(err) } - ms, ks, _, _, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + ms, ks, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(wantKs, ks) { t.Errorf("unexpected keyspace value: got:\n%v, want\n%v", ks, wantKs) @@ -2443,11 +2481,17 @@ func TestStopAfterCopyFlag(t *testing.T) { t.Fatal(err) } - ms1, _, _, _, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + w := &workflow{ + ts: env.ws.ts, + tmc: env.ws.tmc, + logger: env.ws.Logger(), + parser: env.ws.SQLParser(), + } + ms1, _, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) require.Equal(t, ms1.StopAfterCopy, true) - ms2, _, _, _, err := env.ws.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, true) + ms2, _, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, true) require.NoError(t, err) require.Equal(t, ms2.StopAfterCopy, false) } diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 1f1fc18b691..d62dfada001 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -29,18 +29,15 @@ import ( "time" "github.com/google/uuid" - "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" "golang.org/x/sync/semaphore" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" "google.golang.org/protobuf/encoding/prototext" - "google.golang.org/protobuf/proto" "vitess.io/vitess/go/constants/sidecar" "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/ptr" - "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/trace" "vitess.io/vitess/go/vt/concurrency" @@ -49,7 +46,6 @@ import ( "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/mysqlctl/tmutils" - "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/topoproto" @@ -571,7 +567,13 @@ func (s *Server) LookupVindexCreate(ctx context.Context, req *vtctldatapb.Lookup span.Annotate("cells", req.Cells) span.Annotate("tablet_types", req.TabletTypes) - ms, sourceVSchema, targetVSchema, cancelFunc, err := s.prepareCreateLookup(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) + w := &workflow{ + ts: s.ts, + tmc: s.tmc, + logger: s.Logger(), + parser: s.SQLParser(), + } + ms, sourceVSchema, targetVSchema, cancelFunc, err := w.prepareCreateLookup(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) if err != nil { return nil, err } @@ -3416,417 +3418,6 @@ func fillStringTemplate(tmpl string, vars any) (string, error) { return data.String(), nil } -// prepareCreateLookup performs the preparatory steps for creating a -// Lookup Vindex. -func (s *Server) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( - ms *vtctldatapb.MaterializeSettings, sourceVSchema, targetVSchema *vschemapb.Keyspace, cancelFunc func() error, err error) { - // Important variables are pulled out here. - var ( - vindexName string - vindex *vschemapb.Vindex - targetKeyspace string - targetTableName string - vindexFromCols []string - vindexToCol string - vindexIgnoreNulls bool - - sourceTableName string - // sourceTable is the supplied table info. - sourceTable *vschemapb.Table - // sourceVSchemaTable is the table info present in the vschema. - sourceVSchemaTable *vschemapb.Table - // sourceVindexColumns are computed from the input sourceTable. - sourceVindexColumns []string - - // Target table info. - createDDL string - materializeQuery string - ) - - // Validate input vindex. - if specs == nil { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") - } - if len(specs.Vindexes) != 1 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "only one vindex must be specified") - } - vindexName = maps.Keys(specs.Vindexes)[0] - vindex = maps.Values(specs.Vindexes)[0] - if !strings.Contains(vindex.Type, "lookup") { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex %s is not a lookup type", vindex.Type) - } - targetKeyspace, targetTableName, err = s.env.Parser().ParseTable(vindex.Params["table"]) - if err != nil || targetKeyspace == "" { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex table name (%s) must be in the form .
", vindex.Params["table"]) - } - vindexFromCols = strings.Split(vindex.Params["from"], ",") - for i, col := range vindexFromCols { - vindexFromCols[i] = strings.TrimSpace(col) - } - if strings.Contains(vindex.Type, "unique") { - if len(vindexFromCols) != 1 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unique vindex 'from' should have only one column") - } - } else { - if len(vindexFromCols) < 2 { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "non-unique vindex 'from' should have more than one column") - } - } - vindexToCol = vindex.Params["to"] - // Make the vindex write_only. If one exists already in the vschema, - // it will need to match this vindex exactly, including the write_only setting. - vindex.Params["write_only"] = "true" - // See if we can create the vindex without errors. - if _, err := vindexes.CreateVindex(vindex.Type, vindexName, vindex.Params); err != nil { - return nil, nil, nil, nil, err - } - if ignoreNullsStr, ok := vindex.Params["ignore_nulls"]; ok { - // This mirrors the behavior of vindexes.boolFromMap(). - switch ignoreNullsStr { - case "true": - vindexIgnoreNulls = true - case "false": - vindexIgnoreNulls = false - default: - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ignore_nulls (%s) value must be 'true' or 'false'", - ignoreNullsStr) - } - } - - // Validate input table. - if len(specs.Tables) < 1 || len(specs.Tables) > 2 { - return nil, nil, nil, nil, fmt.Errorf("one or two tables must be specified") - } - // Loop executes once or twice. - for tableName, table := range specs.Tables { - if len(table.ColumnVindexes) != 1 { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "exactly one ColumnVindex must be specified for the %s table", - tableName) - } - if tableName != targetTableName { // This is the source table. - sourceTableName = tableName - sourceTable = table - continue - } - // This is a primary vindex definition for the target table - // which allows you to override the vindex type used. - var vindexCols []string - if len(table.ColumnVindexes[0].Columns) != 0 { - vindexCols = table.ColumnVindexes[0].Columns - } else { - if table.ColumnVindexes[0].Column == "" { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", - tableName) - } - vindexCols = []string{table.ColumnVindexes[0].Column} - } - if !slices.Equal(vindexCols, vindexFromCols) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "columns in the lookup table %s primary vindex (%s) don't match the 'from' columns specified (%s)", - tableName, strings.Join(vindexCols, ","), strings.Join(vindexFromCols, ",")) - } - } - - // Validate input table and vindex consistency. - if sourceTable == nil || len(sourceTable.ColumnVindexes) != 1 { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No ColumnVindex found for the owner table (%s) in the %s keyspace", - sourceTable, keyspace) - } - if sourceTable.ColumnVindexes[0].Name != vindexName { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ColumnVindex name (%s) must match vindex name (%s)", - sourceTable.ColumnVindexes[0].Name, vindexName) - } - if vindex.Owner != "" && vindex.Owner != sourceTableName { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex owner (%s) must match table name (%s)", - vindex.Owner, sourceTableName) - } - if len(sourceTable.ColumnVindexes[0].Columns) != 0 { - sourceVindexColumns = sourceTable.ColumnVindexes[0].Columns - } else { - if sourceTable.ColumnVindexes[0].Column == "" { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "at least one column must be specified in ColumnVindexes for the %s table", - sourceTableName) - } - sourceVindexColumns = []string{sourceTable.ColumnVindexes[0].Column} - } - if len(sourceVindexColumns) != len(vindexFromCols) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "length of table columns (%d) differs from length of vindex columns (%d)", - len(sourceVindexColumns), len(vindexFromCols)) - } - - // Validate against source vschema. - sourceVSchema, err = s.ts.GetVSchema(ctx, keyspace) - if err != nil { - return nil, nil, nil, nil, err - } - if sourceVSchema.Vindexes == nil { - sourceVSchema.Vindexes = make(map[string]*vschemapb.Vindex) - } - // If source and target keyspaces are the same, make vschemas point - // to the same object. - if keyspace == targetKeyspace { - targetVSchema = sourceVSchema - } else { - targetVSchema, err = s.ts.GetVSchema(ctx, targetKeyspace) - if err != nil { - return nil, nil, nil, nil, err - } - } - if targetVSchema.Vindexes == nil { - targetVSchema.Vindexes = make(map[string]*vschemapb.Vindex) - } - if targetVSchema.Tables == nil { - targetVSchema.Tables = make(map[string]*vschemapb.Table) - } - if existing, ok := sourceVSchema.Vindexes[vindexName]; ok { - if !proto.Equal(existing, vindex) { // If the exact same vindex already exists then we can re-use it - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "a conflicting vindex named %s already exists in the %s keyspace", - vindexName, keyspace) - } - } - sourceVSchemaTable = sourceVSchema.Tables[sourceTableName] - if sourceVSchemaTable == nil && !schema.IsInternalOperationTableName(sourceTableName) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "table %s not found in the %s keyspace", sourceTableName, keyspace) - } - for _, colVindex := range sourceVSchemaTable.ColumnVindexes { - // For a conflict, the vindex name and column should match. - if colVindex.Name != vindexName { - continue - } - var colNames []string - if len(colVindex.Columns) == 0 { - colNames = []string{colVindex.Column} - } else { - colNames = colVindex.Columns - } - // If this is the exact same definition then we can use the existing one. If they - // are not the same then they are two distinct conflicting vindexes and we should - // not proceed. - if !slices.Equal(colNames, sourceVindexColumns) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting ColumnVindex on column(s) %s in table %s already exists in the %s keyspace", - strings.Join(colNames, ","), sourceTableName, keyspace) - } - } - - // Validate against source schema. - sourceShards, err := s.ts.GetServingShards(ctx, keyspace) - if err != nil { - return nil, nil, nil, nil, err - } - onesource := sourceShards[0] - if onesource.PrimaryAlias == nil { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "source shard %s has no primary", onesource.ShardName()) - } - req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{sourceTableName}} - tableSchema, err := schematools.GetSchema(ctx, s.ts, s.tmc, onesource.PrimaryAlias, req) - if err != nil { - return nil, nil, nil, nil, err - } - if len(tableSchema.TableDefinitions) != 1 { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected number of tables (%d) returned from %s schema", - len(tableSchema.TableDefinitions), keyspace) - } - - // Generate "create table" statement. - lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") - if len(lines) < 3 { - // Should never happen. - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "schema looks incorrect: %s, expecting at least four lines", - tableSchema.TableDefinitions[0].Schema) - } - var modified []string - modified = append(modified, strings.Replace(lines[0], sourceTableName, targetTableName, 1)) - for i := range sourceVindexColumns { - line, err := generateColDef(lines, sourceVindexColumns[i], vindexFromCols[i]) - if err != nil { - return nil, nil, nil, nil, err - } - modified = append(modified, line) - } - - if vindex.Params["data_type"] == "" || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { - modified = append(modified, fmt.Sprintf(" %s varbinary(128),", sqlescape.EscapeID(vindexToCol))) - } else { - modified = append(modified, fmt.Sprintf(" %s %s,", sqlescape.EscapeID(vindexToCol), sqlescape.EscapeID(vindex.Params["data_type"]))) - } - buf := sqlparser.NewTrackedBuffer(nil) - fmt.Fprintf(buf, " PRIMARY KEY (") - prefix := "" - for _, col := range vindexFromCols { - fmt.Fprintf(buf, "%s%s", prefix, sqlescape.EscapeID(col)) - prefix = ", " - } - fmt.Fprintf(buf, ")") - modified = append(modified, buf.String()) - modified = append(modified, ")") - createDDL = strings.Join(modified, "\n") - // Confirm that our DDL is valid before we create anything. - if _, err = s.env.Parser().ParseStrictDDL(createDDL); err != nil { - return nil, nil, nil, nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error: %v; invalid lookup table definition generated: %s", - err, createDDL) - } - - // Generate vreplication query. - buf = sqlparser.NewTrackedBuffer(nil) - buf.Myprintf("select ") - for i := range vindexFromCols { - buf.Myprintf("%s as %s, ", sqlparser.String(sqlparser.NewIdentifierCI(sourceVindexColumns[i])), sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - } - if strings.EqualFold(vindexToCol, "keyspace_id") || strings.EqualFold(vindex.Type, "consistent_lookup_unique") || strings.EqualFold(vindex.Type, "consistent_lookup") { - buf.Myprintf("keyspace_id() as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } else { - buf.Myprintf("%s as %s ", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol)), sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } - buf.Myprintf("from %s", sqlparser.String(sqlparser.NewIdentifierCS(sourceTableName))) - if vindexIgnoreNulls { - buf.Myprintf(" where ") - lastValIdx := len(vindexFromCols) - 1 - for i := range vindexFromCols { - buf.Myprintf("%s is not null", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - if i != lastValIdx { - buf.Myprintf(" and ") - } - } - } - if vindex.Owner != "" { - // Only backfill. - buf.Myprintf(" group by ") - for i := range vindexFromCols { - buf.Myprintf("%s, ", sqlparser.String(sqlparser.NewIdentifierCI(vindexFromCols[i]))) - } - buf.Myprintf("%s", sqlparser.String(sqlparser.NewIdentifierCI(vindexToCol))) - } - materializeQuery = buf.String() - - // Save a copy of the original vschema if we modify it and need to provide - // a cancelFunc. - ogTargetVSchema := targetVSchema.CloneVT() - targetChanged := false - - // Update targetVSchema. - targetTable := specs.Tables[targetTableName] - if targetVSchema.Sharded { - // Choose a primary vindex type for the lookup table based on the source - // definition if one was not explicitly specified. - var targetVindexType string - var targetVindex *vschemapb.Vindex - for _, field := range tableSchema.TableDefinitions[0].Fields { - if sourceVindexColumns[0] == field.Name { - if targetTable != nil && len(targetTable.ColumnVindexes) > 0 { - targetVindexType = targetTable.ColumnVindexes[0].Name - } - if targetVindexType == "" { - targetVindexType, err = vindexes.ChooseVindexForType(field.Type) - if err != nil { - return nil, nil, nil, nil, err - } - } - targetVindex = &vschemapb.Vindex{ - Type: targetVindexType, - } - break - } - } - if targetVindex == nil { - // Unreachable. We validated column names when generating the DDL. - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INTERNAL, "column %s not found in target schema %s", - sourceVindexColumns[0], tableSchema.TableDefinitions[0].Schema) - } - if existing, ok := targetVSchema.Vindexes[targetVindexType]; ok { - if !proto.Equal(existing, targetVindex) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting vindex named %v already exists in the %s keyspace", - targetVindexType, targetKeyspace) - } - } else { - targetVSchema.Vindexes[targetVindexType] = targetVindex - targetChanged = true - } - - targetTable = &vschemapb.Table{ - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: vindexFromCols[0], - Name: targetVindexType, - }}, - } - } else { - targetTable = &vschemapb.Table{} - } - if existing, ok := targetVSchema.Tables[targetTableName]; ok { - if !proto.Equal(existing, targetTable) { - return nil, nil, nil, nil, - vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "a conflicting table named %s already exists in the %s vschema", - targetTableName, targetKeyspace) - } - } else { - targetVSchema.Tables[targetTableName] = targetTable - targetChanged = true - } - - if targetChanged { - cancelFunc = func() error { - // Restore the original target vschema. - return s.ts.SaveVSchema(ctx, targetKeyspace, ogTargetVSchema) - } - } - - ms = &vtctldatapb.MaterializeSettings{ - Workflow: workflow, - MaterializationIntent: vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX, - SourceKeyspace: keyspace, - TargetKeyspace: targetKeyspace, - StopAfterCopy: vindex.Owner != "" && !continueAfterCopyWithOwner, - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: targetTableName, - SourceExpression: materializeQuery, - CreateDdl: createDDL, - }}, - } - - // Update sourceVSchema - sourceVSchema.Vindexes[vindexName] = vindex - sourceVSchemaTable.ColumnVindexes = append(sourceVSchemaTable.ColumnVindexes, sourceTable.ColumnVindexes[0]) - - return ms, sourceVSchema, targetVSchema, cancelFunc, nil -} - -// func generateColDef(lines []string, sourceVindexCol, vindexFromCol string) (string, error) { -// source := sqlescape.EscapeID(sourceVindexCol) -// target := sqlescape.EscapeID(vindexFromCol) - -// for _, line := range lines[1:] { -// if strings.Contains(line, source) { -// line = strings.Replace(line, source, target, 1) -// line = strings.Replace(line, " AUTO_INCREMENT", "", 1) -// line = strings.Replace(line, " DEFAULT NULL", "", 1) -// // Ensure that the column definition ends with a comma as we will -// // be appending the TO column and PRIMARY KEY definitions. If the -// // souce column here was the last entity defined in the source -// // table's definition then it will not already have the comma. -// if !strings.HasSuffix(strings.TrimSpace(line), ",") { -// line += "," -// } -// return line, nil -// } -// } -// return "", fmt.Errorf("column %s not found in schema %v", sourceVindexCol, lines) -// } - func (s *Server) MigrateCreate(ctx context.Context, req *vtctldatapb.MigrateCreateRequest) (*vtctldatapb.WorkflowStatusResponse, error) { moveTablesCreateRequest := &vtctldatapb.MoveTablesCreateRequest{ Workflow: req.Workflow, From acedfdef02cd0e6e17979fe121115ab2858c5f1c Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Mon, 16 Dec 2024 12:48:56 +0530 Subject: [PATCH 09/12] refac: use lookup instead of workflowFetcher for lookup vindex actions Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/lookup.go | 40 +++++++++++++++-------- go/vt/vtctl/workflow/materializer_test.go | 30 ++++++++--------- go/vt/vtctl/workflow/server.go | 4 +-- 3 files changed, 43 insertions(+), 31 deletions(-) diff --git a/go/vt/vtctl/workflow/lookup.go b/go/vt/vtctl/workflow/lookup.go index bd4527328b1..a8b8343428b 100644 --- a/go/vt/vtctl/workflow/lookup.go +++ b/go/vt/vtctl/workflow/lookup.go @@ -26,11 +26,14 @@ import ( "google.golang.org/protobuf/proto" "vitess.io/vitess/go/sqlescape" + "vitess.io/vitess/go/vt/logutil" "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/vtctl/schematools" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/vindexes" + "vitess.io/vitess/go/vt/vttablet/tmclient" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" vschemapb "vitess.io/vitess/go/vt/proto/vschema" @@ -38,9 +41,18 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) +// lookup is responsible for performing actions related to lookup vindexes. +type lookup struct { + ts *topo.Server + tmc tmclient.TabletManagerClient + + logger logutil.Logger + parser *sqlparser.Parser +} + // prepareCreateLookup performs the preparatory steps for creating a // Lookup Vindex. -func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( +func (l *lookup) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( ms *vtctldatapb.MaterializeSettings, sourceVSchema, targetVSchema *vschemapb.Keyspace, cancelFunc func() error, err error) { var ( // sourceVSchemaTable is the table info present in the vschema. @@ -54,7 +66,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke ) // Validate input vindex. - vindex, vInfo, err := wf.validateAndGetVindex(specs) + vindex, vInfo, err := l.validateAndGetVindex(specs) if err != nil { return nil, nil, nil, nil, err } @@ -69,7 +81,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke return nil, nil, nil, nil, err } - sourceVSchema, targetVSchema, err = wf.getTargetAndSourceVSchema(ctx, keyspace, vInfo.targetKeyspace) + sourceVSchema, targetVSchema, err = l.getTargetAndSourceVSchema(ctx, keyspace, vInfo.targetKeyspace) if err != nil { return nil, nil, nil, nil, err } @@ -91,7 +103,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke } // Validate against source schema. - sourceShards, err := wf.ts.GetServingShards(ctx, keyspace) + sourceShards, err := l.ts.GetServingShards(ctx, keyspace) if err != nil { return nil, nil, nil, nil, err } @@ -102,7 +114,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke } req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{vInfo.sourceTableName}} - tableSchema, err := schematools.GetSchema(ctx, wf.ts, wf.tmc, onesource.PrimaryAlias, req) + tableSchema, err := schematools.GetSchema(ctx, l.ts, l.tmc, onesource.PrimaryAlias, req) if err != nil { return nil, nil, nil, nil, err } @@ -113,7 +125,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke } // Generate "create table" statement. - createDDL, err = wf.generateCreateDDLStatement(tableSchema, sourceVindexColumns, vInfo, vindex) + createDDL, err = l.generateCreateDDLStatement(tableSchema, sourceVindexColumns, vInfo, vindex) if err != nil { return nil, nil, nil, nil, err } @@ -191,7 +203,7 @@ func (wf *workflowFetcher) prepareCreateLookup(ctx context.Context, workflow, ke if targetChanged { cancelFunc = func() error { // Restore the original target vschema. - return wf.ts.SaveVSchema(ctx, vInfo.targetKeyspace, ogTargetVSchema) + return l.ts.SaveVSchema(ctx, vInfo.targetKeyspace, ogTargetVSchema) } } @@ -230,7 +242,7 @@ type vindexInfo struct { } // validateAndGetVindex validates and extracts vindex configuration -func (wf *workflowFetcher) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vindex, *vindexInfo, error) { +func (l *lookup) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vindex, *vindexInfo, error) { if specs == nil { return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") } @@ -245,7 +257,7 @@ func (wf *workflowFetcher) validateAndGetVindex(specs *vschemapb.Keyspace) (*vsc return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex %s is not a lookup type", vindex.Type) } - targetKeyspace, targetTableName, err := wf.parser.ParseTable(vindex.Params["table"]) + targetKeyspace, targetTableName, err := l.parser.ParseTable(vindex.Params["table"]) if err != nil || targetKeyspace == "" { return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "vindex table name (%s) must be in the form .
", vindex.Params["table"]) @@ -306,8 +318,8 @@ func (wf *workflowFetcher) validateAndGetVindex(specs *vschemapb.Keyspace) (*vsc }, nil } -func (wf *workflowFetcher) getTargetAndSourceVSchema(ctx context.Context, sourceKeyspace string, targetKeyspace string) (sourceVSchema *vschemapb.Keyspace, targetVSchema *vschemapb.Keyspace, err error) { - sourceVSchema, err = wf.ts.GetVSchema(ctx, sourceKeyspace) +func (l *lookup) getTargetAndSourceVSchema(ctx context.Context, sourceKeyspace string, targetKeyspace string) (sourceVSchema *vschemapb.Keyspace, targetVSchema *vschemapb.Keyspace, err error) { + sourceVSchema, err = l.ts.GetVSchema(ctx, sourceKeyspace) if err != nil { return nil, nil, err } @@ -319,7 +331,7 @@ func (wf *workflowFetcher) getTargetAndSourceVSchema(ctx context.Context, source if sourceKeyspace == targetKeyspace { targetVSchema = sourceVSchema } else { - targetVSchema, err = wf.ts.GetVSchema(ctx, targetKeyspace) + targetVSchema, err = l.ts.GetVSchema(ctx, targetKeyspace) if err != nil { return nil, nil, err } @@ -367,7 +379,7 @@ func getSourceTable(specs *vschemapb.Keyspace, targetTableName string, fromCols return sourceTable, sourceTableName, nil } -func (wf *workflowFetcher) generateCreateDDLStatement(tableSchema *tabletmanagerdatapb.SchemaDefinition, sourceVindexColumns []string, vInfo *vindexInfo, vindex *vschemapb.Vindex) (string, error) { +func (l *lookup) generateCreateDDLStatement(tableSchema *tabletmanagerdatapb.SchemaDefinition, sourceVindexColumns []string, vInfo *vindexInfo, vindex *vschemapb.Vindex) (string, error) { lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") if len(lines) < 3 { // Should never happen. @@ -405,7 +417,7 @@ func (wf *workflowFetcher) generateCreateDDLStatement(tableSchema *tabletmanager createDDL := strings.Join(modified, "\n") // Confirm that our DDL is valid before we create anything. - if _, err := wf.parser.ParseStrictDDL(createDDL); err != nil { + if _, err := l.parser.ParseStrictDDL(createDDL); err != nil { return "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error: %v; invalid lookup table definition generated: %s", err, createDDL) } diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go index f8e32d2ec39..7c71d79bb7b 100644 --- a/go/vt/vtctl/workflow/materializer_test.go +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -1515,13 +1515,13 @@ func TestCreateLookupVindexCreateDDL(t *testing.T) { setStartingVschema() }() } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - outms, _, _, cancelFunc, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) + outms, _, _, cancelFunc, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) if tcase.err != "" { require.Error(t, err) require.Contains(t, err.Error(), tcase.err, "prepareCreateLookup(%s) err: %v, does not contain %v", tcase.description, err, tcase.err) @@ -1769,13 +1769,13 @@ func TestCreateLookupVindexSourceVSchema(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + _, got, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, tcase.out) { t.Errorf("%s: got:\n%v, want\n%v", tcase.description, got, tcase.out) @@ -2011,13 +2011,13 @@ func TestCreateLookupVindexTargetVSchema(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, _, got, cancelFunc, err := w.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + _, _, got, cancelFunc, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) if tcase.err != "" { if err == nil || !strings.Contains(err.Error(), tcase.err) { t.Errorf("prepareCreateLookup(%s) err: %v, must contain %v", tcase.description, err, tcase.err) @@ -2139,13 +2139,13 @@ func TestCreateLookupVindexSameKeyspace(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := w.prepareCreateLookup(ctx, "keyspace", ms.TargetKeyspace, specs, false) + _, got, _, _, err := l.prepareCreateLookup(ctx, "keyspace", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("same keyspace: got:\n%v, want\n%v", got, want) @@ -2271,13 +2271,13 @@ func TestCreateCustomizedVindex(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + _, got, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("customize create lookup error same: got:\n%v, want\n%v", got, want) @@ -2395,13 +2395,13 @@ func TestCreateLookupVindexIgnoreNulls(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - ms, ks, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + ms, ks, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(wantKs, ks) { t.Errorf("unexpected keyspace value: got:\n%v, want\n%v", ks, wantKs) @@ -2481,17 +2481,17 @@ func TestStopAfterCopyFlag(t *testing.T) { t.Fatal(err) } - w := &workflowFetcher{ + l := &lookup{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - ms1, _, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + ms1, _, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) require.Equal(t, ms1.StopAfterCopy, true) - ms2, _, _, _, err := w.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, true) + ms2, _, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, true) require.NoError(t, err) require.Equal(t, ms2.StopAfterCopy, false) } diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 6f9c413e1fb..afabea11b7b 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -567,13 +567,13 @@ func (s *Server) LookupVindexCreate(ctx context.Context, req *vtctldatapb.Lookup span.Annotate("cells", req.Cells) span.Annotate("tablet_types", req.TabletTypes) - w := &workflowFetcher{ + l := &lookup{ ts: s.ts, tmc: s.tmc, logger: s.Logger(), parser: s.SQLParser(), } - ms, sourceVSchema, targetVSchema, cancelFunc, err := w.prepareCreateLookup(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) + ms, sourceVSchema, targetVSchema, cancelFunc, err := l.prepareCreateLookup(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) if err != nil { return nil, err } From 919bfe45896593f48f50eb6e73a9e5fb813caada Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Mon, 16 Dec 2024 21:19:27 +0530 Subject: [PATCH 10/12] Remove the wrong validation for non-unique lookups Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/lookup.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/go/vt/vtctl/workflow/lookup.go b/go/vt/vtctl/workflow/lookup.go index a8b8343428b..a0d5a701302 100644 --- a/go/vt/vtctl/workflow/lookup.go +++ b/go/vt/vtctl/workflow/lookup.go @@ -272,10 +272,6 @@ func (l *lookup) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vin if len(vindexFromCols) != 1 { return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "unique vindex 'from' should have only one column") } - } else { - if len(vindexFromCols) < 2 { - return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "non-unique vindex 'from' should have more than one column") - } } vindexToCol := vindex.Params["to"] From f0a04ec9264a961e5d9447f9fd7df6192019b516 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Wed, 18 Dec 2024 23:26:41 +0530 Subject: [PATCH 11/12] Rename lookup to lookupVindex for better readability Signed-off-by: Noble Mittal --- .../workflow/{lookup.go => lookup_vindex.go} | 13 ++++---- go/vt/vtctl/workflow/materializer_test.go | 30 +++++++++---------- go/vt/vtctl/workflow/server.go | 4 +-- 3 files changed, 23 insertions(+), 24 deletions(-) rename go/vt/vtctl/workflow/{lookup.go => lookup_vindex.go} (96%) diff --git a/go/vt/vtctl/workflow/lookup.go b/go/vt/vtctl/workflow/lookup_vindex.go similarity index 96% rename from go/vt/vtctl/workflow/lookup.go rename to go/vt/vtctl/workflow/lookup_vindex.go index a0d5a701302..e86fa9531de 100644 --- a/go/vt/vtctl/workflow/lookup.go +++ b/go/vt/vtctl/workflow/lookup_vindex.go @@ -42,7 +42,7 @@ import ( ) // lookup is responsible for performing actions related to lookup vindexes. -type lookup struct { +type lookupVindex struct { ts *topo.Server tmc tmclient.TabletManagerClient @@ -50,9 +50,8 @@ type lookup struct { parser *sqlparser.Parser } -// prepareCreateLookup performs the preparatory steps for creating a -// Lookup Vindex. -func (l *lookup) prepareCreateLookup(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( +// prepareCreate performs the preparatory steps for creating a Lookup Vindex. +func (l *lookupVindex) prepareCreate(ctx context.Context, workflow, keyspace string, specs *vschemapb.Keyspace, continueAfterCopyWithOwner bool) ( ms *vtctldatapb.MaterializeSettings, sourceVSchema, targetVSchema *vschemapb.Keyspace, cancelFunc func() error, err error) { var ( // sourceVSchemaTable is the table info present in the vschema. @@ -242,7 +241,7 @@ type vindexInfo struct { } // validateAndGetVindex validates and extracts vindex configuration -func (l *lookup) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vindex, *vindexInfo, error) { +func (l *lookupVindex) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vindex, *vindexInfo, error) { if specs == nil { return nil, nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no vindex provided") } @@ -314,7 +313,7 @@ func (l *lookup) validateAndGetVindex(specs *vschemapb.Keyspace) (*vschemapb.Vin }, nil } -func (l *lookup) getTargetAndSourceVSchema(ctx context.Context, sourceKeyspace string, targetKeyspace string) (sourceVSchema *vschemapb.Keyspace, targetVSchema *vschemapb.Keyspace, err error) { +func (l *lookupVindex) getTargetAndSourceVSchema(ctx context.Context, sourceKeyspace string, targetKeyspace string) (sourceVSchema *vschemapb.Keyspace, targetVSchema *vschemapb.Keyspace, err error) { sourceVSchema, err = l.ts.GetVSchema(ctx, sourceKeyspace) if err != nil { return nil, nil, err @@ -375,7 +374,7 @@ func getSourceTable(specs *vschemapb.Keyspace, targetTableName string, fromCols return sourceTable, sourceTableName, nil } -func (l *lookup) generateCreateDDLStatement(tableSchema *tabletmanagerdatapb.SchemaDefinition, sourceVindexColumns []string, vInfo *vindexInfo, vindex *vschemapb.Vindex) (string, error) { +func (l *lookupVindex) generateCreateDDLStatement(tableSchema *tabletmanagerdatapb.SchemaDefinition, sourceVindexColumns []string, vInfo *vindexInfo, vindex *vschemapb.Vindex) (string, error) { lines := strings.Split(tableSchema.TableDefinitions[0].Schema, "\n") if len(lines) < 3 { // Should never happen. diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go index 7c71d79bb7b..2b33bfc7afe 100644 --- a/go/vt/vtctl/workflow/materializer_test.go +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -1515,13 +1515,13 @@ func TestCreateLookupVindexCreateDDL(t *testing.T) { setStartingVschema() }() } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - outms, _, _, cancelFunc, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) + outms, _, _, cancelFunc, err := lv.prepareCreate(ctx, "workflow", ms.SourceKeyspace, tcase.specs, false) if tcase.err != "" { require.Error(t, err) require.Contains(t, err.Error(), tcase.err, "prepareCreateLookup(%s) err: %v, does not contain %v", tcase.description, err, tcase.err) @@ -1769,13 +1769,13 @@ func TestCreateLookupVindexSourceVSchema(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + _, got, _, _, err := lv.prepareCreate(ctx, "workflow", ms.SourceKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, tcase.out) { t.Errorf("%s: got:\n%v, want\n%v", tcase.description, got, tcase.out) @@ -2011,13 +2011,13 @@ func TestCreateLookupVindexTargetVSchema(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, _, got, cancelFunc, err := l.prepareCreateLookup(ctx, "workflow", ms.SourceKeyspace, specs, false) + _, _, got, cancelFunc, err := lv.prepareCreate(ctx, "workflow", ms.SourceKeyspace, specs, false) if tcase.err != "" { if err == nil || !strings.Contains(err.Error(), tcase.err) { t.Errorf("prepareCreateLookup(%s) err: %v, must contain %v", tcase.description, err, tcase.err) @@ -2139,13 +2139,13 @@ func TestCreateLookupVindexSameKeyspace(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := l.prepareCreateLookup(ctx, "keyspace", ms.TargetKeyspace, specs, false) + _, got, _, _, err := lv.prepareCreate(ctx, "keyspace", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("same keyspace: got:\n%v, want\n%v", got, want) @@ -2271,13 +2271,13 @@ func TestCreateCustomizedVindex(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - _, got, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + _, got, _, _, err := lv.prepareCreate(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(got, want) { t.Errorf("customize create lookup error same: got:\n%v, want\n%v", got, want) @@ -2395,13 +2395,13 @@ func TestCreateLookupVindexIgnoreNulls(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - ms, ks, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + ms, ks, _, _, err := lv.prepareCreate(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) if !proto.Equal(wantKs, ks) { t.Errorf("unexpected keyspace value: got:\n%v, want\n%v", ks, wantKs) @@ -2481,17 +2481,17 @@ func TestStopAfterCopyFlag(t *testing.T) { t.Fatal(err) } - l := &lookup{ + lv := &lookupVindex{ ts: env.ws.ts, tmc: env.ws.tmc, logger: env.ws.Logger(), parser: env.ws.SQLParser(), } - ms1, _, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, false) + ms1, _, _, _, err := lv.prepareCreate(ctx, "workflow", ms.TargetKeyspace, specs, false) require.NoError(t, err) require.Equal(t, ms1.StopAfterCopy, true) - ms2, _, _, _, err := l.prepareCreateLookup(ctx, "workflow", ms.TargetKeyspace, specs, true) + ms2, _, _, _, err := lv.prepareCreate(ctx, "workflow", ms.TargetKeyspace, specs, true) require.NoError(t, err) require.Equal(t, ms2.StopAfterCopy, false) } diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index afabea11b7b..badc4c5917e 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -567,13 +567,13 @@ func (s *Server) LookupVindexCreate(ctx context.Context, req *vtctldatapb.Lookup span.Annotate("cells", req.Cells) span.Annotate("tablet_types", req.TabletTypes) - l := &lookup{ + lv := &lookupVindex{ ts: s.ts, tmc: s.tmc, logger: s.Logger(), parser: s.SQLParser(), } - ms, sourceVSchema, targetVSchema, cancelFunc, err := l.prepareCreateLookup(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) + ms, sourceVSchema, targetVSchema, cancelFunc, err := lv.prepareCreate(ctx, req.Workflow, req.Keyspace, req.Vindex, req.ContinueAfterCopyWithOwner) if err != nil { return nil, err } From 4e8a53929d8f56309594bd18fabb98fa821a2e50 Mon Sep 17 00:00:00 2001 From: Noble Mittal Date: Thu, 19 Dec 2024 09:28:37 +0530 Subject: [PATCH 12/12] Update the lookupVindex comment Signed-off-by: Noble Mittal --- go/vt/vtctl/workflow/lookup_vindex.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vtctl/workflow/lookup_vindex.go b/go/vt/vtctl/workflow/lookup_vindex.go index e86fa9531de..d1de6e23bda 100644 --- a/go/vt/vtctl/workflow/lookup_vindex.go +++ b/go/vt/vtctl/workflow/lookup_vindex.go @@ -41,7 +41,7 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) -// lookup is responsible for performing actions related to lookup vindexes. +// lookupVindex is responsible for performing actions related to lookup vindexes. type lookupVindex struct { ts *topo.Server tmc tmclient.TabletManagerClient