is primary
+ rrTabletType = topodatapb.TabletType_PRIMARY.String()
+ }
+ tabletType, err := topoproto.ParseTabletType(rrTabletType)
require.NoError(t, err)
- switch {
- case keyspace == tc.sourceKeyspace && tc.req.Direction == int32(DirectionForward):
- require.True(t, hasDeniedTableEntry(si))
- case keyspace == tc.sourceKeyspace && tc.req.Direction == int32(DirectionBackward):
- require.False(t, hasDeniedTableEntry(si))
- case keyspace == tc.targetKeyspace && tc.req.Direction == int32(DirectionForward):
- require.False(t, hasDeniedTableEntry(si))
- case keyspace == tc.targetKeyspace && tc.req.Direction == int32(DirectionBackward):
- require.True(t, hasDeniedTableEntry(si))
+
+ var to string
+ if slices.Contains(tc.req.TabletTypes, tabletType) {
+ to = fmt.Sprintf("%s.%s", tc.targetKeyspace.KeyspaceName, tableName)
+ if tc.req.Direction == int32(DirectionBackward) {
+ to = fmt.Sprintf("%s.%s", tc.sourceKeyspace.KeyspaceName, tableName)
+ }
+ } else {
+ to = fmt.Sprintf("%s.%s", tc.sourceKeyspace.KeyspaceName, tableName)
+ if tc.req.Direction == int32(DirectionBackward) {
+ to = fmt.Sprintf("%s.%s", tc.targetKeyspace.KeyspaceName, tableName)
+ }
+ }
+ for _, tt := range rr.ToTables {
+ require.Equal(t, to, tt, "Additional info: tablet type: %s, rr.FromTable: %s, rr.ToTables: %v, to string: %s",
+ tabletType.String(), rr.FromTable, rr.ToTables, to)
+ }
+ }
+ }
+
+ // Confirm that we have the expected denied tables entries.
+ if slices.Contains(tc.req.TabletTypes, topodatapb.TabletType_PRIMARY) {
+ for _, keyspace := range []*testKeyspace{tc.sourceKeyspace, tc.targetKeyspace} {
+ for _, shardName := range keyspace.ShardNames {
+ si, err := env.ts.GetShard(ctx, keyspace.KeyspaceName, shardName)
+ require.NoError(t, err)
+ switch {
+ case keyspace == tc.sourceKeyspace && tc.req.Direction == int32(DirectionForward):
+ require.True(t, hasDeniedTableEntry(si))
+ case keyspace == tc.sourceKeyspace && tc.req.Direction == int32(DirectionBackward):
+ require.False(t, hasDeniedTableEntry(si))
+ case keyspace == tc.targetKeyspace && tc.req.Direction == int32(DirectionForward):
+ require.False(t, hasDeniedTableEntry(si))
+ case keyspace == tc.targetKeyspace && tc.req.Direction == int32(DirectionBackward):
+ require.True(t, hasDeniedTableEntry(si))
+ }
}
}
}
@@ -671,11 +1826,6 @@ func TestMoveTablesTrafficSwitchingDryRun(t *testing.T) {
sourceKeyspaceName := "sourceks"
targetKeyspaceName := "targetks"
vrID := 1
- tabletTypes := []topodatapb.TabletType{
- topodatapb.TabletType_PRIMARY,
- topodatapb.TabletType_REPLICA,
- topodatapb.TabletType_RDONLY,
- }
schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
table1Name: {
TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
@@ -728,7 +1878,7 @@ func TestMoveTablesTrafficSwitchingDryRun(t *testing.T) {
Keyspace: targetKeyspaceName,
Workflow: workflowName,
Direction: int32(DirectionForward),
- TabletTypes: tabletTypes,
+ TabletTypes: allTabletTypes,
DryRun: true,
},
want: []string{
@@ -769,13 +1919,13 @@ func TestMoveTablesTrafficSwitchingDryRun(t *testing.T) {
Keyspace: targetKeyspaceName,
Workflow: workflowName,
Direction: int32(DirectionBackward),
- TabletTypes: tabletTypes,
+ TabletTypes: allTabletTypes,
DryRun: true,
},
want: []string{
fmt.Sprintf("Lock keyspace %s", targetKeyspaceName),
fmt.Sprintf("Mirroring 0.00 percent of traffic from keyspace %s to keyspace %s for tablet types [REPLICA,RDONLY]", targetKeyspaceName, sourceKeyspaceName),
- fmt.Sprintf("Switch reads for tables [%s] to keyspace %s for tablet types [REPLICA,RDONLY]", tablesStr, targetKeyspaceName),
+ fmt.Sprintf("Switch reads for tables [%s] to keyspace %s for tablet types [REPLICA,RDONLY]", tablesStr, sourceKeyspaceName),
fmt.Sprintf("Routing rules for tables [%s] will be updated", tablesStr),
fmt.Sprintf("Unlock keyspace %s", targetKeyspaceName),
fmt.Sprintf("Lock keyspace %s", targetKeyspaceName),
@@ -796,6 +1946,32 @@ func TestMoveTablesTrafficSwitchingDryRun(t *testing.T) {
fmt.Sprintf("Unlock keyspace %s", targetKeyspaceName),
},
},
+ {
+ name: "backward for read-only tablets",
+ sourceKeyspace: &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"-80", "80-"},
+ },
+ targetKeyspace: &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"-80", "80-"},
+ },
+ req: &vtctldatapb.WorkflowSwitchTrafficRequest{
+ Keyspace: targetKeyspaceName,
+ Workflow: workflowName,
+ Direction: int32(DirectionBackward),
+ TabletTypes: roTabletTypes,
+ DryRun: true,
+ },
+ want: []string{
+ fmt.Sprintf("Lock keyspace %s", sourceKeyspaceName),
+ fmt.Sprintf("Mirroring 0.00 percent of traffic from keyspace %s to keyspace %s for tablet types [REPLICA,RDONLY]", sourceKeyspaceName, targetKeyspaceName),
+ fmt.Sprintf("Switch reads for tables [%s] to keyspace %s for tablet types [REPLICA,RDONLY]", tablesStr, sourceKeyspaceName),
+ fmt.Sprintf("Routing rules for tables [%s] will be updated", tablesStr),
+ fmt.Sprintf("Serving VSchema will be rebuilt for the %s keyspace", sourceKeyspaceName),
+ fmt.Sprintf("Unlock keyspace %s", sourceKeyspaceName),
+ },
+ },
}
for _, tc := range testcases {
t.Run(tc.name, func(t *testing.T) {
@@ -816,13 +1992,20 @@ func TestMoveTablesTrafficSwitchingDryRun(t *testing.T) {
} else {
env.tmc.reverse.Store(true)
// Setup the routing rules as they would be after having previously done SwitchTraffic.
- env.addTableRoutingRules(t, ctx, tabletTypes, tables)
- env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.sourceKeyspace.KeyspaceName, copyTableQR)
- for i := 0; i < len(tc.targetKeyspace.ShardNames); i++ { // Per stream
- env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.targetKeyspace.KeyspaceName, journalQR)
- }
- for i := 0; i < len(tc.targetKeyspace.ShardNames); i++ { // Per stream
- env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.targetKeyspace.KeyspaceName, lockTableQR)
+ env.updateTableRoutingRules(t, ctx, tc.req.TabletTypes, tables,
+ tc.sourceKeyspace.KeyspaceName, tc.targetKeyspace.KeyspaceName, tc.targetKeyspace.KeyspaceName)
+ if !slices.Contains(tc.req.TabletTypes, topodatapb.TabletType_PRIMARY) {
+ for i := 0; i < len(tc.sourceKeyspace.ShardNames); i++ { // Per stream
+ env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.sourceKeyspace.KeyspaceName, journalQR)
+ }
+ } else {
+ env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.sourceKeyspace.KeyspaceName, copyTableQR)
+ for i := 0; i < len(tc.sourceKeyspace.ShardNames); i++ { // Per stream
+ env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.targetKeyspace.KeyspaceName, journalQR)
+ }
+ for i := 0; i < len(tc.sourceKeyspace.ShardNames); i++ { // Per stream
+ env.tmc.expectVRQueryResultOnKeyspaceTablets(tc.targetKeyspace.KeyspaceName, lockTableQR)
+ }
}
}
got, err := env.ws.WorkflowSwitchTraffic(ctx, tc.req)
@@ -855,6 +2038,15 @@ func TestMirrorTraffic(t *testing.T) {
topodatapb.TabletType_RDONLY,
}
+ initialRoutingRules := map[string][]string{
+ fmt.Sprintf("%s.%s", sourceKs, table1): {fmt.Sprintf("%s.%s", sourceKs, table1)},
+ fmt.Sprintf("%s.%s", sourceKs, table2): {fmt.Sprintf("%s.%s", sourceKs, table2)},
+ fmt.Sprintf("%s.%s@replica", sourceKs, table1): {fmt.Sprintf("%s.%s@replica", sourceKs, table1)},
+ fmt.Sprintf("%s.%s@replica", sourceKs, table2): {fmt.Sprintf("%s.%s@replica", sourceKs, table2)},
+ fmt.Sprintf("%s.%s@rdonly", sourceKs, table1): {fmt.Sprintf("%s.%s@rdonly", sourceKs, table1)},
+ fmt.Sprintf("%s.%s@rdonly", sourceKs, table2): {fmt.Sprintf("%s.%s@rdonly", sourceKs, table2)},
+ }
+
tests := []struct {
name string
@@ -942,8 +2134,8 @@ func TestMirrorTraffic(t *testing.T) {
Percent: 50.0,
},
routingRules: map[string][]string{
- fmt.Sprintf("%s.%s@rdonly", targetKs, table1): {fmt.Sprintf("%s.%s@rdonly", targetKs, table1)},
- fmt.Sprintf("%s.%s@rdonly", targetKs, table2): {fmt.Sprintf("%s.%s@rdonly", targetKs, table2)},
+ fmt.Sprintf("%s.%s@rdonly", sourceKs, table1): {fmt.Sprintf("%s.%s@rdonly", targetKs, table1)},
+ fmt.Sprintf("%s.%s@rdonly", sourceKs, table2): {fmt.Sprintf("%s.%s@rdonly", targetKs, table2)},
},
wantErr: "cannot mirror [rdonly] traffic for workflow src2target at this time: traffic for those tablet types is switched",
wantMirrorRules: make(map[string]map[string]float32),
@@ -957,8 +2149,8 @@ func TestMirrorTraffic(t *testing.T) {
Percent: 50.0,
},
routingRules: map[string][]string{
- fmt.Sprintf("%s.%s@replica", targetKs, table1): {fmt.Sprintf("%s.%s@replica", targetKs, table1)},
- fmt.Sprintf("%s.%s@replica", targetKs, table2): {fmt.Sprintf("%s.%s@replica", targetKs, table2)},
+ fmt.Sprintf("%s.%s@replica", sourceKs, table1): {fmt.Sprintf("%s.%s@replica", targetKs, table1)},
+ fmt.Sprintf("%s.%s@replica", sourceKs, table2): {fmt.Sprintf("%s.%s@replica", targetKs, table2)},
},
wantErr: "cannot mirror [replica] traffic for workflow src2target at this time: traffic for those tablet types is switched",
wantMirrorRules: make(map[string]map[string]float32),
@@ -972,8 +2164,8 @@ func TestMirrorTraffic(t *testing.T) {
Percent: 50.0,
},
routingRules: map[string][]string{
- table1: {fmt.Sprintf("%s.%s", targetKs, table1)},
- table2: {fmt.Sprintf("%s.%s", targetKs, table2)},
+ fmt.Sprintf("%s.%s", sourceKs, table1): {fmt.Sprintf("%s.%s", targetKs, table1)},
+ fmt.Sprintf("%s.%s", sourceKs, table2): {fmt.Sprintf("%s.%s", targetKs, table2)},
},
wantErr: "cannot mirror [primary] traffic for workflow src2target at this time: traffic for those tablet types is switched",
wantMirrorRules: make(map[string]map[string]float32),
@@ -1054,6 +2246,7 @@ func TestMirrorTraffic(t *testing.T) {
TabletTypes: tabletTypes,
Percent: 50.0,
},
+ routingRules: initialRoutingRules,
wantMirrorRules: map[string]map[string]float32{
fmt.Sprintf("%s.%s", sourceKs, table1): {
fmt.Sprintf("%s.%s", targetKs, table1): 50.0,
@@ -1088,6 +2281,7 @@ func TestMirrorTraffic(t *testing.T) {
TabletTypes: tabletTypes,
Percent: 50.0,
},
+ routingRules: initialRoutingRules,
wantMirrorRules: map[string]map[string]float32{
fmt.Sprintf("%s.%s", sourceKs, table1): {
fmt.Sprintf("%s.%s", targetKs, table1): 50.0,
@@ -1125,6 +2319,7 @@ func TestMirrorTraffic(t *testing.T) {
fmt.Sprintf("%s.%s", targetKs, table1): 25.0,
},
},
+ routingRules: initialRoutingRules,
req: &vtctldatapb.WorkflowMirrorTrafficRequest{
Keyspace: targetKs,
Workflow: workflow,
@@ -1245,3 +2440,122 @@ func createReadVReplicationWorkflowFunc(t *testing.T, workflowType binlogdatapb.
}, nil
}
}
+
+// Test checks that we don't include logs from non-existent streams in the result.
+// Ensures that we just skip the logs from non-existent streams and include the rest.
+func TestGetWorkflowsStreamLogs(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)
+
+ logResult := sqltypes.MakeTestResult(
+ 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",
+ )
+
+ 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{})
+ te.tmc.expectVRQuery(200, "select id from _vt.vreplication where db_name = 'vt_target_keyspace' and workflow = 'test_workflow'", &sqltypes.Result{})
+ te.tmc.expectVRQuery(200, "select id, vrepl_id, type, state, message, created_at, updated_at, `count` from _vt.vreplication_log where vrepl_id in (1) order by vrepl_id asc, id asc", logResult)
+
+ res, err := te.ws.GetWorkflows(ctx, &vtctldatapb.GetWorkflowsRequest{
+ Keyspace: targetKeyspace,
+ Workflow: workflow,
+ IncludeLogs: true,
+ })
+ require.NoError(t, err)
+
+ assert.Len(t, res.Workflows, 1)
+ assert.NotNil(t, res.Workflows[0].ShardStreams["-/cell-0000000200"])
+ assert.Len(t, res.Workflows[0].ShardStreams["-/cell-0000000200"].Streams, 1)
+
+ gotLogs := res.Workflows[0].ShardStreams["-/cell-0000000200"].Streams[0].Logs
+
+ // The non-existent stream logs shouldn't be part of the result
+ assert.Len(t, gotLogs, 1)
+ assert.Equal(t, gotLogs[0].Message, "log message")
+ 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/stream_migrator.go b/go/vt/vtctl/workflow/stream_migrator.go
index b294ba1fcd0..a700a1338dd 100644
--- a/go/vt/vtctl/workflow/stream_migrator.go
+++ b/go/vt/vtctl/workflow/stream_migrator.go
@@ -1001,7 +1001,7 @@ func (sm *StreamMigrator) createTargetStreams(ctx context.Context, tmpl []*VRepl
}
ig.AddRow(vrs.Workflow, vrs.BinlogSource, replication.EncodePosition(vrs.Position), "", "",
- vrs.WorkflowType, vrs.WorkflowSubType, vrs.DeferSecondaryKeys)
+ vrs.WorkflowType, vrs.WorkflowSubType, vrs.DeferSecondaryKeys, "")
return nil
}
diff --git a/go/vt/vtctl/workflow/stream_migrator_test.go b/go/vt/vtctl/workflow/stream_migrator_test.go
index 38ae10280f7..5e9c2a79038 100644
--- a/go/vt/vtctl/workflow/stream_migrator_test.go
+++ b/go/vt/vtctl/workflow/stream_migrator_test.go
@@ -19,17 +19,22 @@ package workflow
import (
"context"
"encoding/json"
+ "fmt"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/key"
+ "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
"vitess.io/vitess/go/vt/sqlparser"
-
+ "vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
)
@@ -347,3 +352,271 @@ func stringifyVRS(streams []*VReplicationStream) string {
b, _ := json.Marshal(converted)
return string(b)
}
+
+var testVSchema = &vschemapb.Keyspace{
+ Sharded: true,
+ Vindexes: map[string]*vschemapb.Vindex{
+ "xxhash": {
+ Type: "xxhash",
+ },
+ },
+ Tables: map[string]*vschemapb.Table{
+ "t1": {
+ ColumnVindexes: []*vschemapb.ColumnVindex{{
+ Columns: []string{"c1"},
+ Name: "xxhash",
+ }},
+ },
+ "t2": {
+ ColumnVindexes: []*vschemapb.ColumnVindex{{
+ Columns: []string{"c1"},
+ Name: "xxhash",
+ }},
+ },
+ "ref": {
+ Type: vindexes.TypeReference,
+ },
+ },
+}
+
+var (
+ commerceKeyspace = &testKeyspace{
+ KeyspaceName: "commerce",
+ ShardNames: []string{"0"},
+ }
+ customerUnshardedKeyspace = &testKeyspace{
+ KeyspaceName: "customer",
+ ShardNames: []string{"0"},
+ }
+ customerShardedKeyspace = &testKeyspace{
+ KeyspaceName: "customer",
+ ShardNames: []string{"-80", "80-"},
+ }
+)
+
+type streamMigratorEnv struct {
+ tenv *testEnv
+ ts *testTrafficSwitcher
+ sourceTabletIds []int
+ targetTabletIds []int
+}
+
+func (env *streamMigratorEnv) close() {
+ env.tenv.close()
+}
+
+func (env *streamMigratorEnv) addSourceQueries(queries []string) {
+ for _, id := range env.sourceTabletIds {
+ for _, q := range queries {
+ env.tenv.tmc.expectVRQuery(id, q, &sqltypes.Result{})
+ }
+ }
+}
+
+func (env *streamMigratorEnv) addTargetQueries(queries []string) {
+ for _, id := range env.targetTabletIds {
+ for _, q := range queries {
+ env.tenv.tmc.expectVRQuery(id, q, &sqltypes.Result{})
+ }
+ }
+}
+
+func newStreamMigratorEnv(ctx context.Context, t *testing.T, sourceKeyspace, targetKeyspace *testKeyspace) *streamMigratorEnv {
+ tenv := newTestEnv(t, ctx, "cell1", sourceKeyspace, targetKeyspace)
+ env := &streamMigratorEnv{tenv: tenv}
+
+ ksschema, err := vindexes.BuildKeyspaceSchema(testVSchema, "ks", sqlparser.NewTestParser())
+ require.NoError(t, err, "could not create test keyspace %+v", testVSchema)
+ sources := make(map[string]*MigrationSource, len(sourceKeyspace.ShardNames))
+ targets := make(map[string]*MigrationTarget, len(targetKeyspace.ShardNames))
+ for i, shard := range sourceKeyspace.ShardNames {
+ tablet := tenv.tablets[sourceKeyspace.KeyspaceName][startingSourceTabletUID+(i*tabletUIDStep)]
+ kr, _ := key.ParseShardingSpec(shard)
+ sources[shard] = &MigrationSource{
+ si: topo.NewShardInfo(sourceKeyspace.KeyspaceName, shard, &topodatapb.Shard{KeyRange: kr[0]}, nil),
+ primary: &topo.TabletInfo{
+ Tablet: tablet,
+ },
+ }
+ env.sourceTabletIds = append(env.sourceTabletIds, int(tablet.Alias.Uid))
+ }
+ for i, shard := range targetKeyspace.ShardNames {
+ tablet := tenv.tablets[targetKeyspace.KeyspaceName][startingTargetTabletUID+(i*tabletUIDStep)]
+ kr, _ := key.ParseShardingSpec(shard)
+ targets[shard] = &MigrationTarget{
+ si: topo.NewShardInfo(targetKeyspace.KeyspaceName, shard, &topodatapb.Shard{KeyRange: kr[0]}, nil),
+ primary: &topo.TabletInfo{
+ Tablet: tablet,
+ },
+ }
+ env.targetTabletIds = append(env.targetTabletIds, int(tablet.Alias.Uid))
+ }
+ ts := &testTrafficSwitcher{
+ trafficSwitcher: trafficSwitcher{
+ migrationType: binlogdatapb.MigrationType_SHARDS,
+ workflow: "wf1",
+ id: 1,
+ sources: sources,
+ targets: targets,
+ sourceKeyspace: sourceKeyspace.KeyspaceName,
+ targetKeyspace: targetKeyspace.KeyspaceName,
+ sourceKSSchema: ksschema,
+ workflowType: binlogdatapb.VReplicationWorkflowType_Reshard,
+ ws: tenv.ws,
+ },
+ sourceKeyspaceSchema: ksschema,
+ }
+ env.ts = ts
+
+ return env
+}
+
+func addMaterializeWorkflow(t *testing.T, env *streamMigratorEnv, id int32, sourceShard string) {
+ var wfs tabletmanagerdata.ReadVReplicationWorkflowsResponse
+ wfName := "wfMat1"
+ wfs.Workflows = append(wfs.Workflows, &tabletmanagerdata.ReadVReplicationWorkflowResponse{
+ Workflow: wfName,
+ WorkflowType: binlogdatapb.VReplicationWorkflowType_Materialize,
+ })
+ wfs.Workflows[0].Streams = append(wfs.Workflows[0].Streams, &tabletmanagerdata.ReadVReplicationWorkflowResponse_Stream{
+ Id: id,
+ Bls: &binlogdatapb.BinlogSource{
+ Keyspace: env.tenv.sourceKeyspace.KeyspaceName,
+ Shard: sourceShard,
+ Filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{
+ {Match: "t1", Filter: "select * from t1"},
+ },
+ },
+ },
+ Pos: position,
+ State: binlogdatapb.VReplicationWorkflowState_Running,
+ })
+ workflowKey := env.tenv.tmc.GetWorkflowKey(env.tenv.sourceKeyspace.KeyspaceName, sourceShard)
+ workflowResponses := []*tabletmanagerdata.ReadVReplicationWorkflowsResponse{
+ nil, // this is the response for getting stopped workflows
+ &wfs, &wfs, &wfs, // return the full list for subsequent GetWorkflows calls
+ }
+ for _, resp := range workflowResponses {
+ env.tenv.tmc.AddVReplicationWorkflowsResponse(workflowKey, resp)
+ }
+ queries := []string{
+ fmt.Sprintf("select distinct vrepl_id from _vt.copy_state where vrepl_id in (%d)", id),
+ fmt.Sprintf("update _vt.vreplication set state='Stopped', message='for cutover' where id in (%d)", id),
+ fmt.Sprintf("delete from _vt.vreplication where db_name='vt_%s' and workflow in ('%s')",
+ env.tenv.sourceKeyspace.KeyspaceName, wfName),
+ }
+ env.addSourceQueries(queries)
+ queries = []string{
+ fmt.Sprintf("delete from _vt.vreplication where db_name='vt_%s' and workflow in ('%s')",
+ env.tenv.sourceKeyspace.KeyspaceName, wfName),
+ }
+ env.addTargetQueries(queries)
+
+}
+
+func addReferenceWorkflow(t *testing.T, env *streamMigratorEnv, id int32, sourceShard string) {
+ var wfs tabletmanagerdata.ReadVReplicationWorkflowsResponse
+ wfName := "wfRef1"
+ wfs.Workflows = append(wfs.Workflows, &tabletmanagerdata.ReadVReplicationWorkflowResponse{
+ Workflow: wfName,
+ WorkflowType: binlogdatapb.VReplicationWorkflowType_Materialize,
+ })
+ wfs.Workflows[0].Streams = append(wfs.Workflows[0].Streams, &tabletmanagerdata.ReadVReplicationWorkflowResponse_Stream{
+ Id: id,
+ Bls: &binlogdatapb.BinlogSource{
+ Keyspace: env.tenv.sourceKeyspace.KeyspaceName,
+ Shard: sourceShard,
+ Filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{
+ {Match: "ref", Filter: "select * from ref"},
+ },
+ },
+ },
+ Pos: position,
+ State: binlogdatapb.VReplicationWorkflowState_Running,
+ })
+ workflowKey := env.tenv.tmc.GetWorkflowKey(env.tenv.sourceKeyspace.KeyspaceName, sourceShard)
+ workflowResponses := []*tabletmanagerdata.ReadVReplicationWorkflowsResponse{
+ nil, // this is the response for getting stopped workflows
+ &wfs, &wfs, &wfs, // return the full list for subsequent GetWorkflows calls
+ }
+ for _, resp := range workflowResponses {
+ env.tenv.tmc.AddVReplicationWorkflowsResponse(workflowKey, resp)
+ }
+}
+
+func TestBuildStreamMigratorOneMaterialize(t *testing.T) {
+ ctx := context.Background()
+ env := newStreamMigratorEnv(ctx, t, customerUnshardedKeyspace, customerShardedKeyspace)
+ defer env.close()
+ tmc := env.tenv.tmc
+
+ addMaterializeWorkflow(t, env, 100, "0")
+
+ // FIXME: Note: currently it is not optimal: we create two streams for each shard from all the
+ // shards even if the key ranges don't intersect. TBD
+ getInsert := func(shard string) string {
+ s := "/insert into _vt.vreplication.*"
+ s += fmt.Sprintf("shard:\"-80\".*in_keyrange.*c1.*%s.*", shard)
+ s += fmt.Sprintf("shard:\"80-\".*in_keyrange.*c1.*%s.*", shard)
+ return s
+ }
+ tmc.expectVRQuery(200, getInsert("-80"), &sqltypes.Result{})
+ tmc.expectVRQuery(210, getInsert("80-"), &sqltypes.Result{})
+
+ sm, err := BuildStreamMigrator(ctx, env.ts, false, sqlparser.NewTestParser())
+ require.NoError(t, err)
+ require.NotNil(t, sm)
+ require.NotNil(t, sm.streams)
+ require.Equal(t, 1, len(sm.streams))
+
+ workflows, err := sm.StopStreams(ctx)
+ require.NoError(t, err)
+ require.Equal(t, 1, len(workflows))
+ require.NoError(t, sm.MigrateStreams(ctx))
+ require.Len(t, sm.templates, 1)
+ env.addTargetQueries([]string{
+ fmt.Sprintf("update _vt.vreplication set state='Running' where db_name='vt_%s' and workflow in ('%s')",
+ env.tenv.sourceKeyspace.KeyspaceName, "wfMat1"),
+ })
+ require.NoError(t, StreamMigratorFinalize(ctx, env.ts, []string{"wfMat1"}))
+}
+
+func TestBuildStreamMigratorNoStreams(t *testing.T) {
+ ctx := context.Background()
+ env := newStreamMigratorEnv(ctx, t, customerUnshardedKeyspace, customerShardedKeyspace)
+ defer env.close()
+
+ sm, err := BuildStreamMigrator(ctx, env.ts, false, sqlparser.NewTestParser())
+ require.NoError(t, err)
+ require.NotNil(t, sm)
+ require.NotNil(t, sm.streams)
+ require.Equal(t, 0, len(sm.streams))
+
+ workflows, err := sm.StopStreams(ctx)
+ require.NoError(t, err)
+ require.Equal(t, 0, len(workflows))
+ require.NoError(t, sm.MigrateStreams(ctx))
+ require.Len(t, sm.templates, 0)
+}
+
+func TestBuildStreamMigratorRefStream(t *testing.T) {
+ ctx := context.Background()
+ env := newStreamMigratorEnv(ctx, t, customerUnshardedKeyspace, customerShardedKeyspace)
+ defer env.close()
+
+ addReferenceWorkflow(t, env, 100, "0")
+
+ sm, err := BuildStreamMigrator(ctx, env.ts, false, sqlparser.NewTestParser())
+ require.NoError(t, err)
+ require.NotNil(t, sm)
+ require.NotNil(t, sm.streams)
+ require.Equal(t, 0, len(sm.streams))
+
+ workflows, err := sm.StopStreams(ctx)
+ require.NoError(t, err)
+ require.Equal(t, 0, len(workflows))
+ require.NoError(t, sm.MigrateStreams(ctx))
+ require.Len(t, sm.templates, 0)
+}
diff --git a/go/vt/vtctl/workflow/traffic_switcher.go b/go/vt/vtctl/workflow/traffic_switcher.go
index bcc42d13ce9..4fc34992b0f 100644
--- a/go/vt/vtctl/workflow/traffic_switcher.go
+++ b/go/vt/vtctl/workflow/traffic_switcher.go
@@ -30,6 +30,7 @@ import (
"golang.org/x/sync/errgroup"
"vitess.io/vitess/go/json2"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqlescape"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
@@ -37,6 +38,7 @@ import (
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logutil"
+ "vitess.io/vitess/go/vt/mysqlctl/tmutils"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
@@ -75,9 +77,10 @@ const (
// Use pt-osc's naming convention, this format also ensures vstreamer ignores such tables.
renameTableTemplate = "_%.59s_old" // limit table name to 64 characters
- sqlDeleteWorkflow = "delete from _vt.vreplication where db_name = %s and workflow = %s"
- sqlGetMaxSequenceVal = "select max(%a) as maxval from %a.%a"
- sqlInitSequenceTable = "insert into %a.%a (id, next_id, cache) values (0, %d, 1000) on duplicate key update next_id = if(next_id < %d, %d, next_id)"
+ sqlDeleteWorkflow = "delete from _vt.vreplication where db_name = %s and workflow = %s"
+ sqlGetMaxSequenceVal = "select max(%a) as maxval from %a.%a"
+ sqlInitSequenceTable = "insert into %a.%a (id, next_id, cache) values (0, %d, 1000) on duplicate key update next_id = if(next_id < %d, %d, next_id)"
+ sqlCreateSequenceTable = "create table if not exists %a (id int, next_id bigint, cache bigint, primary key(id)) comment 'vitess_sequence'"
)
// accessType specifies the type of access for a shard (allow/disallow writes).
@@ -222,10 +225,13 @@ type trafficSwitcher struct {
logger logutil.Logger
migrationType binlogdatapb.MigrationType
- isPartialMigration bool
+ isPartialMigration bool // Is this on a subset of shards
workflow string
- // if frozen is true, the rest of the fields are not set.
+ // Should we continue if we encounter some potentially non-fatal errors such
+ // as partial tablet refreshes?
+ force bool
+ // If frozen is true, the rest of the fields are not set.
frozen bool
reverseWorkflow string
id int64
@@ -251,7 +257,7 @@ func (ts *trafficSwitcher) TopoServer() *topo.Server {
func (ts *trafficSwitcher) TabletManagerClient() tmclient.TabletManagerClient { return ts.ws.tmc }
func (ts *trafficSwitcher) Logger() logutil.Logger {
if ts.logger == nil {
- ts.logger = logutil.NewConsoleLogger()
+ ts.logger = logutil.NewConsoleLogger() // Use the default system logger
}
return ts.logger
}
@@ -289,7 +295,7 @@ func (ts *trafficSwitcher) ForAllSources(f func(source *MigrationSource) error)
return allErrors.AggrError(vterrors.Aggregate)
}
-func (ts *trafficSwitcher) ForAllTargets(f func(source *MigrationTarget) error) error {
+func (ts *trafficSwitcher) ForAllTargets(f func(target *MigrationTarget) error) error {
var wg sync.WaitGroup
allErrors := &concurrency.AllErrorRecorder{}
for _, target := range ts.targets {
@@ -435,7 +441,7 @@ func (ts *trafficSwitcher) deleteShardRoutingRules(ctx context.Context) error {
srr, err := topotools.GetShardRoutingRules(ctx, ts.TopoServer())
if err != nil {
if topo.IsErrType(err, topo.NoNode) {
- log.Warningf("No shard routing rules found when attempting to delete the ones for the %s keyspace", ts.targetKeyspace)
+ ts.Logger().Warningf("No shard routing rules found when attempting to delete the ones for the %s keyspace", ts.targetKeyspace)
return nil
}
return err
@@ -453,7 +459,7 @@ func (ts *trafficSwitcher) deleteKeyspaceRoutingRules(ctx context.Context) error
if !ts.IsMultiTenantMigration() {
return nil
}
- log.Infof("deleteKeyspaceRoutingRules: workflow %s.%s", ts.targetKeyspace, ts.workflow)
+ ts.Logger().Infof("deleteKeyspaceRoutingRules: workflow %s.%s", ts.targetKeyspace, ts.workflow)
reason := fmt.Sprintf("Deleting rules for %s", ts.SourceKeyspaceName())
return topotools.UpdateKeyspaceRoutingRules(ctx, ts.TopoServer(), reason,
func(ctx context.Context, rules *map[string]string) error {
@@ -473,7 +479,17 @@ func (ts *trafficSwitcher) dropSourceDeniedTables(ctx context.Context) error {
}
rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout)
defer cancel()
- _, _, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), source.GetShard(), nil, ts.Logger())
+ isPartial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), source.GetShard(), nil, ts.Logger())
+ if isPartial {
+ msg := fmt.Sprintf("failed to successfully refresh all tablets in the %s/%s source shard (%v):\n %v",
+ source.GetShard().Keyspace(), source.GetShard().ShardName(), err, partialDetails)
+ if ts.force {
+ log.Warning(msg)
+ return nil
+ } else {
+ return errors.New(msg)
+ }
+ }
return err
})
}
@@ -487,7 +503,17 @@ func (ts *trafficSwitcher) dropTargetDeniedTables(ctx context.Context) error {
}
rtbsCtx, cancel := context.WithTimeout(ctx, shardTabletRefreshTimeout)
defer cancel()
- _, _, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), target.GetShard(), nil, ts.Logger())
+ isPartial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), target.GetShard(), nil, ts.Logger())
+ if isPartial {
+ msg := fmt.Sprintf("failed to successfully refresh all tablets in the %s/%s target shard (%v):\n %v",
+ target.GetShard().Keyspace(), target.GetShard().ShardName(), err, partialDetails)
+ if ts.force {
+ log.Warning(msg)
+ return nil
+ } else {
+ return errors.New(msg)
+ }
+ }
return err
})
}
@@ -581,20 +607,28 @@ func (ts *trafficSwitcher) dropSourceShards(ctx context.Context) error {
}
func (ts *trafficSwitcher) switchShardReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, direction TrafficSwitchDirection) error {
+ ts.Logger().Infof("switchShardReads: workflow: %s, direction: %s, cells: %v, tablet types: %v",
+ ts.workflow, direction.String(), cells, servedTypes)
+
+ var fromShards, toShards []*topo.ShardInfo
+ if direction == DirectionForward {
+ fromShards, toShards = ts.SourceShards(), ts.TargetShards()
+ } else {
+ fromShards, toShards = ts.TargetShards(), ts.SourceShards()
+ }
+
cellsStr := strings.Join(cells, ",")
- log.Infof("switchShardReads: cells: %s, tablet types: %+v, direction %d", cellsStr, servedTypes, direction)
- fromShards, toShards := ts.SourceShards(), ts.TargetShards()
if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), cellsStr); err != nil {
err2 := vterrors.Wrapf(err, "Before switching shard reads, found SrvKeyspace for %s is corrupt in cell %s",
ts.TargetKeyspaceName(), cellsStr)
- log.Errorf("%w", err2)
+ ts.Logger().Errorf("%w", err2)
return err2
}
for _, servedType := range servedTypes {
- if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), fromShards, cells, servedType, true /* isFrom */, false /* clearSourceShards */, ts.logger); err != nil {
+ if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), fromShards, cells, servedType, true /* isFrom */, false /* clearSourceShards */, ts.Logger()); err != nil {
return err
}
- if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), toShards, cells, servedType, false, false, ts.logger); err != nil {
+ if err := ts.ws.updateShardRecords(ctx, ts.SourceKeyspaceName(), toShards, cells, servedType, false, false, ts.Logger()); err != nil {
return err
}
err := ts.TopoServer().MigrateServedType(ctx, ts.SourceKeyspaceName(), toShards, fromShards, servedType, cells)
@@ -605,14 +639,16 @@ func (ts *trafficSwitcher) switchShardReads(ctx context.Context, cells []string,
if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), cellsStr); err != nil {
err2 := vterrors.Wrapf(err, "after switching shard reads, found SrvKeyspace for %s is corrupt in cell %s",
ts.TargetKeyspaceName(), cellsStr)
- log.Errorf("%w", err2)
+ ts.Logger().Errorf("%w", err2)
return err2
}
return nil
}
func (ts *trafficSwitcher) switchTableReads(ctx context.Context, cells []string, servedTypes []topodatapb.TabletType, rebuildSrvVSchema bool, direction TrafficSwitchDirection) error {
- log.Infof("switchTableReads: cells: %s, tablet types: %+v, direction: %s", strings.Join(cells, ","), servedTypes, direction)
+ ts.Logger().Infof("switchTableReads: workflow: %s, direction: %s, cells: %v, tablet types: %v",
+ ts.workflow, direction.String(), cells, servedTypes)
+
rules, err := topotools.GetRoutingRules(ctx, ts.TopoServer())
if err != nil {
return err
@@ -626,13 +662,19 @@ func (ts *trafficSwitcher) switchTableReads(ctx context.Context, cells []string,
if servedType != topodatapb.TabletType_REPLICA && servedType != topodatapb.TabletType_RDONLY {
return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid tablet type specified when switching reads: %v", servedType)
}
-
tt := strings.ToLower(servedType.String())
for _, table := range ts.Tables() {
- toTarget := []string{ts.TargetKeyspaceName() + "." + table}
- rules[table+"@"+tt] = toTarget
- rules[ts.TargetKeyspaceName()+"."+table+"@"+tt] = toTarget
- rules[ts.SourceKeyspaceName()+"."+table+"@"+tt] = toTarget
+ if direction == DirectionForward {
+ toTarget := []string{ts.TargetKeyspaceName() + "." + table}
+ rules[table+"@"+tt] = toTarget
+ rules[ts.TargetKeyspaceName()+"."+table+"@"+tt] = toTarget
+ rules[ts.SourceKeyspaceName()+"."+table+"@"+tt] = toTarget
+ } else {
+ toSource := []string{ts.SourceKeyspaceName() + "." + table}
+ rules[table+"@"+tt] = toSource
+ rules[ts.TargetKeyspaceName()+"."+table+"@"+tt] = toSource
+ rules[ts.SourceKeyspaceName()+"."+table+"@"+tt] = toSource
+ }
}
}
if err := topotools.SaveRoutingRules(ctx, ts.TopoServer(), rules); err != nil {
@@ -654,7 +696,7 @@ func (ts *trafficSwitcher) startReverseVReplication(ctx context.Context) error {
}
func (ts *trafficSwitcher) createJournals(ctx context.Context, sourceWorkflows []string) error {
- log.Infof("In createJournals for source workflows %+v", sourceWorkflows)
+ ts.Logger().Infof("In createJournals for source workflows %+v", sourceWorkflows)
return ts.ForAllSources(func(source *MigrationSource) error {
if source.Journaled {
return nil
@@ -691,7 +733,6 @@ func (ts *trafficSwitcher) createJournals(ctx context.Context, sourceWorkflows [
})
}
- log.Infof("Creating journal %v", journal)
ts.Logger().Infof("Creating journal: %v", journal)
statement := fmt.Sprintf("insert into _vt.resharding_journal "+
"(id, db_name, val) "+
@@ -708,7 +749,7 @@ func (ts *trafficSwitcher) changeShardsAccess(ctx context.Context, keyspace stri
if err := ts.TopoServer().UpdateDisableQueryService(ctx, keyspace, shards, topodatapb.TabletType_PRIMARY, nil, access == disallowWrites /* disable */); err != nil {
return err
}
- return ts.ws.refreshPrimaryTablets(ctx, shards)
+ return ts.ws.refreshPrimaryTablets(ctx, shards, ts.force)
}
func (ts *trafficSwitcher) allowTargetWrites(ctx context.Context) error {
@@ -772,7 +813,7 @@ func (ts *trafficSwitcher) changeWriteRoute(ctx context.Context) error {
func (ts *trafficSwitcher) changeShardRouting(ctx context.Context) error {
if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), ""); err != nil {
err2 := vterrors.Wrapf(err, "Before changing shard routes, found SrvKeyspace for %s is corrupt", ts.TargetKeyspaceName())
- log.Errorf("%w", err2)
+ ts.Logger().Errorf("%w", err2)
return err2
}
err := ts.ForAllSources(func(source *MigrationSource) error {
@@ -801,7 +842,7 @@ func (ts *trafficSwitcher) changeShardRouting(ctx context.Context) error {
}
if err := ts.TopoServer().ValidateSrvKeyspace(ctx, ts.TargetKeyspaceName(), ""); err != nil {
err2 := vterrors.Wrapf(err, "after changing shard routes, found SrvKeyspace for %s is corrupt", ts.TargetKeyspaceName())
- log.Errorf("%w", err2)
+ ts.Logger().Errorf("%w", err2)
return err2
}
return nil
@@ -921,7 +962,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error
Filter: filter,
})
}
- log.Infof("Creating reverse workflow vreplication stream on tablet %s: workflow %s, startPos %s",
+ ts.Logger().Infof("Creating reverse workflow vreplication stream on tablet %s: workflow %s, startPos %s",
source.GetPrimary().GetAlias(), ts.ReverseWorkflowName(), target.Position)
_, err = ts.VReplicationExec(ctx, source.GetPrimary().GetAlias(),
binlogplayer.CreateVReplicationState(ts.ReverseWorkflowName(), reverseBls, target.Position,
@@ -938,7 +979,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error
updateQuery := ts.getReverseVReplicationUpdateQuery(target.GetPrimary().GetAlias().GetCell(),
source.GetPrimary().GetAlias().GetCell(), source.GetPrimary().DbName(), string(optionsJSON))
if updateQuery != "" {
- log.Infof("Updating vreplication stream entry on %s with: %s", source.GetPrimary().GetAlias(), updateQuery)
+ ts.Logger().Infof("Updating vreplication stream entry on %s with: %s", source.GetPrimary().GetAlias(), updateQuery)
_, err = ts.VReplicationExec(ctx, source.GetPrimary().GetAlias(), updateQuery)
return err
}
@@ -949,15 +990,7 @@ func (ts *trafficSwitcher) createReverseVReplication(ctx context.Context) error
func (ts *trafficSwitcher) addTenantFilter(ctx context.Context, filter string) (string, error) {
parser := ts.ws.env.Parser()
- vschema, err := ts.TopoServer().GetVSchema(ctx, ts.targetKeyspace)
- if err != nil {
- return "", err
- }
- targetVSchema, err := vindexes.BuildKeyspaceSchema(vschema, ts.targetKeyspace, parser)
- if err != nil {
- return "", err
- }
- tenantClause, err := getTenantClause(ts.options, targetVSchema, parser)
+ tenantClause, err := ts.buildTenantPredicate(ctx)
if err != nil {
return "", err
}
@@ -974,6 +1007,23 @@ func (ts *trafficSwitcher) addTenantFilter(ctx context.Context, filter string) (
return filter, nil
}
+func (ts *trafficSwitcher) buildTenantPredicate(ctx context.Context) (*sqlparser.Expr, error) {
+ parser := ts.ws.env.Parser()
+ vschema, err := ts.TopoServer().GetVSchema(ctx, ts.targetKeyspace)
+ if err != nil {
+ return nil, err
+ }
+ targetVSchema, err := vindexes.BuildKeyspaceSchema(vschema, ts.targetKeyspace, parser)
+ if err != nil {
+ return nil, err
+ }
+ tenantPredicate, err := getTenantClause(ts.options, targetVSchema, parser)
+ if err != nil {
+ return nil, err
+ }
+ return tenantPredicate, nil
+}
+
func (ts *trafficSwitcher) waitForCatchup(ctx context.Context, filteredReplicationWaitTime time.Duration) error {
ctx, cancel := context.WithTimeout(ctx, filteredReplicationWaitTime)
defer cancel()
@@ -988,12 +1038,12 @@ func (ts *trafficSwitcher) waitForCatchup(ctx context.Context, filteredReplicati
if err := ts.TabletManagerClient().VReplicationWaitForPos(ctx, target.GetPrimary().Tablet, uid, source.Position); err != nil {
return err
}
- log.Infof("After catchup: target keyspace:shard: %v:%v, source position %v, uid %d",
+ ts.Logger().Infof("After catchup: target keyspace:shard: %v:%v, source position %v, uid %d",
ts.TargetKeyspaceName(), target.GetShard().ShardName(), source.Position, uid)
ts.Logger().Infof("After catchup: position for keyspace:shard: %v:%v reached, uid %d",
ts.TargetKeyspaceName(), target.GetShard().ShardName(), uid)
if _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, binlogplayer.StopVReplication(uid, "stopped for cutover")); err != nil {
- log.Infof("Error marking stopped for cutover on %s, uid %d", topoproto.TabletAliasString(target.GetPrimary().GetAlias()), uid)
+ ts.Logger().Infof("Error marking stopped for cutover on %s, uid %d", topoproto.TabletAliasString(target.GetPrimary().GetAlias()), uid)
return err
}
return nil
@@ -1017,19 +1067,10 @@ func (ts *trafficSwitcher) stopSourceWrites(ctx context.Context) error {
err = ts.changeShardsAccess(ctx, ts.SourceKeyspaceName(), ts.SourceShards(), disallowWrites)
}
if err != nil {
- log.Warningf("Error: %s", err)
+ ts.Logger().Warningf("Error stopping writes on migration sources: %v", err)
return err
}
- return ts.ForAllSources(func(source *MigrationSource) error {
- var err error
- source.Position, err = ts.TabletManagerClient().PrimaryPosition(ctx, source.GetPrimary().Tablet)
- log.Infof("Stopped Source Writes. Position for source %v:%v: %v",
- ts.SourceKeyspaceName(), source.GetShard().ShardName(), source.Position)
- if err != nil {
- log.Warningf("Error: %s", err)
- }
- return err
- })
+ return nil
}
// switchDeniedTables switches the denied tables rules for the traffic switch.
@@ -1051,8 +1092,14 @@ func (ts *trafficSwitcher) switchDeniedTables(ctx context.Context) error {
defer cancel()
isPartial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), source.GetShard(), nil, ts.Logger())
if isPartial {
- err = fmt.Errorf("failed to successfully refresh all tablets in the %s/%s source shard (%v):\n %v",
+ msg := fmt.Sprintf("failed to successfully refresh all tablets in the %s/%s source shard (%v):\n %v",
source.GetShard().Keyspace(), source.GetShard().ShardName(), err, partialDetails)
+ if ts.force {
+ log.Warning(msg)
+ return nil
+ } else {
+ return errors.New(msg)
+ }
}
return err
})
@@ -1068,44 +1115,65 @@ func (ts *trafficSwitcher) switchDeniedTables(ctx context.Context) error {
defer cancel()
isPartial, partialDetails, err := topotools.RefreshTabletsByShard(rtbsCtx, ts.TopoServer(), ts.TabletManagerClient(), target.GetShard(), nil, ts.Logger())
if isPartial {
- err = fmt.Errorf("failed to successfully refresh all tablets in the %s/%s target shard (%v):\n %v",
+ msg := fmt.Sprintf("failed to successfully refresh all tablets in the %s/%s target shard (%v):\n %v",
target.GetShard().Keyspace(), target.GetShard().ShardName(), err, partialDetails)
+ if ts.force {
+ log.Warning(msg)
+ return nil
+ } else {
+ return errors.New(msg)
+ }
}
return err
})
})
if err := egrp.Wait(); err != nil {
- log.Warningf("Error in switchDeniedTables: %s", err)
+ ts.Logger().Warningf("Error in switchDeniedTables: %s", err)
return err
}
return nil
}
+// cancelMigration attempts to revert all changes made during the migration so that we can get back to the
+// state when traffic switching (or reversing) was initiated.
func (ts *trafficSwitcher) cancelMigration(ctx context.Context, sm *StreamMigrator) {
var err error
+
+ if ctx.Err() != nil {
+ // Even though we create a new context later on we still record any context error:
+ // for forensics in case of failures.
+ ts.Logger().Infof("In Cancel migration: original context invalid: %s", ctx.Err())
+ }
+
+ // We create a new context while canceling the migration, so that we are independent of the original
+ // context being cancelled prior to or during the cancel operation.
+ cmTimeout := 60 * time.Second
+ cmCtx, cmCancel := context.WithTimeout(context.Background(), cmTimeout)
+ defer cmCancel()
+
if ts.MigrationType() == binlogdatapb.MigrationType_TABLES {
- err = ts.switchDeniedTables(ctx)
+ err = ts.switchDeniedTables(cmCtx)
} else {
- err = ts.changeShardsAccess(ctx, ts.SourceKeyspaceName(), ts.SourceShards(), allowWrites)
+ err = ts.changeShardsAccess(cmCtx, ts.SourceKeyspaceName(), ts.SourceShards(), allowWrites)
}
if err != nil {
- ts.Logger().Errorf("Cancel migration failed: %v", err)
+ ts.Logger().Errorf("Cancel migration failed: could not revert denied tables / shard access: %v", err)
}
- sm.CancelStreamMigrations(ctx)
+ sm.CancelStreamMigrations(cmCtx)
err = ts.ForAllTargets(func(target *MigrationTarget) error {
query := fmt.Sprintf("update _vt.vreplication set state='Running', message='' where db_name=%s and workflow=%s",
encodeString(target.GetPrimary().DbName()), encodeString(ts.WorkflowName()))
- _, err := ts.TabletManagerClient().VReplicationExec(ctx, target.GetPrimary().Tablet, query)
+ _, err := ts.TabletManagerClient().VReplicationExec(cmCtx, target.GetPrimary().Tablet, query)
return err
})
if err != nil {
ts.Logger().Errorf("Cancel migration failed: could not restart vreplication: %v", err)
}
- err = ts.deleteReverseVReplication(ctx)
+ err = ts.deleteReverseVReplication(cmCtx)
if err != nil {
ts.Logger().Errorf("Cancel migration failed: could not delete reverse vreplication streams: %v", err)
}
@@ -1157,7 +1225,7 @@ func (ts *trafficSwitcher) dropSourceReverseVReplicationStreams(ctx context.Cont
func (ts *trafficSwitcher) removeTargetTables(ctx context.Context) error {
err := ts.ForAllTargets(func(target *MigrationTarget) error {
- log.Infof("ForAllTargets: %+v", target)
+ ts.Logger().Infof("ForAllTargets: %+v", target)
for _, tableName := range ts.Tables() {
primaryDbName, err := sqlescape.EnsureEscaped(target.GetPrimary().DbName())
if err != nil {
@@ -1176,7 +1244,7 @@ func (ts *trafficSwitcher) removeTargetTables(ctx context.Context) error {
ReloadSchema: true,
DisableForeignKeyChecks: true,
})
- log.Infof("Removed target table with result: %+v", res)
+ ts.Logger().Infof("Removed target table with result: %+v", res)
if err != nil {
if IsTableDidNotExistError(err) {
// The table was already gone, so we can ignore the error.
@@ -1215,7 +1283,7 @@ func (ts *trafficSwitcher) dropTargetShards(ctx context.Context) error {
func (ts *trafficSwitcher) validate(ctx context.Context) error {
if ts.MigrationType() == binlogdatapb.MigrationType_TABLES {
if ts.isPartialMigration ||
- (ts.IsMultiTenantMigration() && len(ts.options.GetShards()) > 0) {
+ (ts.IsMultiTenantMigration() && ts.options != nil && len(ts.options.GetShards()) > 0) {
return nil
}
sourceTopo := ts.ws.ts
@@ -1319,6 +1387,24 @@ func (ts *trafficSwitcher) gatherPositions(ctx context.Context) error {
})
}
+// gatherSourcePositions will get the current replication position for all
+// migration sources.
+func (ts *trafficSwitcher) gatherSourcePositions(ctx context.Context) error {
+ return ts.ForAllSources(func(source *MigrationSource) error {
+ var err error
+ tablet := source.GetPrimary().Tablet
+ tabletAlias := topoproto.TabletAliasString(tablet.Alias)
+ source.Position, err = ts.TabletManagerClient().PrimaryPosition(ctx, tablet)
+ if err != nil {
+ ts.Logger().Errorf("Error getting migration source position on %s: %s", tabletAlias, err)
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to get position on migration source %s: %v",
+ tabletAlias, err)
+ }
+ ts.Logger().Infof("Position on migration source %s after having stopped writes: %s", tabletAlias, source.Position)
+ return nil
+ })
+}
+
func (ts *trafficSwitcher) isSequenceParticipating(ctx context.Context) (bool, error) {
vschema, err := ts.TopoServer().GetVSchema(ctx, ts.targetKeyspace)
if err != nil {
@@ -1373,7 +1459,7 @@ func (ts *trafficSwitcher) getTargetSequenceMetadata(ctx context.Context) (map[s
// be in another unsharded keyspace.
smMu := sync.Mutex{}
tableCount := len(sequencesByBackingTable)
- tablesFound := 0 // Used to short circuit the search
+ tablesFound := make(map[string]struct{}) // Used to short circuit the search
// Define the function used to search each keyspace.
searchKeyspace := func(sctx context.Context, done chan struct{}, keyspace string) error {
kvs, kerr := ts.TopoServer().GetVSchema(sctx, keyspace)
@@ -1388,7 +1474,7 @@ func (ts *trafficSwitcher) getTargetSequenceMetadata(ctx context.Context) (map[s
// The table name can be escaped in the vschema definition.
unescapedTableName, err := sqlescape.UnescapeID(tableName)
if err != nil {
- return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid table name %s in keyspace %s: %v",
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid table name %q in keyspace %s: %v",
tableName, keyspace, err)
}
select {
@@ -1404,13 +1490,13 @@ func (ts *trafficSwitcher) getTargetSequenceMetadata(ctx context.Context) (map[s
sm := sequencesByBackingTable[unescapedTableName]
if tableDef != nil && tableDef.Type == vindexes.TypeSequence &&
sm != nil && unescapedTableName == sm.backingTableName {
- tablesFound++ // This is also protected by the mutex
+ tablesFound[tableName] = struct{}{} // This is also protected by the mutex
sm.backingTableKeyspace = keyspace
// Set the default keyspace name. We will later check to
// see if the tablet we send requests to is using a dbname
// override and use that if it is.
sm.backingTableDBName = "vt_" + keyspace
- if tablesFound == tableCount { // Short circuit the search
+ if len(tablesFound) == tableCount { // Short circuit the search
select {
case <-done: // It's already been closed
return true
@@ -1437,7 +1523,7 @@ func (ts *trafficSwitcher) getTargetSequenceMetadata(ctx context.Context) (map[s
// The keyspace name could be escaped so we need to unescape it.
ks, err := sqlescape.UnescapeID(keyspace)
if err != nil { // Should never happen
- return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid keyspace name %s: %v", keyspace, err)
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid keyspace name %q: %v", keyspace, err)
}
searchGroup.Go(func() error {
return searchKeyspace(gctx, searchCompleted, ks)
@@ -1447,13 +1533,90 @@ func (ts *trafficSwitcher) getTargetSequenceMetadata(ctx context.Context) (map[s
return nil, err
}
- if tablesFound != tableCount {
- return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to locate all of the backing sequence tables being used: %s",
- strings.Join(maps.Keys(sequencesByBackingTable), ","))
+ if len(tablesFound) != tableCount {
+ // Try and create the missing backing sequence tables if we can.
+ if err := ts.createMissingSequenceTables(ctx, sequencesByBackingTable, tablesFound); err != nil {
+ return nil, err
+ }
}
+
return sequencesByBackingTable, nil
}
+// createMissingSequenceTables will create the backing sequence tables for those that
+// could not be found in any current keyspace.
+func (ts trafficSwitcher) createMissingSequenceTables(ctx context.Context, sequencesByBackingTable map[string]*sequenceMetadata, tablesFound map[string]struct{}) error {
+ globalKeyspace := ts.options.GetGlobalKeyspace()
+ if globalKeyspace == "" {
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to locate all of the backing sequence tables being used and no global-keyspace was provided to auto create them in: %s",
+ strings.Join(maps.Keys(sequencesByBackingTable), ","))
+ }
+ shards, err := ts.ws.ts.GetShardNames(ctx, globalKeyspace)
+ if err != nil {
+ return err
+ }
+ if len(shards) != 1 {
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "global-keyspace %s is not unsharded", globalKeyspace)
+ }
+ globalVSchema, err := ts.ws.ts.GetVSchema(ctx, globalKeyspace)
+ if err != nil {
+ return err
+ }
+ updatedGlobalVSchema := false
+ for tableName, sequenceMetadata := range sequencesByBackingTable {
+ if _, ok := tablesFound[tableName]; !ok {
+ // Create the backing table.
+ shard, err := ts.ws.ts.GetShard(ctx, globalKeyspace, shards[0])
+ if err != nil {
+ return err
+ }
+ if shard.PrimaryAlias == nil {
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "global-keyspace %s does not currently have a primary tablet",
+ globalKeyspace)
+ }
+ primary, err := ts.ws.ts.GetTablet(ctx, shard.PrimaryAlias)
+ if err != nil {
+ return err
+ }
+ escapedTableName, err := sqlescape.EnsureEscaped(tableName)
+ if err != nil {
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid table name %s: %v",
+ tableName, err)
+ }
+ stmt := sqlparser.BuildParsedQuery(sqlCreateSequenceTable, escapedTableName)
+ _, err = ts.ws.tmc.ApplySchema(ctx, primary.Tablet, &tmutils.SchemaChange{
+ SQL: stmt.Query,
+ Force: false,
+ AllowReplication: true,
+ SQLMode: vreplication.SQLMode,
+ DisableForeignKeyChecks: true,
+ })
+ if err != nil {
+ return vterrors.Wrapf(err, "failed to create sequence backing table %s in global-keyspace %s",
+ tableName, globalKeyspace)
+ }
+ if bt := globalVSchema.Tables[sequenceMetadata.backingTableName]; bt == nil {
+ if globalVSchema.Tables == nil {
+ globalVSchema.Tables = make(map[string]*vschemapb.Table)
+ }
+ globalVSchema.Tables[tableName] = &vschemapb.Table{
+ Type: vindexes.TypeSequence,
+ }
+ updatedGlobalVSchema = true
+ sequenceMetadata.backingTableDBName = "vt_" + globalKeyspace // This will be overridden later if needed
+ sequenceMetadata.backingTableKeyspace = globalKeyspace
+ }
+ }
+ }
+ if updatedGlobalVSchema {
+ err = ts.ws.ts.SaveVSchema(ctx, globalKeyspace, globalVSchema)
+ if err != nil {
+ return vterrors.Wrapf(err, "failed to update vschema in the global-keyspace %s", globalKeyspace)
+ }
+ }
+ return nil
+}
+
// findSequenceUsageInKeyspace searches the keyspace's vschema for usage
// of sequences. It returns a map of sequence metadata keyed by the backing
// sequence table name -- if any usage is found -- along with a boolean to
@@ -1478,7 +1641,7 @@ func (ts *trafficSwitcher) findSequenceUsageInKeyspace(vschema *vschemapb.Keyspa
// in the vschema.
unescapedTable, err := sqlescape.UnescapeID(table)
if err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid table name %s defined in the sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid table name %q defined in the sequence table %+v: %v",
table, seqTable, err)
}
sm := &sequenceMetadata{
@@ -1490,17 +1653,17 @@ func (ts *trafficSwitcher) findSequenceUsageInKeyspace(vschema *vschemapb.Keyspa
if strings.Contains(seqTable.AutoIncrement.Sequence, ".") {
keyspace, tableName, found := strings.Cut(seqTable.AutoIncrement.Sequence, ".")
if !found {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence table name %s defined in the %s keyspace",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence table name %q defined in the %s keyspace",
seqTable.AutoIncrement.Sequence, ts.targetKeyspace)
}
// Unescape the table name and keyspace name as they may be escaped in the
// vschema definition if they e.g. contain dashes.
if keyspace, err = sqlescape.UnescapeID(keyspace); err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid keyspace in qualified sequence table name %s defined in sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid keyspace in qualified sequence table name %q defined in sequence table %+v: %v",
seqTable.AutoIncrement.Sequence, seqTable, err)
}
if tableName, err = sqlescape.UnescapeID(tableName); err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid qualified sequence table name %s defined in sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid qualified sequence table name %q defined in sequence table %+v: %v",
seqTable.AutoIncrement.Sequence, seqTable, err)
}
sm.backingTableKeyspace = keyspace
@@ -1514,7 +1677,7 @@ func (ts *trafficSwitcher) findSequenceUsageInKeyspace(vschema *vschemapb.Keyspa
} else {
sm.backingTableName, err = sqlescape.UnescapeID(seqTable.AutoIncrement.Sequence)
if err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence table name %s defined in sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence table name %q defined in sequence table %+v: %v",
seqTable.AutoIncrement.Sequence, seqTable, err)
}
seqTable.AutoIncrement.Sequence = sm.backingTableName
@@ -1522,16 +1685,28 @@ func (ts *trafficSwitcher) findSequenceUsageInKeyspace(vschema *vschemapb.Keyspa
}
// The column names can be escaped in the vschema definition.
for i := range seqTable.ColumnVindexes {
- unescapedColumn, err := sqlescape.UnescapeID(seqTable.ColumnVindexes[i].Column)
+ var (
+ unescapedColumn string
+ err error
+ )
+ if len(seqTable.ColumnVindexes[i].Columns) > 0 {
+ for n := range seqTable.ColumnVindexes[i].Columns {
+ unescapedColumn, err = sqlescape.UnescapeID(seqTable.ColumnVindexes[i].Columns[n])
+ seqTable.ColumnVindexes[i].Columns[n] = unescapedColumn
+ }
+ } else {
+ // This is the legacy vschema definition.
+ unescapedColumn, err = sqlescape.UnescapeID(seqTable.ColumnVindexes[i].Column)
+ seqTable.ColumnVindexes[i].Column = unescapedColumn
+ }
if err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence column vindex name %s defined in sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid sequence column vindex name %q defined in sequence table %+v: %v",
seqTable.ColumnVindexes[i].Column, seqTable, err)
}
- seqTable.ColumnVindexes[i].Column = unescapedColumn
}
unescapedAutoIncCol, err := sqlescape.UnescapeID(seqTable.AutoIncrement.Column)
if err != nil {
- return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid auto-increment column name %s defined in sequence table %+v: %v",
+ return nil, false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid auto-increment column name %q defined in sequence table %+v: %v",
seqTable.AutoIncrement.Column, seqTable, err)
}
seqTable.AutoIncrement.Column = unescapedAutoIncCol
@@ -1545,12 +1720,16 @@ func (ts *trafficSwitcher) findSequenceUsageInKeyspace(vschema *vschemapb.Keyspa
// initializeTargetSequences initializes the backing sequence tables
// using a map keyed by the backing sequence table name.
//
-// The backing tables must have already been created. This function will
-// then ensure that the next value is set to a value greater than any
-// currently stored in the using table on the target keyspace. If the
-// backing table is updated to a new higher value then it will also tell
-// the primary tablet serving the sequence to refresh/reset its cache to
-// be sure that it does not provide a value that is less than the current max.
+// The backing tables must have already been created, unless a default
+// global keyspace exists for the trafficSwitcher -- in which case we
+// will create the backing table there if needed.
+
+// This function will then ensure that the next value is set to a value
+// greater than any currently stored in the using table on the target
+// keyspace. If the backing table is updated to a new higher value then
+// it will also tell the primary tablet serving the sequence to
+// refresh/reset its cache to be sure that it does not provide a value
+// that is less than the current max.
func (ts *trafficSwitcher) initializeTargetSequences(ctx context.Context, sequencesByBackingTable map[string]*sequenceMetadata) error {
initSequenceTable := func(ictx context.Context, sequenceMetadata *sequenceMetadata) error {
// Now we need to run this query on the target shards in order
@@ -1662,13 +1841,37 @@ func (ts *trafficSwitcher) initializeTargetSequences(ctx context.Context, sequen
)
// Now execute this on the primary tablet of the unsharded keyspace
// housing the backing table.
+ initialize:
qr, ierr := ts.ws.tmc.ExecuteFetchAsApp(ictx, sequenceTablet.Tablet, true, &tabletmanagerdatapb.ExecuteFetchAsAppRequest{
Query: []byte(query.Query),
MaxRows: 1,
})
if ierr != nil {
- return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to initialize the backing sequence table %s.%s: %v",
+ vterr := vterrors.Errorf(vtrpcpb.Code_INTERNAL, "failed to initialize the backing sequence table %s.%s: %v",
sequenceMetadata.backingTableDBName, sequenceMetadata.backingTableName, ierr)
+ // If the sequence table doesn't exist, let's try and create it, otherwise
+ // return the error.
+ if sqlErr, ok := sqlerror.NewSQLErrorFromError(ierr).(*sqlerror.SQLError); !ok ||
+ (sqlErr.Num != sqlerror.ERNoSuchTable && sqlErr.Num != sqlerror.ERBadTable) {
+ return vterr
+ }
+ stmt := sqlparser.BuildParsedQuery(sqlCreateSequenceTable, backingTable)
+ _, ierr = ts.ws.tmc.ApplySchema(ctx, sequenceTablet.Tablet, &tmutils.SchemaChange{
+ SQL: stmt.Query,
+ Force: false,
+ AllowReplication: true,
+ SQLMode: vreplication.SQLMode,
+ DisableForeignKeyChecks: true,
+ })
+ if ierr != nil {
+ return vterrors.Wrapf(vterr, "could not create missing sequence table: %v", err)
+ }
+ select {
+ case <-ctx.Done():
+ return vterrors.Wrapf(vterr, "could not create missing sequence table: %v", ctx.Err())
+ default:
+ goto initialize
+ }
}
// If we actually updated the backing sequence table, then we need
// to tell the primary tablet managing the sequence to refresh/reset
diff --git a/go/vt/vtctl/workflow/traffic_switcher_test.go b/go/vt/vtctl/workflow/traffic_switcher_test.go
index 5c0b2aba682..b06c95b6c16 100644
--- a/go/vt/vtctl/workflow/traffic_switcher_test.go
+++ b/go/vt/vtctl/workflow/traffic_switcher_test.go
@@ -19,16 +19,26 @@ package workflow
import (
"context"
"fmt"
- "reflect"
+ "strconv"
+ "strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/sqlescape"
+ "vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/mysqlctl/tmutils"
"vitess.io/vitess/go/vt/proto/vschema"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vtgate/vindexes"
+ "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata"
)
type testTrafficSwitcher struct {
@@ -70,6 +80,8 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
cell := "cell1"
workflow := "wf1"
table := "`t1`"
+ tableDDL := "create table t1 (id int not null auto_increment primary key, c1 varchar(10))"
+ table2 := "t2"
unescapedTable := "t1"
sourceKeyspace := &testKeyspace{
KeyspaceName: "source-ks",
@@ -87,12 +99,25 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
env := newTestEnv(t, ctx, cell, sourceKeyspace, targetKeyspace)
defer env.close()
+ env.tmc.schema = map[string]*tabletmanagerdatapb.SchemaDefinition{
+ unescapedTable: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: unescapedTable,
+ Schema: tableDDL,
+ },
+ },
+ },
+ }
+
type testCase struct {
- name string
- sourceVSchema *vschema.Keyspace
- targetVSchema *vschema.Keyspace
- want map[string]*sequenceMetadata
- err string
+ name string
+ sourceVSchema *vschema.Keyspace
+ targetVSchema *vschema.Keyspace
+ options *vtctldatapb.WorkflowOptions
+ want map[string]*sequenceMetadata
+ expectSourceApplySchemaRequestResponse *applySchemaRequestResponse
+ err string
}
tests := []testCase{
{
@@ -148,6 +173,66 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
+ {
+ name: "auto_increment replaced with sequence",
+ sourceVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{}, // Sequence table will be created
+ },
+ options: &vtctldatapb.WorkflowOptions{
+ ShardedAutoIncrementHandling: vtctldatapb.ShardedAutoIncrementHandling_REPLACE,
+ GlobalKeyspace: sourceKeyspace.KeyspaceName,
+ },
+ expectSourceApplySchemaRequestResponse: &applySchemaRequestResponse{
+ change: &tmutils.SchemaChange{
+ SQL: sqlparser.BuildParsedQuery(sqlCreateSequenceTable,
+ sqlescape.EscapeID(fmt.Sprintf(autoSequenceTableFormat, unescapedTable))).Query,
+ Force: false,
+ AllowReplication: true,
+ SQLMode: vreplication.SQLMode,
+ DisableForeignKeyChecks: true,
+ },
+ res: &tabletmanagerdatapb.SchemaChangeResult{},
+ },
+ targetVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{
+ table: {
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Name: "xxhash",
+ Column: "`my-col`",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "my-col",
+ Sequence: fmt.Sprintf(autoSequenceTableFormat, unescapedTable),
+ },
+ },
+ },
+ },
+ want: map[string]*sequenceMetadata{
+ fmt.Sprintf(autoSequenceTableFormat, unescapedTable): {
+ backingTableName: fmt.Sprintf(autoSequenceTableFormat, unescapedTable),
+ backingTableKeyspace: "source-ks",
+ backingTableDBName: "vt_source-ks",
+ usingTableName: unescapedTable,
+ usingTableDBName: "vt_targetks",
+ usingTableDefinition: &vschema.Table{
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Column: "my-col",
+ Name: "xxhash",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "my-col",
+ Sequence: fmt.Sprintf(autoSequenceTableFormat, unescapedTable),
+ },
+ },
+ },
+ },
+ },
{
name: "sequences with backticks",
sourceVSchema: &vschema.Keyspace{
@@ -197,6 +282,138 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
+ {
+ name: "sequences using vindexes with both column definition structures",
+ sourceVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{
+ "seq1": {
+ Type: "sequence",
+ },
+ "seq2": {
+ Type: "sequence",
+ },
+ },
+ },
+ targetVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{
+ table: {
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Name: "xxhash",
+ Column: "col1",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col1",
+ Sequence: fmt.Sprintf("%s.seq1", sourceKeyspace.KeyspaceName),
+ },
+ },
+ table2: {
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Name: "xxhash",
+ Columns: []string{"col2"},
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col2",
+ Sequence: fmt.Sprintf("%s.seq2", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ },
+ want: map[string]*sequenceMetadata{
+ "seq1": {
+ backingTableName: "seq1",
+ backingTableKeyspace: "source-ks",
+ backingTableDBName: "vt_source-ks",
+ usingTableName: unescapedTable,
+ usingTableDBName: "vt_targetks",
+ usingTableDefinition: &vschema.Table{
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Column: "col1",
+ Name: "xxhash",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col1",
+ Sequence: fmt.Sprintf("%s.seq1", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ "seq2": {
+ backingTableName: "seq2",
+ backingTableKeyspace: "source-ks",
+ backingTableDBName: "vt_source-ks",
+ usingTableName: table2,
+ usingTableDBName: "vt_targetks",
+ usingTableDefinition: &vschema.Table{
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Columns: []string{"col2"},
+ Name: "xxhash",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col2",
+ Sequence: fmt.Sprintf("%s.seq2", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ },
+ },
+ {
+ name: "sequence with table having mult-col vindex",
+ sourceVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{
+ "seq1": {
+ Type: "sequence",
+ },
+ },
+ },
+ targetVSchema: &vschema.Keyspace{
+ Vindexes: vindexes,
+ Tables: map[string]*vschema.Table{
+ table: {
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Name: "xxhash",
+ Columns: []string{"col3", "col4"},
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col1",
+ Sequence: fmt.Sprintf("%s.seq1", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ },
+ want: map[string]*sequenceMetadata{
+ "seq1": {
+ backingTableName: "seq1",
+ backingTableKeyspace: "source-ks",
+ backingTableDBName: "vt_source-ks",
+ usingTableName: unescapedTable,
+ usingTableDBName: "vt_targetks",
+ usingTableDefinition: &vschema.Table{
+ ColumnVindexes: []*vschema.ColumnVindex{
+ {
+ Columns: []string{"col3", "col4"},
+ Name: "xxhash",
+ },
+ },
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "col1",
+ Sequence: fmt.Sprintf("%s.seq1", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ },
+ },
{
name: "invalid table name",
sourceVSchema: &vschema.Keyspace{
@@ -224,7 +441,7 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
- err: "invalid table name `my-`seq1` in keyspace source-ks: UnescapeID err: unexpected single backtick at position 3 in 'my-`seq1'",
+ err: "invalid table name \"`my-`seq1`\" in keyspace source-ks: UnescapeID err: unexpected single backtick at position 3 in 'my-`seq1'",
},
{
name: "invalid keyspace name",
@@ -253,7 +470,7 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
- err: "invalid keyspace in qualified sequence table name `ks`1`.`my-seq1` defined in sequence table column_vindexes:{column:\"`my-col`\" name:\"xxhash\"} auto_increment:{column:\"`my-col`\" sequence:\"`ks`1`.`my-seq1`\"}: UnescapeID err: unexpected single backtick at position 2 in 'ks`1'",
+ err: "invalid keyspace in qualified sequence table name \"`ks`1`.`my-seq1`\" defined in sequence table column_vindexes:{column:\"`my-col`\" name:\"xxhash\"} auto_increment:{column:\"`my-col`\" sequence:\"`ks`1`.`my-seq1`\"}: UnescapeID err: unexpected single backtick at position 2 in 'ks`1'",
},
{
name: "invalid auto-inc column name",
@@ -282,7 +499,7 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
- err: "invalid auto-increment column name `my`-col` defined in sequence table column_vindexes:{column:\"my-col\" name:\"xxhash\"} auto_increment:{column:\"`my`-col`\" sequence:\"my-seq1\"}: UnescapeID err: unexpected single backtick at position 2 in 'my`-col'",
+ err: "invalid auto-increment column name \"`my`-col`\" defined in sequence table column_vindexes:{column:\"my-col\" name:\"xxhash\"} auto_increment:{column:\"`my`-col`\" sequence:\"my-seq1\"}: UnescapeID err: unexpected single backtick at position 2 in 'my`-col'",
},
{
name: "invalid sequence name",
@@ -311,7 +528,7 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
},
},
},
- err: "invalid sequence table name `my-`seq1` defined in sequence table column_vindexes:{column:\"`my-col`\" name:\"xxhash\"} auto_increment:{column:\"`my-col`\" sequence:\"`my-`seq1`\"}: UnescapeID err: unexpected single backtick at position 3 in 'my-`seq1'",
+ err: "invalid sequence table name \"`my-`seq1`\" defined in sequence table column_vindexes:{column:\"`my-col`\" name:\"xxhash\"} auto_increment:{column:\"`my-col`\" sequence:\"`my-`seq1`\"}: UnescapeID err: unexpected single backtick at position 3 in 'my-`seq1'",
},
}
@@ -332,6 +549,9 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
Tablet: tablet,
},
}
+ if tc.expectSourceApplySchemaRequestResponse != nil {
+ env.tmc.expectApplySchemaRequest(tablet.Alias.Uid, tc.expectSourceApplySchemaRequestResponse)
+ }
}
for i, shard := range targetKeyspace.ShardNames {
tablet := env.tablets[targetKeyspace.KeyspaceName][startingTargetTabletUID+(i*tabletUIDStep)]
@@ -345,11 +565,12 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
id: 1,
ws: env.ws,
workflow: workflow,
- tables: []string{table},
+ tables: []string{table, table2},
sourceKeyspace: sourceKeyspace.KeyspaceName,
targetKeyspace: targetKeyspace.KeyspaceName,
sources: sources,
targets: targets,
+ options: tc.options,
}
got, err := ts.getTargetSequenceMetadata(ctx)
if tc.err != "" {
@@ -357,7 +578,328 @@ func TestGetTargetSequenceMetadata(t *testing.T) {
} else {
require.NoError(t, err)
}
- require.True(t, reflect.DeepEqual(tc.want, got), "want: %v, got: %v", tc.want, got)
+ require.EqualValues(t, tc.want, got)
})
}
}
+
+// TestSwitchTrafficPositionHandling confirms that if any writes are somehow
+// executed against the source between the stop source writes and wait for
+// catchup steps, that we have the correct position and do not lose the write(s).
+func TestTrafficSwitchPositionHandling(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+
+ workflowName := "wf1"
+ tableName := "t1"
+ sourceKeyspaceName := "sourceks"
+ targetKeyspaceName := "targetks"
+
+ schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
+ tableName: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: tableName,
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", tableName),
+ },
+ },
+ },
+ }
+
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+
+ env := newTestEnv(t, ctx, defaultCellName, sourceKeyspace, targetKeyspace)
+ defer env.close()
+ env.tmc.schema = schema
+
+ ts, _, err := env.ws.getWorkflowState(ctx, targetKeyspaceName, workflowName)
+ require.NoError(t, err)
+ sw := &switcher{ts: ts, s: env.ws}
+
+ lockCtx, sourceUnlock, lockErr := sw.lockKeyspace(ctx, ts.SourceKeyspaceName(), "test")
+ require.NoError(t, lockErr)
+ ctx = lockCtx
+ defer sourceUnlock(&err)
+ lockCtx, targetUnlock, lockErr := sw.lockKeyspace(ctx, ts.TargetKeyspaceName(), "test")
+ require.NoError(t, lockErr)
+ ctx = lockCtx
+ defer targetUnlock(&err)
+
+ err = ts.stopSourceWrites(ctx)
+ require.NoError(t, err)
+
+ // Now we simulate a write on the source.
+ newPosition := position[:strings.LastIndex(position, "-")+1]
+ oldSeqNo, err := strconv.Atoi(position[strings.LastIndex(position, "-")+1:])
+ require.NoError(t, err)
+ newPosition = fmt.Sprintf("%s%d", newPosition, oldSeqNo+1)
+ env.tmc.setPrimaryPosition(env.tablets[sourceKeyspaceName][startingSourceTabletUID], newPosition)
+
+ // And confirm that we picked up the new position.
+ err = ts.gatherSourcePositions(ctx)
+ require.NoError(t, err)
+ err = ts.ForAllSources(func(ms *MigrationSource) error {
+ require.Equal(t, newPosition, ms.Position)
+ return nil
+ })
+ require.NoError(t, err)
+}
+
+func TestInitializeTargetSequences(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+
+ workflowName := "wf1"
+ tableName := "t1"
+ sourceKeyspaceName := "sourceks"
+ targetKeyspaceName := "targetks"
+
+ schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
+ tableName: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: tableName,
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", tableName),
+ },
+ },
+ },
+ }
+
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+
+ env := newTestEnv(t, ctx, defaultCellName, sourceKeyspace, targetKeyspace)
+ defer env.close()
+ env.tmc.schema = schema
+
+ ts, _, err := env.ws.getWorkflowState(ctx, targetKeyspaceName, workflowName)
+ require.NoError(t, err)
+ sw := &switcher{ts: ts, s: env.ws}
+
+ sequencesByBackingTable := map[string]*sequenceMetadata{
+ "my-seq1": {
+ backingTableName: "my-seq1",
+ backingTableKeyspace: sourceKeyspaceName,
+ backingTableDBName: fmt.Sprintf("vt_%s", sourceKeyspaceName),
+ usingTableName: tableName,
+ usingTableDBName: "vt_targetks",
+ usingTableDefinition: &vschema.Table{
+ AutoIncrement: &vschema.AutoIncrement{
+ Column: "my-col",
+ Sequence: fmt.Sprintf("%s.my-seq1", sourceKeyspace.KeyspaceName),
+ },
+ },
+ },
+ }
+
+ env.tmc.expectVRQuery(200, "/select max.*", sqltypes.MakeTestResult(sqltypes.MakeTestFields("maxval", "int64"), "34"))
+ // Expect the insert query to be executed with 35 as a params, since we provide a maxID of 34 in the last query
+ env.tmc.expectVRQuery(100, "/insert into.*35.*", &sqltypes.Result{RowsAffected: 1})
+
+ err = sw.initializeTargetSequences(ctx, sequencesByBackingTable)
+ assert.NoError(t, err)
+
+ // Expect the queries to be cleared
+ assert.Empty(t, env.tmc.vrQueries[100])
+ assert.Empty(t, env.tmc.vrQueries[200])
+}
+
+func TestAddTenantFilter(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+
+ workflowName := "wf1"
+ tableName := "t1"
+ sourceKeyspaceName := "sourceks"
+ targetKeyspaceName := "targetks"
+
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+
+ schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
+ tableName: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: tableName,
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", tableName),
+ },
+ },
+ },
+ }
+
+ env := newTestEnv(t, ctx, defaultCellName, sourceKeyspace, targetKeyspace)
+ defer env.close()
+ env.tmc.schema = schema
+
+ err := env.ts.SaveVSchema(ctx, targetKeyspaceName, &vschema.Keyspace{
+ MultiTenantSpec: &vschema.MultiTenantSpec{
+ TenantIdColumnName: "tenant_id",
+ TenantIdColumnType: sqltypes.Int64,
+ },
+ })
+ require.NoError(t, err)
+
+ ts, _, err := env.ws.getWorkflowState(ctx, targetKeyspaceName, workflowName)
+ require.NoError(t, err)
+
+ ts.options.TenantId = "123"
+
+ filter, err := ts.addTenantFilter(ctx, fmt.Sprintf("select * from %s where id < 5", tableName))
+ assert.NoError(t, err)
+ assert.Equal(t, "select * from t1 where tenant_id = 123 and id < 5", filter)
+}
+
+func TestChangeShardRouting(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+
+ workflowName := "wf1"
+ tableName := "t1"
+ sourceKeyspaceName := "sourceks"
+ targetKeyspaceName := "targetks"
+
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+
+ schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
+ tableName: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: tableName,
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", tableName),
+ },
+ },
+ },
+ }
+
+ env := newTestEnv(t, ctx, defaultCellName, sourceKeyspace, targetKeyspace)
+ defer env.close()
+ env.tmc.schema = schema
+
+ ts, _, err := env.ws.getWorkflowState(ctx, targetKeyspaceName, workflowName)
+ require.NoError(t, err)
+
+ err = env.ws.ts.UpdateSrvKeyspace(ctx, "cell", targetKeyspaceName, &topodatapb.SrvKeyspace{
+ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{
+ {
+ ShardReferences: []*topodatapb.ShardReference{
+ {
+ Name: "0",
+ },
+ },
+ },
+ },
+ })
+ require.NoError(t, err)
+
+ err = env.ws.ts.UpdateSrvKeyspace(ctx, "cell", sourceKeyspaceName, &topodatapb.SrvKeyspace{
+ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{
+ {
+ ShardReferences: []*topodatapb.ShardReference{
+ {
+ Name: "0",
+ },
+ },
+ },
+ },
+ })
+ require.NoError(t, err)
+
+ ctx, _, err = env.ws.ts.LockShard(ctx, targetKeyspaceName, "0", "targetks0")
+ require.NoError(t, err)
+
+ ctx, _, err = env.ws.ts.LockKeyspace(ctx, targetKeyspaceName, "targetks0")
+ require.NoError(t, err)
+
+ err = ts.changeShardRouting(ctx)
+ assert.NoError(t, err)
+
+ sourceShardInfo, err := env.ws.ts.GetShard(ctx, sourceKeyspaceName, "0")
+ assert.NoError(t, err)
+ assert.False(t, sourceShardInfo.IsPrimaryServing, "source shard shouldn't have it's primary serving after changeShardRouting() is called.")
+
+ targetShardInfo, err := env.ws.ts.GetShard(ctx, targetKeyspaceName, "0")
+ assert.NoError(t, err)
+ assert.True(t, targetShardInfo.IsPrimaryServing, "target shard should have it's primary serving after changeShardRouting() is called.")
+}
+
+func TestAddParticipatingTablesToKeyspace(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+
+ workflowName := "wf1"
+ tableName := "t1"
+ sourceKeyspaceName := "sourceks"
+ targetKeyspaceName := "targetks"
+
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: sourceKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: targetKeyspaceName,
+ ShardNames: []string{"0"},
+ }
+
+ schema := map[string]*tabletmanagerdatapb.SchemaDefinition{
+ tableName: {
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: tableName,
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", tableName),
+ },
+ },
+ },
+ }
+
+ env := newTestEnv(t, ctx, defaultCellName, sourceKeyspace, targetKeyspace)
+ defer env.close()
+ env.tmc.schema = schema
+
+ ts, _, err := env.ws.getWorkflowState(ctx, targetKeyspaceName, workflowName)
+ require.NoError(t, err)
+
+ err = ts.addParticipatingTablesToKeyspace(ctx, sourceKeyspaceName, "")
+ assert.NoError(t, err)
+
+ vs, err := env.ts.GetVSchema(ctx, sourceKeyspaceName)
+ assert.NoError(t, err)
+ assert.NotNil(t, vs.Tables["t1"])
+ assert.Empty(t, vs.Tables["t1"])
+
+ specs := `{"t1":{"column_vindexes":[{"column":"col1","name":"v1"}, {"column":"col2","name":"v2"}]},"t2":{"column_vindexes":[{"column":"col2","name":"v2"}]}}`
+ err = ts.addParticipatingTablesToKeyspace(ctx, sourceKeyspaceName, specs)
+ assert.NoError(t, err)
+
+ vs, err = env.ts.GetVSchema(ctx, sourceKeyspaceName)
+ assert.NoError(t, err)
+ require.NotNil(t, vs.Tables["t1"])
+ require.NotNil(t, vs.Tables["t2"])
+ assert.Len(t, vs.Tables["t1"].ColumnVindexes, 2)
+ assert.Len(t, vs.Tables["t2"].ColumnVindexes, 1)
+}
diff --git a/go/vt/vtctl/workflow/utils.go b/go/vt/vtctl/workflow/utils.go
index 9cedf01733e..65fa49fde86 100644
--- a/go/vt/vtctl/workflow/utils.go
+++ b/go/vt/vtctl/workflow/utils.go
@@ -37,6 +37,7 @@ import (
"vitess.io/vitess/go/vt/discovery"
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
@@ -122,7 +123,8 @@ func validateNewWorkflow(ctx context.Context, ts *topo.Server, tmc tmclient.Tabl
}
for _, wf := range res.Workflows {
if wf.Workflow == workflow {
- allErrors.RecordError(fmt.Errorf("workflow %s already exists in keyspace %s on tablet %v", workflow, keyspace, primary.Alias))
+ allErrors.RecordError(fmt.Errorf("workflow %s already exists in keyspace %s on tablet %s",
+ workflow, keyspace, topoproto.TabletAliasString(primary.Alias)))
return
}
}
@@ -216,21 +218,34 @@ func stripTableForeignKeys(ddl string, parser *sqlparser.Parser) (string, error)
return newDDL, nil
}
-func stripAutoIncrement(ddl string, parser *sqlparser.Parser) (string, error) {
+// stripAutoIncrement will strip any MySQL auto_increment clause in the given
+// table definition. If an optional replace function is specified then that
+// callback will be used to e.g. replace the MySQL clause with a Vitess
+// VSchema AutoIncrement definition.
+func stripAutoIncrement(ddl string, parser *sqlparser.Parser, replace func(columnName string) error) (string, error) {
newDDL, err := parser.ParseStrictDDL(ddl)
if err != nil {
return "", err
}
- _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
+ err = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
switch node := node.(type) {
case *sqlparser.ColumnDefinition:
if node.Type.Options.Autoincrement {
node.Type.Options.Autoincrement = false
+ if replace != nil {
+ if err := replace(sqlparser.String(node.Name)); err != nil {
+ return false, vterrors.Wrapf(err, "failed to replace auto_increment column %q in %q", sqlparser.String(node.Name), ddl)
+ }
+
+ }
}
}
return true, nil
}, newDDL)
+ if err != nil {
+ return "", err
+ }
return sqlparser.String(newDDL), nil
}
@@ -397,8 +412,7 @@ func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManag
optionsJSON := wf.GetOptions()
if optionsJSON != "" {
if err := json.Unmarshal([]byte(optionsJSON), &options); err != nil {
- log.Errorf("failed to unmarshal options: %v %s", err, optionsJSON)
- return nil, err
+ return nil, vterrors.Wrapf(err, "failed to unmarshal options: %s", optionsJSON)
}
}
@@ -544,7 +558,7 @@ func doValidateWorkflowHasCompleted(ctx context.Context, ts *trafficSwitcher) er
_ = ts.ForAllSources(func(source *MigrationSource) error {
wg.Add(1)
if source.GetShard().IsPrimaryServing {
- rec.RecordError(fmt.Errorf(fmt.Sprintf("Shard %s is still serving", source.GetShard().ShardName())))
+ rec.RecordError(fmt.Errorf("shard %s is still serving", source.GetShard().ShardName()))
}
wg.Done()
return nil
@@ -640,7 +654,7 @@ func parseTabletTypes(tabletTypes []topodatapb.TabletType) (hasReplica, hasRdonl
func areTabletsAvailableToStreamFrom(ctx context.Context, req *vtctldatapb.WorkflowSwitchTrafficRequest, ts *trafficSwitcher, keyspace string, shards []*topo.ShardInfo) error {
// We use the value from the workflow for the TabletPicker.
tabletTypesStr := ts.optTabletTypes
- cells := req.Cells
+ cells := req.GetCells()
// If no cells were provided in the command then use the value from the workflow.
if len(cells) == 0 && ts.optCells != "" {
cells = strings.Split(strings.TrimSpace(ts.optCells), ",")
@@ -670,7 +684,7 @@ func areTabletsAvailableToStreamFrom(ctx context.Context, req *vtctldatapb.Workf
wg.Wait()
if allErrors.HasErrors() {
- log.Errorf("%s", allErrors.Error())
+ ts.Logger().Error(allErrors.Error())
return allErrors.Error()
}
return nil
@@ -902,7 +916,7 @@ func validateTenantId(dataType querypb.Type, value string) error {
}
func updateKeyspaceRoutingState(ctx context.Context, ts *topo.Server, sourceKeyspace, targetKeyspace string, state *State) error {
- // For multi-tenant migrations, we only support switching traffic to all cells at once
+ // For multi-tenant migrations, we only support switching traffic to all cells at once.
cells, err := ts.GetCellInfoNames(ctx)
if err != nil {
return err
@@ -959,3 +973,81 @@ func IsTableDidNotExistError(err error) bool {
}
return false
}
+
+func getOptionsJSON(workflowOptions *vtctldatapb.WorkflowOptions) (string, error) {
+ defaultJSON := "{}"
+ if workflowOptions == nil {
+ return defaultJSON, nil
+ }
+ optionsJSON, err := json.Marshal(workflowOptions)
+ if err != nil || optionsJSON == nil {
+ return defaultJSON, err
+ }
+ return string(optionsJSON), nil
+}
+
+// defaultErrorHandler provides a way to consistently handle errors by logging and
+// returning them.
+func defaultErrorHandler(logger logutil.Logger, message string, err error) (*[]string, error) {
+ werr := vterrors.Wrap(err, message)
+ logger.Error(werr)
+ return nil, werr
+}
+
+// applyTargetShards applies the targetShards, coming from a command, to the trafficSwitcher's
+// migration targets.
+// It will return an error if the targetShards list contains a shard that is not a valid shard
+// for the workflow.
+// It will then remove any migration targets from the trafficSwitcher that are not in the
+// targetShards list.
+func applyTargetShards(ts *trafficSwitcher, targetShards []string) error {
+ if ts == nil {
+ return nil
+ }
+ if ts.targets == nil {
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no targets found for workflow %s", ts.workflow)
+ }
+ tsm := make(map[string]struct{}, len(targetShards))
+ for _, targetShard := range targetShards {
+ if _, ok := ts.targets[targetShard]; !ok {
+ return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "specified target shard %s not a valid target for workflow %s",
+ targetShard, ts.workflow)
+ }
+ tsm[targetShard] = struct{}{}
+ }
+ for key, target := range ts.targets {
+ if target == nil || target.GetShard() == nil { // Should never happen
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid target found for workflow %s", ts.workflow)
+ }
+ if _, ok := tsm[target.GetShard().ShardName()]; !ok {
+ delete(ts.targets, key)
+ }
+ }
+ return nil
+}
+
+// validateSourceTablesExist validates that tables provided are present
+// in the source keyspace.
+func validateSourceTablesExist(ctx context.Context, sourceKeyspace string, ksTables, tables []string) error {
+ var missingTables []string
+ for _, table := range tables {
+ if schema.IsInternalOperationTableName(table) {
+ continue
+ }
+ found := false
+
+ for _, ksTable := range ksTables {
+ if table == ksTable {
+ found = true
+ break
+ }
+ }
+ if !found {
+ missingTables = append(missingTables, table)
+ }
+ }
+ if len(missingTables) > 0 {
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "table(s) not found in source keyspace %s: %s", sourceKeyspace, strings.Join(missingTables, ","))
+ }
+ return nil
+}
diff --git a/go/vt/vtctl/workflow/utils_test.go b/go/vt/vtctl/workflow/utils_test.go
index d79c4710b77..8458cf60995 100644
--- a/go/vt/vtctl/workflow/utils_test.go
+++ b/go/vt/vtctl/workflow/utils_test.go
@@ -11,6 +11,7 @@ import (
"testing"
"time"
+ "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3"
@@ -20,8 +21,82 @@ import (
"vitess.io/vitess/go/vt/topo/etcd2topo"
"vitess.io/vitess/go/vt/topo/memorytopo"
"vitess.io/vitess/go/vt/topotools"
+
+ "vitess.io/vitess/go/vt/proto/vtctldata"
)
+// TestCreateDefaultShardRoutingRules confirms that the default shard routing rules are created correctly for sharded
+// and unsharded keyspaces.
+func TestCreateDefaultShardRoutingRules(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ ks1 := &testKeyspace{
+ KeyspaceName: "sourceks",
+ }
+ ks2 := &testKeyspace{
+ KeyspaceName: "targetks",
+ }
+
+ type testCase struct {
+ name string
+ sourceKeyspace *testKeyspace
+ targetKeyspace *testKeyspace
+ shards []string
+ want map[string]string
+ }
+ getExpectedRules := func(sourceKeyspace, targetKeyspace *testKeyspace) map[string]string {
+ rules := make(map[string]string)
+ for _, targetShard := range targetKeyspace.ShardNames {
+ rules[fmt.Sprintf("%s.%s", targetKeyspace.KeyspaceName, targetShard)] = sourceKeyspace.KeyspaceName
+ }
+ return rules
+
+ }
+ testCases := []testCase{
+ {
+ name: "unsharded",
+ sourceKeyspace: ks1,
+ targetKeyspace: ks2,
+ shards: []string{"0"},
+ },
+ {
+ name: "sharded",
+ sourceKeyspace: ks2,
+ targetKeyspace: ks1,
+ shards: []string{"-80", "80-"},
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ tc.sourceKeyspace.ShardNames = tc.shards
+ tc.targetKeyspace.ShardNames = tc.shards
+ env := newTestEnv(t, ctx, defaultCellName, tc.sourceKeyspace, tc.targetKeyspace)
+ defer env.close()
+ ms := &vtctldata.MaterializeSettings{
+ Workflow: "wf1",
+ SourceKeyspace: tc.sourceKeyspace.KeyspaceName,
+ TargetKeyspace: tc.targetKeyspace.KeyspaceName,
+ TableSettings: []*vtctldata.TableMaterializeSettings{
+ {
+ TargetTable: "t1",
+ SourceExpression: "select * from t1",
+ },
+ },
+ Cell: "zone1",
+ SourceShards: tc.sourceKeyspace.ShardNames,
+ }
+ err := createDefaultShardRoutingRules(ctx, ms, env.ts)
+ require.NoError(t, err)
+ rules, err := topotools.GetShardRoutingRules(ctx, env.ts)
+ require.NoError(t, err)
+ require.Len(t, rules, len(tc.shards))
+ want := getExpectedRules(tc.sourceKeyspace, tc.targetKeyspace)
+ require.EqualValues(t, want, rules)
+ })
+ }
+}
+
// TestUpdateKeyspaceRoutingRule confirms that the keyspace routing rules are updated correctly.
func TestUpdateKeyspaceRoutingRule(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
@@ -170,3 +245,39 @@ func startEtcd(t *testing.T) string {
return clientAddr
}
+
+func TestValidateSourceTablesExist(t *testing.T) {
+ ctx := context.Background()
+ ks := "source_keyspace"
+ ksTables := []string{"table1", "table2"}
+
+ testCases := []struct {
+ name string
+ tables []string
+ errContains string
+ }{
+ {
+ name: "no error",
+ tables: []string{"table2"},
+ },
+ {
+ name: "ignore internal table",
+ tables: []string{"_vt_hld_6ace8bcef73211ea87e9f875a4d24e90_20200915120410_", "table1", "table2"},
+ },
+ {
+ name: "table not found error",
+ tables: []string{"table3", "table1", "table2"},
+ errContains: "table3",
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ err := validateSourceTablesExist(ctx, ks, ksTables, tc.tables)
+ if tc.errContains != "" {
+ assert.ErrorContains(t, err, tc.errContains)
+ } else {
+ assert.NoError(t, err)
+ }
+ })
+ }
+}
diff --git a/go/vt/vtctl/workflow/vreplication_stream_test.go b/go/vt/vtctl/workflow/vreplication_stream_test.go
new file mode 100644
index 00000000000..6269cfa978e
--- /dev/null
+++ b/go/vt/vtctl/workflow/vreplication_stream_test.go
@@ -0,0 +1,52 @@
+/*
+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 (
+ "fmt"
+ "reflect"
+ "testing"
+)
+
+// TestVReplicationStreams tests various methods of VReplicationStreams.
+func TestVReplicationStreams(t *testing.T) {
+ var streams VReplicationStreams
+ for i := 1; i <= 3; i++ {
+ streams = append(streams, &VReplicationStream{ID: int32(i), Workflow: fmt.Sprintf("workflow%d", i)})
+ }
+
+ tests := []struct {
+ name string
+ funcUnderTest func(VReplicationStreams) interface{}
+ expectedResult interface{}
+ }{
+ {"Test IDs", func(s VReplicationStreams) interface{} { return s.IDs() }, []int32{1, 2, 3}},
+ {"Test Values", func(s VReplicationStreams) interface{} { return s.Values() }, "(1, 2, 3)"},
+ {"Test Workflows", func(s VReplicationStreams) interface{} { return s.Workflows() }, []string{"workflow1", "workflow2", "workflow3"}},
+ {"Test Copy", func(s VReplicationStreams) interface{} { return s.Copy() }, streams.Copy()},
+ {"Test ToSlice", func(s VReplicationStreams) interface{} { return s.ToSlice() }, []*VReplicationStream{streams[0], streams[1], streams[2]}},
+ }
+
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ result := tt.funcUnderTest(streams)
+ if !reflect.DeepEqual(result, tt.expectedResult) {
+ t.Errorf("Failed %s: expected %v, got %v", tt.name, tt.expectedResult, result)
+ }
+ })
+ }
+}
diff --git a/go/vt/vtctl/workflow/workflow_state_test.go b/go/vt/vtctl/workflow/workflow_state_test.go
new file mode 100644
index 00000000000..96f63f8ab31
--- /dev/null
+++ b/go/vt/vtctl/workflow/workflow_state_test.go
@@ -0,0 +1,167 @@
+/*
+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"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+)
+
+func setupMoveTables(t *testing.T, ctx context.Context) *testEnv {
+ schema := map[string]*tabletmanagerdata.SchemaDefinition{
+ "t1": {
+ TableDefinitions: []*tabletmanagerdata.TableDefinition{
+ {
+ Name: "t1",
+ Schema: fmt.Sprintf("CREATE TABLE %s (id BIGINT, name VARCHAR(64), PRIMARY KEY (id))", "t1"),
+ },
+ },
+ },
+ }
+ sourceKeyspace := &testKeyspace{
+ KeyspaceName: "source",
+ ShardNames: []string{"0"},
+ }
+ targetKeyspace := &testKeyspace{
+ KeyspaceName: "target",
+ ShardNames: []string{"0"},
+ }
+ te := newTestEnv(t, ctx, "zone1", sourceKeyspace, targetKeyspace)
+ te.tmc.schema = schema
+ var wfs tabletmanagerdata.ReadVReplicationWorkflowsResponse
+ id := int32(1)
+ wfs.Workflows = append(wfs.Workflows, &tabletmanagerdata.ReadVReplicationWorkflowResponse{
+ Workflow: "wf1",
+ WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables,
+ }, &tabletmanagerdata.ReadVReplicationWorkflowResponse{
+ Workflow: "wf2",
+ WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables,
+ })
+ wfs.Workflows[0].Streams = append(wfs.Workflows[0].Streams, &tabletmanagerdata.ReadVReplicationWorkflowResponse_Stream{
+ Id: id,
+ Bls: &binlogdatapb.BinlogSource{
+ Keyspace: te.sourceKeyspace.KeyspaceName,
+ Shard: "0",
+ Filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{
+ {Match: "t1", Filter: "select * from t1"},
+ },
+ },
+ Tables: []string{"t1"},
+ },
+ Pos: position,
+ State: binlogdatapb.VReplicationWorkflowState_Running,
+ })
+
+ workflowKey := te.tmc.GetWorkflowKey("target", "wf1")
+ workflowResponses := []*tabletmanagerdata.ReadVReplicationWorkflowsResponse{
+ nil, // this is the response for getting stopped workflows
+ &wfs, &wfs, &wfs, &wfs, &wfs, &wfs, // return the full list for subsequent GetWorkflows calls
+ }
+ for _, resp := range workflowResponses {
+ te.tmc.AddVReplicationWorkflowsResponse(workflowKey, resp)
+ }
+ te.tmc.readVReplicationWorkflowRequests[200] = &readVReplicationWorkflowRequestResponse{
+ req: &tabletmanagerdata.ReadVReplicationWorkflowRequest{
+ Workflow: "wf1",
+ },
+ }
+ te.updateTableRoutingRules(t, ctx, nil, []string{"t1"},
+ "source", te.targetKeyspace.KeyspaceName, "source")
+ return te
+}
+
+// TestWorkflowStateMoveTables tests the logic used to determine the state of a MoveTables workflow based on the
+// routing rules. We setup two workflows with the same table in both source and target keyspaces.
+func TestWorkflowStateMoveTables(t *testing.T) {
+ ctx := context.Background()
+ te := setupMoveTables(t, ctx)
+ require.NotNil(t, te)
+ type testCase struct {
+ name string
+ wf1SwitchedTabletTypes []topodatapb.TabletType
+ wf1ExpectedState string
+ // Simulate a second workflow to validate that the logic used to determine the state of the first workflow
+ // from the routing rules is not affected by the presence of other workflows in different states.
+ wf2SwitchedTabletTypes []topodatapb.TabletType
+ }
+ testCases := []testCase{
+ {
+ name: "switch reads",
+ wf1SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY},
+ wf1ExpectedState: "All Reads Switched. Writes Not Switched",
+ wf2SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY},
+ },
+ {
+ name: "switch writes",
+ wf1SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY},
+ wf1ExpectedState: "Reads Not Switched. Writes Switched",
+ wf2SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_REPLICA, topodatapb.TabletType_RDONLY},
+ },
+ {
+ name: "switch reads and writes",
+ wf1SwitchedTabletTypes: defaultTabletTypes,
+ wf1ExpectedState: "All Reads Switched. Writes Switched",
+ },
+ {
+ name: "switch rdonly only",
+ wf1SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY},
+ wf1ExpectedState: "Reads partially switched. Replica not switched. All Rdonly Reads Switched. Writes Not Switched",
+ wf2SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY},
+ },
+ {
+ name: "switch replica only",
+ wf1SwitchedTabletTypes: []topodatapb.TabletType{topodatapb.TabletType_REPLICA},
+ wf1ExpectedState: "Reads partially switched. All Replica Reads Switched. Rdonly not switched. Writes Not Switched",
+ wf2SwitchedTabletTypes: defaultTabletTypes,
+ },
+ }
+ tables := []string{"t1"}
+
+ getStateString := func(targetKeyspace, wfName string) string {
+ tsw, state, err := te.ws.getWorkflowState(ctx, targetKeyspace, wfName)
+ require.NoError(t, err)
+ require.NotNil(t, tsw)
+ require.NotNil(t, state)
+ return state.String()
+ }
+ require.Equal(t, "Reads Not Switched. Writes Not Switched", getStateString("target", "wf1"))
+
+ resetRoutingRules := func() {
+ te.updateTableRoutingRules(t, ctx, nil, tables,
+ "source", te.targetKeyspace.KeyspaceName, "source")
+ te.updateTableRoutingRules(t, ctx, nil, tables,
+ "source2", "target2", "source2")
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ resetRoutingRules()
+ te.updateTableRoutingRules(t, ctx, tc.wf1SwitchedTabletTypes, tables,
+ "source", te.targetKeyspace.KeyspaceName, te.targetKeyspace.KeyspaceName)
+ te.updateTableRoutingRules(t, ctx, tc.wf2SwitchedTabletTypes, tables,
+ "source2", "target2", "target2")
+ require.Equal(t, tc.wf1ExpectedState, getStateString("target", "wf1"))
+ })
+ }
+}
diff --git a/go/vt/vtctl/workflow/workflows.go b/go/vt/vtctl/workflow/workflows.go
new file mode 100644
index 00000000000..da0ee5dfec7
--- /dev/null
+++ b/go/vt/vtctl/workflow/workflows.go
@@ -0,0 +1,672 @@
+/*
+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.
+*/
+
+/*
+This file provides functions for fetching and retrieving information about VReplication workflows
+
+At the moment it is used by the `GetWorkflows` function in `server.go and includes functionality to
+get the following:
+- Fetch workflows by shard
+- Fetch copy states by shard stream
+- Build workflows with metadata
+- Fetch stream logs
+*/
+
+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))
+
+ 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
+ }
+
+ 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, "workflowFetcher.workflow.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 {
+ 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, "workflowFetcher.workflow.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.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["table_name"].ToString(),
+ LastPk: row["lastpk"].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](),
+ }
+ }
+
+ metadata := workflowMetadataMap[workflowName]
+ err := wf.scanWorkflow(ctx, workflow, wfres, tablet, metadata, copyStatesByShardStreamId, req.Keyspace)
+ if err != nil {
+ return nil, err
+ }
+ }
+ }
+
+ for name, workflow := range workflowsMap {
+ meta := workflowMetadataMap[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,
+ meta *workflowMetadata,
+ 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
+ // 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 copyStates, ok := copyStatesByShardStreamId[shardStreamId]; ok {
+ stream.CopyStates = copyStates
+ }
+
+ if rstream.TimeUpdated == nil {
+ rstream.TimeUpdated = &vttimepb.Time{}
+ }
+
+ stream.State = getStreamState(stream, rstream)
+
+ shardStream.Streams = append(shardStream.Streams, stream)
+
+ 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, "workflowFetcher.workflow.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.Named().Rows {
+ id, err := row["id"].ToCastInt64()
+ if err != nil {
+ markErrors(err)
+ continue
+ }
+
+ streamID, err := row["vrepl_id"].ToCastInt64()
+ if err != nil {
+ markErrors(err)
+ continue
+ }
+
+ typ := row["type"].ToString()
+ state := row["state"].ToString()
+ message := row["message"].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["updated_at"].ToString())
+ if err != nil {
+ markErrors(err)
+ continue
+ }
+
+ count, err := row["count"].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
+ }
+ }
+ }
+}
+
+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()
+}
diff --git a/go/vt/vtctl/workflow/workflows_test.go b/go/vt/vtctl/workflow/workflows_test.go
new file mode 100644
index 00000000000..2015c8d1b7c
--- /dev/null
+++ b/go/vt/vtctl/workflow/workflows_test.go
@@ -0,0 +1,260 @@
+/*
+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"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+
+ "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"
+)
+
+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)
+ })
+ }
+}
+
+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)
+
+ wf := workflowFetcher{
+ 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 := wf.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)
+}
+
+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)
+
+ wf := workflowFetcher{
+ 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 := wf.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"])
+}
diff --git a/go/vt/vtenv/vtenv.go b/go/vt/vtenv/vtenv.go
index 1371affff52..6218c96c715 100644
--- a/go/vt/vtenv/vtenv.go
+++ b/go/vt/vtenv/vtenv.go
@@ -67,6 +67,12 @@ func NewTestEnv() *Environment {
}
}
+func NewLegacyTestEnv() *Environment {
+ env := NewTestEnv()
+ env.mysqlVersion = config.LegacyMySQLVersion
+ return env
+}
+
func (e *Environment) CollationEnv() *collations.Environment {
return e.collationEnv
}
diff --git a/go/vt/vterrors/code.go b/go/vt/vterrors/code.go
index 83a87503265..31c98cef280 100644
--- a/go/vt/vterrors/code.go
+++ b/go/vt/vterrors/code.go
@@ -97,6 +97,11 @@ var (
VT09023 = errorWithoutState("VT09023", vtrpcpb.Code_FAILED_PRECONDITION, "could not map %v to a keyspace id", "Unable to determine the shard for the given row.")
VT09024 = errorWithoutState("VT09024", vtrpcpb.Code_FAILED_PRECONDITION, "could not map %v to a unique keyspace id: %v", "Unable to determine the shard for the given row.")
VT09025 = errorWithoutState("VT09025", vtrpcpb.Code_FAILED_PRECONDITION, "atomic transaction error: %v", "Error in atomic transactions")
+ VT09026 = errorWithState("VT09026", vtrpcpb.Code_FAILED_PRECONDITION, CTERecursiveRequiresUnion, "Recursive Common Table Expression '%s' should contain a UNION", "")
+ VT09027 = errorWithState("VT09027", vtrpcpb.Code_FAILED_PRECONDITION, CTERecursiveForbidsAggregation, "Recursive Common Table Expression '%s' can contain neither aggregation nor window functions in recursive query block", "")
+ VT09028 = errorWithState("VT09028", vtrpcpb.Code_FAILED_PRECONDITION, CTERecursiveForbiddenJoinOrder, "In recursive query block of Recursive Common Table Expression '%s', the recursive table must neither be in the right argument of a LEFT JOIN, nor be forced to be non-first with join order hints", "")
+ VT09029 = errorWithState("VT09029", vtrpcpb.Code_FAILED_PRECONDITION, CTERecursiveRequiresSingleReference, "In recursive query block of Recursive Common Table Expression %s, the recursive table must be referenced only once, and not in any subquery", "")
+ VT09030 = errorWithState("VT09030", vtrpcpb.Code_FAILED_PRECONDITION, CTEMaxRecursionDepth, "Recursive query aborted after 1000 iterations.", "")
VT10001 = errorWithoutState("VT10001", vtrpcpb.Code_ABORTED, "foreign key constraints are not allowed", "Foreign key constraints are not allowed, see https://vitess.io/blog/2021-06-15-online-ddl-why-no-fk/.")
VT10002 = errorWithoutState("VT10002", vtrpcpb.Code_ABORTED, "atomic distributed transaction not allowed: %s", "The distributed transaction cannot be committed. A rollback decision is taken.")
@@ -183,6 +188,10 @@ var (
VT09022,
VT09023,
VT09024,
+ VT09026,
+ VT09027,
+ VT09028,
+ VT09029,
VT10001,
VT10002,
VT12001,
diff --git a/go/vt/vterrors/errors_test.go b/go/vt/vterrors/errors_test.go
index 49b77ee0385..3444b0986c4 100644
--- a/go/vt/vterrors/errors_test.go
+++ b/go/vt/vterrors/errors_test.go
@@ -213,7 +213,7 @@ func TestWrapf(t *testing.T) {
}
for _, tt := range tests {
- got := Wrapf(tt.err, tt.message).Error()
+ got := Wrap(tt.err, tt.message).Error()
if got != tt.want {
t.Errorf("Wrapf(%v, %q): got: %v, want %v", tt.err, tt.message, got, tt.want)
}
diff --git a/go/vt/vterrors/state.go b/go/vt/vterrors/state.go
index 82434df382a..528000e9e41 100644
--- a/go/vt/vterrors/state.go
+++ b/go/vt/vterrors/state.go
@@ -62,6 +62,11 @@ const (
NoReferencedRow2
UnknownStmtHandler
KeyDoesNotExist
+ CTERecursiveRequiresSingleReference
+ CTERecursiveRequiresUnion
+ CTERecursiveForbidsAggregation
+ CTERecursiveForbiddenJoinOrder
+ CTEMaxRecursionDepth
// not found
BadDb
diff --git a/go/vt/vterrors/vterrors.go b/go/vt/vterrors/vterrors.go
index 6a322837de9..c97a7c8e45f 100644
--- a/go/vt/vterrors/vterrors.go
+++ b/go/vt/vterrors/vterrors.go
@@ -150,10 +150,14 @@ func Errorf(code vtrpcpb.Code, format string, args ...any) error {
// NewErrorf also records the stack trace at the point it was called.
// Use this for errors in Vitess that we eventually want to mimic as a MySQL error
func NewErrorf(code vtrpcpb.Code, state State, format string, args ...any) error {
- msg := format
- if len(args) != 0 {
- msg = fmt.Sprintf(format, args...)
- }
+ return NewError(code, state, fmt.Sprintf(format, args...))
+}
+
+// NewErrorf formats according to a format specifier and returns the string
+// as a value that satisfies error.
+// NewErrorf also records the stack trace at the point it was called.
+// Use this for errors in Vitess that we eventually want to mimic as a MySQL error
+func NewError(code vtrpcpb.Code, state State, msg string) error {
return &fundamental{
msg: msg,
code: code,
@@ -251,14 +255,7 @@ func Wrap(err error, message string) error {
// at the point Wrapf is call, and the format specifier.
// If err is nil, Wrapf returns nil.
func Wrapf(err error, format string, args ...any) error {
- if err == nil {
- return nil
- }
- return &wrapping{
- cause: err,
- msg: fmt.Sprintf(format, args...),
- stack: callers(),
- }
+ return Wrap(err, fmt.Sprintf(format, args...))
}
// Unwrap attempts to return the Cause of the given error, if it is indeed the result of a vterrors.Wrapf()
diff --git a/go/vt/vtexplain/vtexplain_test.go b/go/vt/vtexplain/vtexplain_test.go
index ed32d0698db..e9420b043c7 100644
--- a/go/vt/vtexplain/vtexplain_test.go
+++ b/go/vt/vtexplain/vtexplain_test.go
@@ -296,7 +296,7 @@ func TestJSONOutput(t *testing.T) {
}`
diff := cmp.Diff(wantJSON, string(actionsJSON))
if diff != "" {
- t.Errorf(diff)
+ t.Error(diff)
}
}
diff --git a/go/vt/vtexplain/vtexplain_vtgate.go b/go/vt/vtexplain/vtexplain_vtgate.go
index d511e2d2ea0..f9ae8be3820 100644
--- a/go/vt/vtexplain/vtexplain_vtgate.go
+++ b/go/vt/vtexplain/vtexplain_vtgate.go
@@ -38,6 +38,7 @@ import (
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate"
"vitess.io/vitess/go/vt/vtgate/engine"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vtgate/logstats"
"vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vttablet/queryservice"
@@ -235,7 +236,7 @@ func (vte *VTExplain) vtgateExecute(sql string) ([]*engine.Plan, map[string]*Tab
// This will ensure that the commit/rollback order is predictable.
vte.sortShardSession()
- _, err := vte.vtgateExecutor.Execute(context.Background(), nil, "VtexplainExecute", vtgate.NewSafeSession(vte.vtgateSession), sql, nil)
+ _, err := vte.vtgateExecutor.Execute(context.Background(), nil, "VtexplainExecute", econtext.NewSafeSession(vte.vtgateSession), sql, nil)
if err != nil {
for _, tc := range vte.explainTopo.TabletConns {
tc.tabletQueries = nil
diff --git a/go/vt/vtexplain/vtexplain_vttablet.go b/go/vt/vtexplain/vtexplain_vttablet.go
index ed977e7bcb0..65cd1a96181 100644
--- a/go/vt/vtexplain/vtexplain_vttablet.go
+++ b/go/vt/vtexplain/vtexplain_vttablet.go
@@ -22,6 +22,7 @@ import (
"reflect"
"strings"
"sync"
+ "time"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/vt/sidecardb"
@@ -113,8 +114,7 @@ func (vte *VTExplain) newTablet(ctx context.Context, env *vtenv.Environment, opt
config := tabletenv.NewCurrentConfig()
config.TrackSchemaVersions = false
if opts.ExecutionMode == ModeTwoPC {
- config.TwoPCAbandonAge = 1.0
- config.TwoPCEnable = true
+ config.TwoPCAbandonAge = 1 * time.Second
}
config.EnableOnlineDDL = false
config.EnableTableGC = false
@@ -232,7 +232,7 @@ func (t *explainTablet) CreateTransaction(ctx context.Context, target *querypb.T
}
// StartCommit is part of the QueryService interface.
-func (t *explainTablet) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
+func (t *explainTablet) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error) {
t.mu.Lock()
t.currentTime = t.vte.batchTime.Wait()
t.mu.Unlock()
@@ -430,6 +430,7 @@ func newTabletEnvironment(ddls []sqlparser.DDLStatement, opts *Options, collatio
showTableRows := make([][]sqltypes.Value, 0, len(ddls))
showTableWithSizesRows := make([][]sqltypes.Value, 0, len(ddls))
+ innodbTableSizesRows := make([][]sqltypes.Value, 0, len(ddls))
for _, ddl := range ddls {
table := ddl.GetTable().Name.String()
@@ -455,9 +456,9 @@ func newTabletEnvironment(ddls []sqlparser.DDLStatement, opts *Options, collatio
Fields: mysql.BaseShowTablesWithSizesFields,
Rows: showTableWithSizesRows,
})
- tEnv.addResult(mysql.TablesWithSize80, &sqltypes.Result{
- Fields: mysql.BaseShowTablesWithSizesFields,
- Rows: showTableWithSizesRows,
+ tEnv.addResult(mysql.InnoDBTableSizes, &sqltypes.Result{
+ Fields: mysql.BaseInnoDBTableSizesFields,
+ Rows: innodbTableSizesRows,
})
indexRows := make([][]sqltypes.Value, 0, 4)
diff --git a/go/vt/vtgate/autocommit_test.go b/go/vt/vtgate/autocommit_test.go
index 1ba99c01ef2..2e65cefbabe 100644
--- a/go/vt/vtgate/autocommit_test.go
+++ b/go/vt/vtgate/autocommit_test.go
@@ -23,10 +23,10 @@ import (
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/sqltypes"
-
querypb "vitess.io/vitess/go/vt/proto/query"
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
)
// This file contains tests for all the autocommit code paths
@@ -382,7 +382,7 @@ func TestAutocommitTransactionStarted(t *testing.T) {
// single shard query - no savepoint needed
sql := "update `user` set a = 2 where id = 1"
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
require.NoError(t, err)
require.Len(t, sbc1.Queries, 1)
require.Equal(t, sql, sbc1.Queries[0].Sql)
@@ -394,7 +394,7 @@ func TestAutocommitTransactionStarted(t *testing.T) {
// multi shard query - savepoint needed
sql = "update `user` set a = 2 where id in (1, 4)"
expectedSql := "update `user` set a = 2 where id in ::__vals"
- _, err = executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ _, err = executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
require.NoError(t, err)
require.Len(t, sbc1.Queries, 2)
require.Contains(t, sbc1.Queries[0].Sql, "savepoint")
@@ -413,7 +413,7 @@ func TestAutocommitDirectTarget(t *testing.T) {
}
sql := "insert into `simple`(val) values ('val')"
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
require.NoError(t, err)
assertQueries(t, sbclookup, []*querypb.BoundQuery{{
@@ -434,7 +434,7 @@ func TestAutocommitDirectRangeTarget(t *testing.T) {
}
sql := "delete from sharded_user_msgs limit 1000"
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
require.NoError(t, err)
assertQueries(t, sbc1, []*querypb.BoundQuery{{
@@ -451,5 +451,5 @@ func autocommitExec(executor *Executor, sql string) (*sqltypes.Result, error) {
TransactionMode: vtgatepb.TransactionMode_MULTI,
}
- return executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ return executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
}
diff --git a/go/vt/vtgate/balancer/balancer.go b/go/vt/vtgate/balancer/balancer.go
new file mode 100644
index 00000000000..bfe85194c05
--- /dev/null
+++ b/go/vt/vtgate/balancer/balancer.go
@@ -0,0 +1,367 @@
+/*
+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 balancer
+
+import (
+ "encoding/json"
+ "fmt"
+ "math/rand/v2"
+ "net/http"
+ "sync"
+
+ "vitess.io/vitess/go/vt/discovery"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+)
+
+/*
+
+The tabletBalancer probabalistically orders the list of available tablets into
+a ranked order of preference in order to satisfy two high-level goals:
+
+1. Balance the load across the available replicas
+2. Prefer a replica in the same cell as the vtgate if possible
+
+In some topologies this is trivial to accomplish by simply preferring tablets in the
+local cell, assuming there are a proportional number of local tablets in each cell to
+satisfy the inbound traffic to the vtgates in that cell.
+
+However, for topologies with a relatively small number of tablets in each cell, a simple
+affinity algorithm does not effectively balance the load.
+
+As a simple example:
+
+ Given three cells with vtgates, four replicas spread into those cells, where each vtgate
+ receives an equal query share. If each routes only to its local cell, the tablets will be
+ unbalanced since two of them receive 1/3 of the queries, but the two replicas in the same
+ cell will only receive 1/6 of the queries.
+
+ Cell A: 1/3 --> vtgate --> 1/3 => vttablet
+
+ Cell B: 1/3 --> vtgate --> 1/3 => vttablet
+
+ Cell C: 1/3 --> vtgate --> 1/6 => vttablet
+ \-> 1/6 => vttablet
+
+Other topologies that can cause similar pathologies include cases where there may be cells
+containing replicas but no local vtgates, and/or cells that have only vtgates but no replicas.
+
+For these topologies, the tabletBalancer proportionally assigns the output flow to each tablet,
+preferring the local cell where possible, but only as long as the global query balance is
+maintained.
+
+To accomplish this goal, the balancer is given:
+
+* The list of cells that receive inbound traffic to vtgates
+* The local cell where the vtgate exists
+* The set of tablets and their cells (learned from discovery)
+
+The model assumes there is an equal probablility of a query coming from each vtgate cell, i.e.
+traffic is effectively load balanced between the cells with vtgates.
+
+Given that information, the balancer builds a simple model to determine how much query load
+would go to each tablet if vtgate only routed to its local cell. Then if any tablets are
+unbalanced, it shifts the desired allocation away from the local cell preference in order to
+even out the query load.
+
+Based on this global model, the vtgate then probabalistically picks a destination for each
+query to be sent and uses these weights to order the available tablets accordingly.
+
+Assuming each vtgate is configured with and discovers the same information about the topology,
+and the input flow is balanced across the vtgate cells (as mentioned above), then each vtgate
+should come the the same conclusion about the global flows, and cooperatively should
+converge on the desired balanced query load.
+
+*/
+
+type TabletBalancer interface {
+ // Pick is the main entry point to the balancer. Returns the best tablet out of the list
+ // for a given query to maintain the desired balanced allocation over multiple executions.
+ Pick(target *querypb.Target, tablets []*discovery.TabletHealth) *discovery.TabletHealth
+
+ // DebugHandler provides a summary of tablet balancer state
+ DebugHandler(w http.ResponseWriter, r *http.Request)
+}
+
+func NewTabletBalancer(localCell string, vtGateCells []string) TabletBalancer {
+ return &tabletBalancer{
+ localCell: localCell,
+ vtGateCells: vtGateCells,
+ allocations: map[discovery.KeyspaceShardTabletType]*targetAllocation{},
+ }
+}
+
+type tabletBalancer struct {
+ //
+ // Configuration
+ //
+
+ // The local cell for the vtgate
+ localCell string
+
+ // The set of cells that have vtgates
+ vtGateCells []string
+
+ // mu protects the allocation map
+ mu sync.Mutex
+
+ //
+ // Allocations for balanced mode, calculated once per target and invalidated
+ // whenever the topology changes.
+ //
+ allocations map[discovery.KeyspaceShardTabletType]*targetAllocation
+}
+
+type targetAllocation struct {
+ // Target flow per cell based on the number of tablets discovered in the cell
+ Target map[string]int // json:target
+
+ // Input flows allocated for each cell
+ Inflows map[string]int
+
+ // Output flows from each vtgate cell to each target cell
+ Outflows map[string]map[string]int
+
+ // Allocation routed to each tablet from the local cell used for ranking
+ Allocation map[uint32]int
+
+ // Tablets that local cell does not route to
+ Unallocated map[uint32]struct{}
+
+ // Total allocation which is basically 1,000,000 / len(vtgatecells)
+ TotalAllocation int
+}
+
+func (b *tabletBalancer) print() string {
+ allocations, _ := json.Marshal(&b.allocations)
+ return fmt.Sprintf("LocalCell: %s, VtGateCells: %s, allocations: %s",
+ b.localCell, b.vtGateCells, string(allocations))
+}
+
+func (b *tabletBalancer) DebugHandler(w http.ResponseWriter, _ *http.Request) {
+ w.Header().Set("Content-Type", "text/plain")
+ fmt.Fprintf(w, "Local Cell: %v\r\n", b.localCell)
+ fmt.Fprintf(w, "Vtgate Cells: %v\r\n", b.vtGateCells)
+
+ b.mu.Lock()
+ defer b.mu.Unlock()
+ allocations, _ := json.MarshalIndent(b.allocations, "", " ")
+ fmt.Fprintf(w, "Allocations: %v\r\n", string(allocations))
+}
+
+// Pick is the main entry point to the balancer.
+//
+// Given the total allocation for the set of tablets, choose the best target
+// by a weighted random sample so that over time the system will achieve the
+// desired balanced allocation.
+func (b *tabletBalancer) Pick(target *querypb.Target, tablets []*discovery.TabletHealth) *discovery.TabletHealth {
+
+ numTablets := len(tablets)
+ if numTablets == 0 {
+ return nil
+ }
+
+ allocationMap, totalAllocation := b.getAllocation(target, tablets)
+
+ r := rand.IntN(totalAllocation)
+ for i := 0; i < numTablets; i++ {
+ flow := allocationMap[tablets[i].Tablet.Alias.Uid]
+ if r < flow {
+ return tablets[i]
+ }
+ r -= flow
+ }
+
+ return tablets[0]
+}
+
+// To stick with integer arithmetic, use 1,000,000 as the full load
+const ALLOCATION = 1000000
+
+func (b *tabletBalancer) allocateFlows(allTablets []*discovery.TabletHealth) *targetAllocation {
+ // Initialization: Set up some data structures and derived values
+ a := targetAllocation{
+ Target: map[string]int{},
+ Inflows: map[string]int{},
+ Outflows: map[string]map[string]int{},
+ Allocation: map[uint32]int{},
+ Unallocated: map[uint32]struct{}{},
+ }
+ flowPerVtgateCell := ALLOCATION / len(b.vtGateCells)
+ flowPerTablet := ALLOCATION / len(allTablets)
+ cellExistsWithNoTablets := false
+
+ for _, th := range allTablets {
+ a.Target[th.Tablet.Alias.Cell] += flowPerTablet
+ }
+
+ //
+ // First pass: Allocate vtgate flow to the local cell where the vtgate exists
+ // and along the way figure out if there are any vtgates with no local tablets.
+ //
+ for _, cell := range b.vtGateCells {
+ outflow := map[string]int{}
+ target := a.Target[cell]
+
+ if target > 0 {
+ a.Inflows[cell] += flowPerVtgateCell
+ outflow[cell] = flowPerVtgateCell
+ } else {
+ cellExistsWithNoTablets = true
+ }
+
+ a.Outflows[cell] = outflow
+ }
+
+ //
+ // Figure out if there is a shortfall
+ //
+ underAllocated := make(map[string]int)
+ unbalancedFlow := 0
+ for cell, allocation := range a.Target {
+ if a.Inflows[cell] < allocation {
+ underAllocated[cell] = allocation - a.Inflows[cell]
+ unbalancedFlow += underAllocated[cell]
+ }
+ }
+
+ //
+ // Second pass: if there are any vtgates with no local tablets, allocate the underallocated amount
+ // proportionally to all cells that may need it
+ //
+ if cellExistsWithNoTablets {
+ for _, vtgateCell := range b.vtGateCells {
+ target := a.Target[vtgateCell]
+ if target != 0 {
+ continue
+ }
+
+ for underAllocatedCell, underAllocatedFlow := range underAllocated {
+ allocation := flowPerVtgateCell * underAllocatedFlow / unbalancedFlow
+ a.Inflows[underAllocatedCell] += allocation
+ a.Outflows[vtgateCell][underAllocatedCell] += allocation
+ }
+ }
+
+ // Recompute underallocated after these flows were assigned
+ unbalancedFlow = 0
+ underAllocated = make(map[string]int)
+ for cell, allocation := range a.Target {
+ if a.Inflows[cell] < allocation {
+ underAllocated[cell] = allocation - a.Inflows[cell]
+ unbalancedFlow += underAllocated[cell]
+ }
+ }
+ }
+
+ //
+ // Third pass: Shift remaining imbalance if any cell is over/under allocated after
+ // assigning local cell traffic and distributing load from cells without tablets.
+ //
+ if /* fudge for integer arithmetic */ unbalancedFlow > 10 {
+
+ // cells which are overallocated
+ overAllocated := make(map[string]int)
+ for cell, allocation := range a.Target {
+ if a.Inflows[cell] > allocation {
+ overAllocated[cell] = a.Inflows[cell] - allocation
+ }
+ }
+
+ // fmt.Printf("outflows %v over %v under %v\n", a.Outflows, overAllocated, underAllocated)
+
+ //
+ // For each overallocated cell, proportionally shift flow from targets that are overallocated
+ // to targets that are underallocated.
+ //
+ // Note this is an O(N^3) loop, but only over the cells which need adjustment.
+ //
+ for _, vtgateCell := range b.vtGateCells {
+ for underAllocatedCell, underAllocatedFlow := range underAllocated {
+ for overAllocatedCell, overAllocatedFlow := range overAllocated {
+
+ currentFlow := a.Outflows[vtgateCell][overAllocatedCell]
+ if currentFlow == 0 {
+ continue
+ }
+
+ // Shift a proportional fraction of the amount that the cell is currently allocated weighted
+ // by the fraction that this vtgate cell is already sending to the overallocated cell, and the
+ // fraction that the new target is underallocated
+ //
+ // Note that the operator order matters -- multiplications need to occur before divisions
+ // to avoid truncating the integer values.
+ shiftFlow := overAllocatedFlow * currentFlow * underAllocatedFlow / a.Inflows[overAllocatedCell] / unbalancedFlow
+
+ //fmt.Printf("shift %d %s %s -> %s (over %d current %d in %d under %d unbalanced %d) \n", shiftFlow, vtgateCell, overAllocatedCell, underAllocatedCell,
+ // overAllocatedFlow, currentFlow, a.Inflows[overAllocatedCell], underAllocatedFlow, unbalancedFlow)
+
+ a.Outflows[vtgateCell][overAllocatedCell] -= shiftFlow
+ a.Inflows[overAllocatedCell] -= shiftFlow
+
+ a.Inflows[underAllocatedCell] += shiftFlow
+ a.Outflows[vtgateCell][underAllocatedCell] += shiftFlow
+ }
+ }
+ }
+ }
+
+ //
+ // Finally, once the cell flows are all adjusted, figure out the local allocation to each
+ // tablet in the target cells
+ //
+ outflow := a.Outflows[b.localCell]
+ for _, tablet := range allTablets {
+ cell := tablet.Tablet.Alias.Cell
+ flow := outflow[cell]
+ if flow > 0 {
+ a.Allocation[tablet.Tablet.Alias.Uid] = flow * flowPerTablet / a.Target[cell]
+ a.TotalAllocation += flow * flowPerTablet / a.Target[cell]
+ } else {
+ a.Unallocated[tablet.Tablet.Alias.Uid] = struct{}{}
+ }
+ }
+
+ return &a
+}
+
+// getAllocation builds the allocation map if needed and returns a copy of the map
+func (b *tabletBalancer) getAllocation(target *querypb.Target, tablets []*discovery.TabletHealth) (map[uint32]int, int) {
+ b.mu.Lock()
+ defer b.mu.Unlock()
+
+ allocation, exists := b.allocations[discovery.KeyFromTarget(target)]
+ if exists && (len(allocation.Allocation)+len(allocation.Unallocated)) == len(tablets) {
+ mismatch := false
+ for _, tablet := range tablets {
+ if _, ok := allocation.Allocation[tablet.Tablet.Alias.Uid]; !ok {
+ if _, ok := allocation.Unallocated[tablet.Tablet.Alias.Uid]; !ok {
+ mismatch = true
+ break
+ }
+ }
+ }
+ if !mismatch {
+ // No change in tablets for this target. Return computed allocation
+ return allocation.Allocation, allocation.TotalAllocation
+ }
+ }
+
+ allocation = b.allocateFlows(tablets)
+ b.allocations[discovery.KeyFromTarget(target)] = allocation
+
+ return allocation.Allocation, allocation.TotalAllocation
+}
diff --git a/go/vt/vtgate/balancer/balancer_test.go b/go/vt/vtgate/balancer/balancer_test.go
new file mode 100644
index 00000000000..1c6a72421fc
--- /dev/null
+++ b/go/vt/vtgate/balancer/balancer_test.go
@@ -0,0 +1,371 @@
+/*
+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 balancer
+
+import (
+ "strconv"
+ "testing"
+
+ "github.com/stretchr/testify/assert"
+
+ "vitess.io/vitess/go/vt/discovery"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/topo"
+)
+
+var nextTestTabletUID int
+
+func createTestTablet(cell string) *discovery.TabletHealth {
+ nextTestTabletUID++
+ tablet := topo.NewTablet(uint32(nextTestTabletUID), cell, strconv.Itoa(nextTestTabletUID))
+ tablet.PortMap["vt"] = 1
+ tablet.PortMap["grpc"] = 2
+ tablet.Keyspace = "k"
+ tablet.Shard = "s"
+
+ return &discovery.TabletHealth{
+ Tablet: tablet,
+ Target: &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA},
+ Serving: false,
+ Stats: nil,
+ PrimaryTermStartTime: 0,
+ }
+}
+
+func TestAllocateFlows(t *testing.T) {
+ cases := []struct {
+ test string
+ tablets []*discovery.TabletHealth
+ vtgateCells []string
+ }{
+ {
+ "balanced one tablet per cell",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "balanced multiple tablets per cell",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "vtgate in cell with no tablets",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+ []string{"a", "b", "c", "d", "e"},
+ },
+ {
+ "vtgates in multiple cells with no tablets",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+ []string{"a", "b", "c", "d", "e", "f", "g"},
+ },
+ {
+ "imbalanced multiple tablets in one cell",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ },
+ []string{"a", "b", "c"},
+ },
+ {
+ "imbalanced multiple tablets in multiple cells",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ createTestTablet("d"),
+ createTestTablet("d"),
+ createTestTablet("d"),
+ },
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "heavy imbalance",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("c"),
+ },
+ []string{"a", "b", "c", "d"},
+ },
+ }
+
+ target := &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}
+
+ for _, c := range cases {
+ t.Logf("\n\nTest Case: %s\n\n", c.test)
+
+ tablets := c.tablets
+ vtGateCells := c.vtgateCells
+
+ tabletsByCell := make(map[string][]*discovery.TabletHealth)
+ for _, tablet := range tablets {
+ cell := tablet.Tablet.Alias.Cell
+ tabletsByCell[cell] = append(tabletsByCell[cell], tablet)
+ }
+
+ allocationPerTablet := make(map[uint32]int)
+ expectedPerTablet := ALLOCATION / len(tablets)
+
+ expectedPerCell := make(map[string]int)
+ for cell := range tabletsByCell {
+ expectedPerCell[cell] = ALLOCATION / len(tablets) * len(tabletsByCell[cell])
+ }
+
+ // Run the balancer over each vtgate cell
+ for _, localCell := range vtGateCells {
+ b := NewTabletBalancer(localCell, vtGateCells).(*tabletBalancer)
+ a := b.allocateFlows(tablets)
+ b.allocations[discovery.KeyFromTarget(target)] = a
+
+ t.Logf("Target Flows %v, Balancer: %s, Allocations: %v \n", expectedPerCell, b.print(), b.allocations)
+
+ // Accumulate all the output per tablet cell
+ outflowPerCell := make(map[string]int)
+ for _, outflow := range a.Outflows {
+ for tabletCell, flow := range outflow {
+ assert.GreaterOrEqual(t, flow, 0, b.print())
+ outflowPerCell[tabletCell] += flow
+ }
+ }
+
+ // Check in / out flow to each tablet cell
+ for cell := range tabletsByCell {
+ expectedForCell := expectedPerCell[cell]
+
+ assert.InEpsilonf(t, expectedForCell, a.Inflows[cell], 0.01,
+ "did not allocate correct inflow to cell %s. Balancer {%s} ExpectedPerCell {%v}",
+ cell, b.print(), expectedPerCell)
+ assert.InEpsilonf(t, expectedForCell, outflowPerCell[cell], 0.01,
+ "did not allocate correct outflow to cell %s. Balancer {%s} ExpectedPerCell {%v}",
+ cell, b.print(), expectedPerCell)
+ }
+
+ // Accumulate the allocations for all runs to compare what the system does as a whole
+ // when routing from all vtgate cells
+ for uid, flow := range a.Allocation {
+ allocationPerTablet[uid] += flow
+ }
+ }
+
+ // Check that the allocations all add up
+ for _, tablet := range tablets {
+ uid := tablet.Tablet.Alias.Uid
+
+ allocation := allocationPerTablet[uid]
+ assert.InEpsilonf(t, expectedPerTablet, allocation, 0.01,
+ "did not allocate full allocation to tablet %d", uid)
+ }
+ }
+}
+
+func TestBalancedPick(t *testing.T) {
+ cases := []struct {
+ test string
+ tablets []*discovery.TabletHealth
+ vtgateCells []string
+ }{
+ {
+ "simple balanced",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "simple unbalanced",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("d"),
+ },
+
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "mixed unbalanced",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("c"),
+ createTestTablet("c"),
+ },
+
+ []string{"a", "b", "c", "d"},
+ },
+ {
+ "one target same cell",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ },
+
+ []string{"a"},
+ },
+ {
+ "one target other cell",
+ []*discovery.TabletHealth{
+ createTestTablet("a"),
+ },
+
+ []string{"b", "c", "d"},
+ },
+ }
+
+ target := &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}
+ for _, c := range cases {
+ t.Logf("\n\nTest Case: %s\n\n", c.test)
+
+ tablets := c.tablets
+ vtGateCells := c.vtgateCells
+
+ // test unbalanced distribution
+
+ routed := make(map[uint32]int)
+
+ expectedPerCell := make(map[string]int)
+ for _, tablet := range tablets {
+ cell := tablet.Tablet.Alias.Cell
+ expectedPerCell[cell] += ALLOCATION / len(tablets)
+ }
+
+ // Run the algorithm a bunch of times to get a random enough sample
+ N := 1000000
+ for _, localCell := range vtGateCells {
+ b := NewTabletBalancer(localCell, vtGateCells).(*tabletBalancer)
+
+ for i := 0; i < N/len(vtGateCells); i++ {
+ th := b.Pick(target, tablets)
+ if i == 0 {
+ t.Logf("Target Flows %v, Balancer: %s\n", expectedPerCell, b.print())
+ }
+
+ routed[th.Tablet.Alias.Uid]++
+ }
+ }
+
+ expected := N / len(tablets)
+ delta := make(map[uint32]int)
+ for _, tablet := range tablets {
+ got := routed[tablet.Tablet.Alias.Uid]
+ delta[tablet.Tablet.Alias.Uid] = got - expected
+ assert.InEpsilonf(t, expected, got, 0.01,
+ "routing to tablet %d", tablet.Tablet.Alias.Uid)
+ }
+ }
+}
+
+func TestTopologyChanged(t *testing.T) {
+ allTablets := []*discovery.TabletHealth{
+ createTestTablet("a"),
+ createTestTablet("a"),
+ createTestTablet("b"),
+ createTestTablet("b"),
+ }
+ target := &querypb.Target{Keyspace: "k", Shard: "s", TabletType: topodatapb.TabletType_REPLICA}
+
+ b := NewTabletBalancer("b", []string{"a", "b"}).(*tabletBalancer)
+
+ N := 1
+
+ // initially create a slice of tablets with just the two in cell a
+ tablets := allTablets
+ tablets = tablets[0:2]
+
+ for i := 0; i < N; i++ {
+ th := b.Pick(target, tablets)
+ allocation, totalAllocation := b.getAllocation(target, tablets)
+
+ assert.Equalf(t, ALLOCATION/2, totalAllocation, "totalAllocation mismatch %s", b.print())
+ assert.Equalf(t, ALLOCATION/4, allocation[th.Tablet.Alias.Uid], "allocation mismatch %s, cell %s", b.print(), allTablets[0].Tablet.Alias.Cell)
+ assert.Equalf(t, "a", th.Tablet.Alias.Cell, "shuffle promoted wrong tablet from cell %s", allTablets[0].Tablet.Alias.Cell)
+ }
+
+ // Run again with the full topology. Now traffic should go to cell b
+ for i := 0; i < N; i++ {
+ th := b.Pick(target, allTablets)
+
+ allocation, totalAllocation := b.getAllocation(target, allTablets)
+
+ assert.Equalf(t, ALLOCATION/2, totalAllocation, "totalAllocation mismatch %s", b.print())
+ assert.Equalf(t, ALLOCATION/4, allocation[th.Tablet.Alias.Uid], "allocation mismatch %s, cell %s", b.print(), allTablets[0].Tablet.Alias.Cell)
+ assert.Equalf(t, "b", th.Tablet.Alias.Cell, "shuffle promoted wrong tablet from cell %s", allTablets[0].Tablet.Alias.Cell)
+ }
+
+ // Run again with a node in the topology replaced.
+ newTablet := createTestTablet("b")
+ allTablets[2] = newTablet
+ for i := 0; i < N; i++ {
+ th := b.Pick(target, allTablets)
+
+ allocation, totalAllocation := b.getAllocation(target, allTablets)
+
+ assert.Equalf(t, ALLOCATION/2, totalAllocation, "totalAllocation mismatch %s", b.print())
+ assert.Equalf(t, ALLOCATION/4, allocation[th.Tablet.Alias.Uid], "allocation mismatch %s, cell %s", b.print(), allTablets[0].Tablet.Alias.Cell)
+ assert.Equalf(t, "b", th.Tablet.Alias.Cell, "shuffle promoted wrong tablet from cell %s", allTablets[0].Tablet.Alias.Cell)
+ }
+
+}
diff --git a/go/vt/vtgate/buffer/buffer.go b/go/vt/vtgate/buffer/buffer.go
index 260fb272544..eb937a6361c 100644
--- a/go/vt/vtgate/buffer/buffer.go
+++ b/go/vt/vtgate/buffer/buffer.go
@@ -94,6 +94,18 @@ func CausedByFailover(err error) bool {
return isFailover
}
+// isErrorDueToReparenting is a stronger check than CausedByFailover, meant to return
+// if the failure is caused because of a reparent.
+func isErrorDueToReparenting(err error) bool {
+ if vterrors.Code(err) != vtrpcpb.Code_CLUSTER_EVENT {
+ return false
+ }
+ if strings.Contains(err.Error(), ClusterEventReshardingInProgress) {
+ return false
+ }
+ return true
+}
+
// for debugging purposes
func getReason(err error) string {
for _, ce := range ClusterEvents {
@@ -175,7 +187,7 @@ func (b *Buffer) GetConfig() *Config {
// It returns an error if buffering failed (e.g. buffer full).
// If it does not return an error, it may return a RetryDoneFunc which must be
// called after the request was retried.
-func (b *Buffer) WaitForFailoverEnd(ctx context.Context, keyspace, shard string, err error) (RetryDoneFunc, error) {
+func (b *Buffer) WaitForFailoverEnd(ctx context.Context, keyspace, shard string, kev *discovery.KeyspaceEventWatcher, err error) (RetryDoneFunc, error) {
// If an err is given, it must be related to a failover.
// We never buffer requests with other errors.
if err != nil && !CausedByFailover(err) {
@@ -192,10 +204,11 @@ func (b *Buffer) WaitForFailoverEnd(ctx context.Context, keyspace, shard string,
requestsSkipped.Add([]string{keyspace, shard, skippedDisabled}, 1)
return nil, nil
}
- return sb.waitForFailoverEnd(ctx, keyspace, shard, err)
+ return sb.waitForFailoverEnd(ctx, keyspace, shard, kev, err)
}
func (b *Buffer) HandleKeyspaceEvent(ksevent *discovery.KeyspaceEvent) {
+ log.Infof("Keyspace Event received for keyspace %v", ksevent.Keyspace)
for _, shard := range ksevent.Shards {
sb := b.getOrCreateBuffer(shard.Target.Keyspace, shard.Target.Shard)
if sb != nil {
diff --git a/go/vt/vtgate/buffer/buffer_helper_test.go b/go/vt/vtgate/buffer/buffer_helper_test.go
index 2deb460fc39..1276f0cd751 100644
--- a/go/vt/vtgate/buffer/buffer_helper_test.go
+++ b/go/vt/vtgate/buffer/buffer_helper_test.go
@@ -50,7 +50,7 @@ func issueRequestAndBlockRetry(ctx context.Context, t *testing.T, b *Buffer, err
bufferingStopped := make(chan error)
go func() {
- retryDone, err := b.WaitForFailoverEnd(ctx, keyspace, shard, failoverErr)
+ retryDone, err := b.WaitForFailoverEnd(ctx, keyspace, shard, nil, failoverErr)
if err != nil {
bufferingStopped <- err
}
diff --git a/go/vt/vtgate/buffer/buffer_test.go b/go/vt/vtgate/buffer/buffer_test.go
index c730a8336d1..fc326ce0ce5 100644
--- a/go/vt/vtgate/buffer/buffer_test.go
+++ b/go/vt/vtgate/buffer/buffer_test.go
@@ -72,6 +72,32 @@ var (
}
)
+func TestIsErrorDueToReparenting(t *testing.T) {
+ testcases := []struct {
+ err error
+ want bool
+ }{
+ {
+ err: vterrors.Errorf(vtrpcpb.Code_CLUSTER_EVENT, ClusterEventReshardingInProgress),
+ want: false,
+ },
+ {
+ err: vterrors.Errorf(vtrpcpb.Code_CLUSTER_EVENT, ClusterEventReparentInProgress),
+ want: true,
+ },
+ {
+ err: vterrors.Errorf(vtrpcpb.Code_CLUSTER_EVENT, "The MySQL server is running with the --super-read-only option"),
+ want: true,
+ },
+ }
+ for _, tt := range testcases {
+ t.Run(tt.err.Error(), func(t *testing.T) {
+ got := isErrorDueToReparenting(tt.err)
+ assert.Equal(t, tt.want, got)
+ })
+ }
+}
+
func TestBuffering(t *testing.T) {
testAllImplementations(t, func(t *testing.T, fail failover) {
testBuffering1WithOptions(t, fail, 1)
@@ -120,7 +146,7 @@ func testBuffering1WithOptions(t *testing.T, fail failover, concurrency int) {
}
// Subsequent requests with errors not related to the failover are not buffered.
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nonFailoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, nonFailoverErr); err != nil || retryDone != nil {
t.Fatalf("requests with non-failover errors must never be buffered. err: %v retryDone: %v", err, retryDone)
}
@@ -168,7 +194,7 @@ func testBuffering1WithOptions(t *testing.T, fail failover, concurrency int) {
}
// Second failover: Buffering is skipped because last failover is too recent.
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("subsequent failovers must be skipped due to -buffer_min_time_between_failovers setting. err: %v retryDone: %v", err, retryDone)
}
if got, want := requestsSkipped.Counts()[statsKeyJoinedLastFailoverTooRecent], int64(1); got != want {
@@ -226,7 +252,7 @@ func testDryRun1(t *testing.T, fail failover) {
b := New(cfg)
// Request does not get buffered.
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("requests must not be buffered during dry-run. err: %v retryDone: %v", err, retryDone)
}
// But the internal state changes though.
@@ -272,10 +298,10 @@ func testPassthrough1(t *testing.T, fail failover) {
b := New(cfg)
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, nil); err != nil || retryDone != nil {
t.Fatalf("requests with no error must never be buffered. err: %v retryDone: %v", err, retryDone)
}
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nonFailoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, nonFailoverErr); err != nil || retryDone != nil {
t.Fatalf("requests with non-failover errors must never be buffered. err: %v retryDone: %v", err, retryDone)
}
@@ -311,7 +337,7 @@ func testLastReparentTooRecentBufferingSkipped1(t *testing.T, fail failover) {
now = now.Add(1 * time.Second)
fail(b, newPrimary, keyspace, shard, now)
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("requests where the failover end was recently detected before the start must not be buffered. err: %v retryDone: %v", err, retryDone)
}
if err := waitForPoolSlots(b, cfg.Size); err != nil {
@@ -408,10 +434,10 @@ func testPassthroughDuringDrain1(t *testing.T, fail failover) {
}
// Requests during the drain will be passed through and not buffered.
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, nil); err != nil || retryDone != nil {
t.Fatalf("requests with no error must not be buffered during a drain. err: %v retryDone: %v", err, retryDone)
}
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, shard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("requests with failover errors must not be buffered during a drain. err: %v retryDone: %v", err, retryDone)
}
@@ -443,7 +469,7 @@ func testPassthroughIgnoredKeyspaceOrShard1(t *testing.T, fail failover) {
b := New(cfg)
ignoredKeyspace := "ignored_ks"
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), ignoredKeyspace, shard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), ignoredKeyspace, shard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("requests for ignored keyspaces must not be buffered. err: %v retryDone: %v", err, retryDone)
}
statsKeyJoined := strings.Join([]string{ignoredKeyspace, shard, skippedDisabled}, ".")
@@ -452,7 +478,7 @@ func testPassthroughIgnoredKeyspaceOrShard1(t *testing.T, fail failover) {
}
ignoredShard := "ff-"
- if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, ignoredShard, failoverErr); err != nil || retryDone != nil {
+ if retryDone, err := b.WaitForFailoverEnd(context.Background(), keyspace, ignoredShard, nil, failoverErr); err != nil || retryDone != nil {
t.Fatalf("requests for ignored shards must not be buffered. err: %v retryDone: %v", err, retryDone)
}
if err := waitForPoolSlots(b, cfg.Size); err != nil {
@@ -634,7 +660,7 @@ func testEvictionNotPossible1(t *testing.T, fail failover) {
// Newer requests of the second failover cannot evict anything because
// they have no entries buffered.
- retryDone, bufferErr := b.WaitForFailoverEnd(context.Background(), keyspace, shard2, failoverErr)
+ retryDone, bufferErr := b.WaitForFailoverEnd(context.Background(), keyspace, shard2, nil, failoverErr)
if bufferErr == nil || retryDone != nil {
t.Fatalf("buffer should have returned an error because it's full: err: %v retryDone: %v", bufferErr, retryDone)
}
diff --git a/go/vt/vtgate/buffer/shard_buffer.go b/go/vt/vtgate/buffer/shard_buffer.go
index b0764c2ad91..66c6ee702e6 100644
--- a/go/vt/vtgate/buffer/shard_buffer.go
+++ b/go/vt/vtgate/buffer/shard_buffer.go
@@ -137,7 +137,7 @@ func (sb *shardBuffer) disabled() bool {
return sb.mode == bufferModeDisabled
}
-func (sb *shardBuffer) waitForFailoverEnd(ctx context.Context, keyspace, shard string, err error) (RetryDoneFunc, error) {
+func (sb *shardBuffer) waitForFailoverEnd(ctx context.Context, keyspace, shard string, kev *discovery.KeyspaceEventWatcher, err error) (RetryDoneFunc, error) {
// We assume if err != nil then it's always caused by a failover.
// Other errors must be filtered at higher layers.
failoverDetected := err != nil
@@ -211,7 +211,11 @@ func (sb *shardBuffer) waitForFailoverEnd(ctx context.Context, keyspace, shard s
return nil, nil
}
- sb.startBufferingLocked(err)
+ // Try to start buffering. If we're unsuccessful, then we exit early.
+ if !sb.startBufferingLocked(ctx, kev, err) {
+ sb.mu.Unlock()
+ return nil, nil
+ }
}
if sb.mode == bufferModeDryRun {
@@ -255,7 +259,16 @@ func (sb *shardBuffer) shouldBufferLocked(failoverDetected bool) bool {
panic("BUG: All possible states must be covered by the switch expression above.")
}
-func (sb *shardBuffer) startBufferingLocked(err error) {
+func (sb *shardBuffer) startBufferingLocked(ctx context.Context, kev *discovery.KeyspaceEventWatcher, err error) bool {
+ if kev != nil {
+ if !kev.MarkShardNotServing(ctx, sb.keyspace, sb.shard, isErrorDueToReparenting(err)) {
+ // We failed to mark the shard as not serving. Do not buffer the request.
+ // This can happen if the keyspace has been deleted or if the keyspace even watcher
+ // hasn't yet seen the shard. Keyspace event watcher might not stop buffering for this
+ // request at all until it times out. It's better to not buffer this request.
+ return false
+ }
+ }
// Reset monitoring data from previous failover.
lastRequestsInFlightMax.Set(sb.statsKey, 0)
lastRequestsDryRunMax.Set(sb.statsKey, 0)
@@ -273,7 +286,7 @@ func (sb *shardBuffer) startBufferingLocked(err error) {
msg = "Dry-run: Would have started buffering"
}
starts.Add(sb.statsKey, 1)
- log.Infof("%v for shard: %s (window: %v, size: %v, max failover duration: %v) (A failover was detected by this seen error: %v.)",
+ log.V(2).Infof("%v for shard: %s (window: %v, size: %v, max failover duration: %v) (A failover was detected by this seen error: %v.)",
msg,
topoproto.KeyspaceShardString(sb.keyspace, sb.shard),
sb.buf.config.Window,
@@ -281,6 +294,7 @@ func (sb *shardBuffer) startBufferingLocked(err error) {
sb.buf.config.MaxFailoverDuration,
errorsanitizer.NormalizeError(err.Error()),
)
+ return true
}
// logErrorIfStateNotLocked logs an error if the current state is not "state".
@@ -474,7 +488,7 @@ func (sb *shardBuffer) recordKeyspaceEvent(alias *topodatapb.TabletAlias, stillS
sb.mu.Lock()
defer sb.mu.Unlock()
- log.Infof("disruption in shard %s/%s resolved (serving: %v), movetable state %#v",
+ log.V(2).Infof("disruption in shard %s/%s resolved (serving: %v), movetable state %#v",
sb.keyspace, sb.shard, stillServing, keyspaceEvent.MoveTablesState)
if !topoproto.TabletAliasEqual(alias, sb.currentPrimary) {
@@ -548,7 +562,7 @@ func (sb *shardBuffer) stopBufferingLocked(reason stopReason, details string) {
if sb.mode == bufferModeDryRun {
msg = "Dry-run: Would have stopped buffering"
}
- log.Infof("%v for shard: %s after: %.1f seconds due to: %v. Draining %d buffered requests now.",
+ log.V(2).Infof("%v for shard: %s after: %.1f seconds due to: %v. Draining %d buffered requests now.",
msg, topoproto.KeyspaceShardString(sb.keyspace, sb.shard), d.Seconds(), details, len(q))
var clientEntryError error
@@ -608,7 +622,7 @@ func (sb *shardBuffer) drain(q []*entry, err error) {
wg.Wait()
d := sb.timeNow().Sub(start)
- log.Infof("Draining finished for shard: %s Took: %v for: %d requests.", topoproto.KeyspaceShardString(sb.keyspace, sb.shard), d, len(q))
+ log.V(2).Infof("Draining finished for shard: %s Took: %v for: %d requests.", topoproto.KeyspaceShardString(sb.keyspace, sb.shard), d, len(q))
requestsDrained.Add(sb.statsKey, int64(len(q)))
// Draining is done. Change state from "draining" to "idle".
diff --git a/go/vt/vtgate/buffer/variables_test.go b/go/vt/vtgate/buffer/variables_test.go
index a0640bde9e4..30d2426c639 100644
--- a/go/vt/vtgate/buffer/variables_test.go
+++ b/go/vt/vtgate/buffer/variables_test.go
@@ -51,7 +51,7 @@ func TestVariablesAreInitialized(t *testing.T) {
// Create a new buffer and make a call which will create the shardBuffer object.
// After that, the variables should be initialized for that shard.
b := New(NewDefaultConfig())
- _, err := b.WaitForFailoverEnd(context.Background(), "init_test", "0", nil /* err */)
+ _, err := b.WaitForFailoverEnd(context.Background(), "init_test", "0", nil, nil)
if err != nil {
t.Fatalf("buffer should just passthrough and not return an error: %v", err)
}
diff --git a/go/vt/vtgate/debugenv.go b/go/vt/vtgate/debugenv.go
index 4fa989c69a3..7213353432d 100644
--- a/go/vt/vtgate/debugenv.go
+++ b/go/vt/vtgate/debugenv.go
@@ -22,9 +22,10 @@ import (
"html"
"net/http"
"strconv"
- "text/template"
"time"
+ "github.com/google/safehtml/template"
+
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/vt/discovery"
"vitess.io/vitess/go/vt/log"
diff --git a/go/vt/vtgate/dynamicconfig/config.go b/go/vt/vtgate/dynamicconfig/config.go
new file mode 100644
index 00000000000..5bb1d991eae
--- /dev/null
+++ b/go/vt/vtgate/dynamicconfig/config.go
@@ -0,0 +1,6 @@
+package dynamicconfig
+
+type DDL interface {
+ OnlineEnabled() bool
+ DirectEnabled() bool
+}
diff --git a/go/vt/vtgate/endtoend/aggr_test.go b/go/vt/vtgate/endtoend/aggr_test.go
index b37697a72f4..402cecc0c6d 100644
--- a/go/vt/vtgate/endtoend/aggr_test.go
+++ b/go/vt/vtgate/endtoend/aggr_test.go
@@ -21,37 +21,35 @@ import (
"fmt"
"testing"
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/test/utils"
+
"vitess.io/vitess/go/mysql"
)
func TestAggregateTypes(t *testing.T) {
ctx := context.Background()
conn, err := mysql.Connect(ctx, &vtParams)
- if err != nil {
- t.Fatal(err)
- }
+ require.NoError(t, err)
defer conn.Close()
exec(t, conn, "insert into aggr_test(id, val1, val2) values(1,'a',1), (2,'A',1), (3,'b',1), (4,'c',3), (5,'c',4)")
exec(t, conn, "insert into aggr_test(id, val1, val2) values(6,'d',null), (7,'e',null), (8,'E',1)")
qr := exec(t, conn, "select val1, count(distinct val2), count(*) from aggr_test group by val1")
- if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`; got != want {
- t.Errorf("select:\n%v want\n%v", got, want)
- }
+ want := `[[VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`
+ utils.MustMatch(t, want, fmt.Sprintf("%v", qr.Rows))
qr = exec(t, conn, "select val1, sum(distinct val2), sum(val2) from aggr_test group by val1")
- if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("a") DECIMAL(1) DECIMAL(2)] [VARCHAR("b") DECIMAL(1) DECIMAL(1)] [VARCHAR("c") DECIMAL(7) DECIMAL(7)] [VARCHAR("d") NULL NULL] [VARCHAR("e") DECIMAL(1) DECIMAL(1)]]`; got != want {
- t.Errorf("select:\n%v want\n%v", got, want)
- }
+ want = `[[VARCHAR("a") DECIMAL(1) DECIMAL(2)] [VARCHAR("b") DECIMAL(1) DECIMAL(1)] [VARCHAR("c") DECIMAL(7) DECIMAL(7)] [VARCHAR("d") NULL NULL] [VARCHAR("e") DECIMAL(1) DECIMAL(1)]]`
+ utils.MustMatch(t, want, fmt.Sprintf("%v", qr.Rows))
qr = exec(t, conn, "select val1, count(distinct val2) k, count(*) from aggr_test group by val1 order by k desc, val1")
- if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)]]`; got != want {
- t.Errorf("select:\n%v want\n%v", got, want)
- }
+ want = `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)]]`
+ utils.MustMatch(t, want, fmt.Sprintf("%v", qr.Rows))
qr = exec(t, conn, "select val1, count(distinct val2) k, count(*) from aggr_test group by val1 order by k desc, val1 limit 4")
- if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`; got != want {
- t.Errorf("select:\n%v want\n%v", got, want)
- }
+ want = `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`
+ utils.MustMatch(t, want, fmt.Sprintf("%v", qr.Rows))
}
diff --git a/go/vt/vtgate/endtoend/vstream_test.go b/go/vt/vtgate/endtoend/vstream_test.go
index 246d17f88b5..8fed95f5d51 100644
--- a/go/vt/vtgate/endtoend/vstream_test.go
+++ b/go/vt/vtgate/endtoend/vstream_test.go
@@ -60,6 +60,7 @@ func initialize(ctx context.Context, t *testing.T) (*vtgateconn.VTGateConn, *mys
}
return gconn, conn, mconn, close
}
+
func TestVStream(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@@ -186,7 +187,7 @@ func TestVStreamCopyBasic(t *testing.T) {
Lastpk: qr,
}}
var shardGtids []*binlogdatapb.ShardGtid
- var vgtid = &binlogdatapb.VGtid{}
+ vgtid := &binlogdatapb.VGtid{}
shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{
Keyspace: "ks",
Shard: "-80",
@@ -264,20 +265,14 @@ func TestVStreamCopyUnspecifiedShardGtid(t *testing.T) {
defer cancel()
conn, err := mysql.Connect(ctx, &vtParams)
- if err != nil {
- require.NoError(t, err)
- }
+ require.NoError(t, err)
defer conn.Close()
_, err = conn.ExecuteFetch("insert into t1_copy_all(id1,id2) values(1,1), (2,2), (3,3), (4,4), (5,5), (6,6), (7,7), (8,8)", 1, false)
- if err != nil {
- require.NoError(t, err)
- }
+ require.NoError(t, err)
_, err = conn.ExecuteFetch("insert into t1_copy_all_ks2(id1,id2) values(10,10), (20,20)", 1, false)
- if err != nil {
- require.NoError(t, err)
- }
+ require.NoError(t, err)
filter := &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{{
@@ -343,13 +338,11 @@ func TestVStreamCopyUnspecifiedShardGtid(t *testing.T) {
gconn, conn, mconn, closeConnections := initialize(ctx, t)
defer closeConnections()
- var vgtid = &binlogdatapb.VGtid{}
+ vgtid := &binlogdatapb.VGtid{}
vgtid.ShardGtids = []*binlogdatapb.ShardGtid{c.shardGtid}
reader, err := gconn.VStream(ctx, topodatapb.TabletType_PRIMARY, vgtid, filter, flags)
_, _ = conn, mconn
- if err != nil {
- require.NoError(t, err)
- }
+ require.NoError(t, err)
require.NotNil(t, reader)
var evs []*binlogdatapb.VEvent
var completedEvs []*binlogdatapb.VEvent
@@ -426,7 +419,7 @@ func TestVStreamCopyResume(t *testing.T) {
}
var shardGtids []*binlogdatapb.ShardGtid
- var vgtid = &binlogdatapb.VGtid{}
+ vgtid := &binlogdatapb.VGtid{}
shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{
Keyspace: "ks",
Shard: "-80",
@@ -526,7 +519,7 @@ func TestVStreamCurrent(t *testing.T) {
defer closeConnections()
var shardGtids []*binlogdatapb.ShardGtid
- var vgtid = &binlogdatapb.VGtid{}
+ vgtid := &binlogdatapb.VGtid{}
shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{
Keyspace: "ks",
Shard: "-80",
@@ -580,7 +573,7 @@ func TestVStreamSharded(t *testing.T) {
defer closeConnections()
var shardGtids []*binlogdatapb.ShardGtid
- var vgtid = &binlogdatapb.VGtid{}
+ vgtid := &binlogdatapb.VGtid{}
shardGtids = append(shardGtids, &binlogdatapb.ShardGtid{
Keyspace: "ks",
Shard: "-80",
@@ -665,7 +658,6 @@ func TestVStreamSharded(t *testing.T) {
t.Fatalf("remote error: %v\n", err)
}
}
-
}
// TestVStreamCopyTransactions tests that we are properly wrapping
@@ -822,9 +814,11 @@ type VEventSorter []*binlogdatapb.VEvent
func (v VEventSorter) Len() int {
return len(v)
}
+
func (v VEventSorter) Swap(i, j int) {
v[i], v[j] = v[j], v[i]
}
+
func (v VEventSorter) Less(i, j int) bool {
valsI := v[i].GetRowEvent().RowChanges[0].After
if valsI == nil {
diff --git a/go/vt/vtgate/engine/cached_size.go b/go/vt/vtgate/engine/cached_size.go
index af9780fe001..c764a6aab08 100644
--- a/go/vt/vtgate/engine/cached_size.go
+++ b/go/vt/vtgate/engine/cached_size.go
@@ -131,7 +131,7 @@ func (cached *DDL) CachedSize(alloc bool) int64 {
}
size := int64(0)
if alloc {
- size += int64(64)
+ size += int64(80)
}
// field Keyspace *vitess.io/vitess/go/vt/vtgate/vindexes.Keyspace
size += cached.Keyspace.CachedSize(true)
@@ -145,6 +145,10 @@ func (cached *DDL) CachedSize(alloc bool) int64 {
size += cached.NormalDDL.CachedSize(true)
// field OnlineDDL *vitess.io/vitess/go/vt/vtgate/engine.OnlineDDL
size += cached.OnlineDDL.CachedSize(true)
+ // field Config vitess.io/vitess/go/vt/vtgate/dynamicconfig.DDL
+ if cc, ok := cached.Config.(cachedObject); ok {
+ size += cc.CachedSize(true)
+ }
return size
}
func (cached *DML) CachedSize(alloc bool) int64 {
@@ -857,6 +861,40 @@ func (cached *Projection) CachedSize(alloc bool) int64 {
}
return size
}
+
+//go:nocheckptr
+func (cached *RecurseCTE) CachedSize(alloc bool) int64 {
+ if cached == nil {
+ return int64(0)
+ }
+ size := int64(0)
+ if alloc {
+ size += int64(48)
+ }
+ // field Seed vitess.io/vitess/go/vt/vtgate/engine.Primitive
+ if cc, ok := cached.Seed.(cachedObject); ok {
+ size += cc.CachedSize(true)
+ }
+ // field Term vitess.io/vitess/go/vt/vtgate/engine.Primitive
+ if cc, ok := cached.Term.(cachedObject); ok {
+ size += cc.CachedSize(true)
+ }
+ // field Vars map[string]int
+ if cached.Vars != nil {
+ size += int64(48)
+ hmap := reflect.ValueOf(cached.Vars)
+ numBuckets := int(math.Pow(2, float64((*(*uint8)(unsafe.Pointer(hmap.Pointer() + uintptr(9)))))))
+ numOldBuckets := (*(*uint16)(unsafe.Pointer(hmap.Pointer() + uintptr(10))))
+ size += hack.RuntimeAllocSize(int64(numOldBuckets * 208))
+ if len(cached.Vars) > 0 || numBuckets > 1 {
+ size += hack.RuntimeAllocSize(int64(numBuckets * 208))
+ }
+ for k := range cached.Vars {
+ size += hack.RuntimeAllocSize(int64(len(k)))
+ }
+ }
+ return size
+}
func (cached *RenameFields) CachedSize(alloc bool) int64 {
if cached == nil {
return int64(0)
@@ -1299,8 +1337,10 @@ func (cached *TransactionStatus) CachedSize(alloc bool) int64 {
}
size := int64(0)
if alloc {
- size += int64(16)
+ size += int64(32)
}
+ // field Keyspace string
+ size += hack.RuntimeAllocSize(int64(len(cached.Keyspace)))
// field TransactionID string
size += hack.RuntimeAllocSize(int64(len(cached.TransactionID)))
return size
@@ -1561,6 +1601,24 @@ func (cached *VitessMetadata) CachedSize(alloc bool) int64 {
size += hack.RuntimeAllocSize(int64(len(cached.Value)))
return size
}
+func (cached *percentBasedMirror) CachedSize(alloc bool) int64 {
+ if cached == nil {
+ return int64(0)
+ }
+ size := int64(0)
+ if alloc {
+ size += int64(48)
+ }
+ // field primitive vitess.io/vitess/go/vt/vtgate/engine.Primitive
+ if cc, ok := cached.primitive.(cachedObject); ok {
+ size += cc.CachedSize(true)
+ }
+ // field target vitess.io/vitess/go/vt/vtgate/engine.Primitive
+ if cc, ok := cached.target.(cachedObject); ok {
+ size += cc.CachedSize(true)
+ }
+ return size
+}
//go:nocheckptr
func (cached *shardRoute) CachedSize(alloc bool) int64 {
diff --git a/go/vt/vtgate/engine/concatenate.go b/go/vt/vtgate/engine/concatenate.go
index 13727124e78..eb93711eed2 100644
--- a/go/vt/vtgate/engine/concatenate.go
+++ b/go/vt/vtgate/engine/concatenate.go
@@ -102,12 +102,14 @@ func (c *Concatenate) TryExecute(ctx context.Context, vcursor VCursor, bindVars
}
var rows [][]sqltypes.Value
- err = c.coerceAndVisitResults(res, fieldTypes, func(result *sqltypes.Result) error {
+ callback := func(result *sqltypes.Result) error {
rows = append(rows, result.Rows...)
return nil
- }, evalengine.ParseSQLMode(vcursor.SQLMode()))
- if err != nil {
- return nil, err
+ }
+ for _, r := range res {
+ if err = c.coerceAndVisitResultsForOneSource([]*sqltypes.Result{r}, fields, fieldTypes, callback, evalengine.ParseSQLMode(vcursor.SQLMode())); err != nil {
+ return nil, err
+ }
}
return &sqltypes.Result{
@@ -245,12 +247,14 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor,
// Mutexes for dealing with concurrent access to shared state.
var (
- muCallback sync.Mutex // Protects callback
- muFields sync.Mutex // Protects field state
- condFields = sync.NewCond(&muFields) // Condition var for field arrival
- wg errgroup.Group // Wait group for all streaming goroutines
- rest = make([]*sqltypes.Result, len(c.Sources)) // Collects first result from each source to derive fields
- fieldTypes []evalengine.Type // Cached final field types
+ muCallback sync.Mutex // Protects callback
+ muFields sync.Mutex // Protects field state
+ condFields = sync.NewCond(&muFields) // Condition var for field arrival
+ wg errgroup.Group // Wait group for all streaming goroutines
+ rest = make([]*sqltypes.Result, len(c.Sources)) // Collects first result from each source to derive fields
+ fieldTypes []evalengine.Type // Cached final field types
+ resultFields []*querypb.Field // Final fields that need to be set for the first result.
+ needsCoercion = make([]bool, len(c.Sources)) // Tracks if coercion is needed for each individual source
)
// Process each result chunk, considering type coercion.
@@ -258,19 +262,8 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor,
muCallback.Lock()
defer muCallback.Unlock()
- // Check if type coercion needed for this source.
- // We only need to check if fields are not in NoNeedToTypeCheck set.
- needsCoercion := false
- for idx, field := range rest[srcIdx].Fields {
- _, skip := c.NoNeedToTypeCheck[idx]
- if !skip && fieldTypes[idx].Type() != field.Type {
- needsCoercion = true
- break
- }
- }
-
// Apply type coercion if needed.
- if needsCoercion {
+ if needsCoercion[srcIdx] {
for _, row := range res.Rows {
if err := c.coerceValuesTo(row, fieldTypes, sqlmode); err != nil {
return err
@@ -296,12 +289,29 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor,
if !slices.Contains(rest, nil) {
// We have received fields from all sources. We can now calculate the output types
var err error
- resultChunk.Fields, fieldTypes, err = c.getFieldTypes(vcursor, rest)
+ resultFields, fieldTypes, err = c.getFieldTypes(vcursor, rest)
if err != nil {
muFields.Unlock()
return err
}
+ // Check if we need coercion for each source.
+ for srcIdx, result := range rest {
+ srcNeedsCoercion := false
+ for idx, field := range result.Fields {
+ _, skip := c.NoNeedToTypeCheck[idx]
+ // We only need to check if fields are not in NoNeedToTypeCheck set.
+ if !skip && fieldTypes[idx].Type() != field.Type {
+ srcNeedsCoercion = true
+ break
+ }
+ }
+ needsCoercion[srcIdx] = srcNeedsCoercion
+ }
+
+ // We only need to send the fields in the first result.
+ // We set this field after the coercion check to avoid calculating incorrect needs coercion value.
+ resultChunk.Fields = resultFields
muFields.Unlock()
defer condFields.Broadcast()
return callback(resultChunk, currIndex)
@@ -310,8 +320,11 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor,
// Wait for fields from all sources.
for slices.Contains(rest, nil) {
+ // This wait call lets go of the muFields lock and acquires it again later after waiting.
condFields.Wait()
}
+ // We only need to send fields in the first result.
+ resultChunk.Fields = nil
muFields.Unlock()
// Context check to avoid extra work.
@@ -368,12 +381,12 @@ func (c *Concatenate) sequentialStreamExec(ctx context.Context, vcursor VCursor,
firsts[i] = result[0]
}
- _, fieldTypes, err := c.getFieldTypes(vcursor, firsts)
+ fields, fieldTypes, err := c.getFieldTypes(vcursor, firsts)
if err != nil {
return err
}
for _, res := range results {
- if err = c.coerceAndVisitResults(res, fieldTypes, callback, sqlmode); err != nil {
+ if err = c.coerceAndVisitResultsForOneSource(res, fields, fieldTypes, callback, sqlmode); err != nil {
return err
}
}
@@ -381,25 +394,33 @@ func (c *Concatenate) sequentialStreamExec(ctx context.Context, vcursor VCursor,
return nil
}
-func (c *Concatenate) coerceAndVisitResults(
+func (c *Concatenate) coerceAndVisitResultsForOneSource(
res []*sqltypes.Result,
+ fields []*querypb.Field,
fieldTypes []evalengine.Type,
callback func(*sqltypes.Result) error,
sqlmode evalengine.SQLMode,
) error {
+ if len(res) == 0 {
+ return nil
+ }
+ needsCoercion := false
+ for idx, field := range res[0].Fields {
+ if fieldTypes[idx].Type() != field.Type {
+ needsCoercion = true
+ break
+ }
+ }
+ if res[0].Fields != nil {
+ res[0].Fields = fields
+ }
+
for _, r := range res {
if len(r.Rows) > 0 &&
len(fieldTypes) != len(r.Rows[0]) {
return errWrongNumberOfColumnsInSelect
}
- needsCoercion := false
- for idx, field := range r.Fields {
- if fieldTypes[idx].Type() != field.Type {
- needsCoercion = true
- break
- }
- }
if needsCoercion {
for _, row := range r.Rows {
err := c.coerceValuesTo(row, fieldTypes, sqlmode)
diff --git a/go/vt/vtgate/engine/concatenate_test.go b/go/vt/vtgate/engine/concatenate_test.go
index dd2b1300e9b..39b9ed961b3 100644
--- a/go/vt/vtgate/engine/concatenate_test.go
+++ b/go/vt/vtgate/engine/concatenate_test.go
@@ -124,27 +124,24 @@ func TestConcatenate_NoErrors(t *testing.T) {
if !tx {
txStr = "NotInTx"
}
- t.Run(fmt.Sprintf("%s-%s-Exec", txStr, tc.testName), func(t *testing.T) {
- qr, err := concatenate.TryExecute(context.Background(), vcursor, nil, true)
+ checkResult := func(t *testing.T, qr *sqltypes.Result, err error) {
if tc.expectedError == "" {
require.NoError(t, err)
utils.MustMatch(t, tc.expectedResult.Fields, qr.Fields, "fields")
- utils.MustMatch(t, tc.expectedResult.Rows, qr.Rows)
+ require.NoError(t, sqltypes.RowsEquals(tc.expectedResult.Rows, qr.Rows))
} else {
require.Error(t, err)
require.Contains(t, err.Error(), tc.expectedError)
}
+ }
+ t.Run(fmt.Sprintf("%s-%s-Exec", txStr, tc.testName), func(t *testing.T) {
+ qr, err := concatenate.TryExecute(context.Background(), vcursor, nil, true)
+ checkResult(t, qr, err)
})
t.Run(fmt.Sprintf("%s-%s-StreamExec", txStr, tc.testName), func(t *testing.T) {
qr, err := wrapStreamExecute(concatenate, vcursor, nil, true)
- if tc.expectedError == "" {
- require.NoError(t, err)
- require.NoError(t, sqltypes.RowsEquals(tc.expectedResult.Rows, qr.Rows))
- } else {
- require.Error(t, err)
- require.Contains(t, err.Error(), tc.expectedError)
- }
+ checkResult(t, qr, err)
})
}
}
diff --git a/go/vt/vtgate/engine/dbddl.go b/go/vt/vtgate/engine/dbddl.go
index 60bb4a7202b..7783e6bdc12 100644
--- a/go/vt/vtgate/engine/dbddl.go
+++ b/go/vt/vtgate/engine/dbddl.go
@@ -55,6 +55,8 @@ type DBDDLPlugin interface {
DropDatabase(ctx context.Context, name string) error
}
+const dbDDLDefaultTimeout = 500 * time.Millisecond
+
// DBDDL is just a container around custom database provisioning plugins
// The default behaviour is to just return an error
type DBDDL struct {
@@ -102,8 +104,12 @@ func (c *DBDDL) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[st
log.Errorf("'%s' database ddl plugin is not registered. Falling back to default plugin", name)
plugin = databaseCreatorPlugins[defaultDBDDLPlugin]
}
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, c.queryTimeout)
- defer cancelFunc()
+
+ if c.queryTimeout > 0 {
+ var cancel context.CancelFunc
+ ctx, cancel = context.WithTimeout(ctx, time.Duration(c.queryTimeout)*time.Millisecond)
+ defer cancel()
+ }
if c.create {
return c.createDatabase(ctx, vcursor, plugin)
@@ -125,9 +131,9 @@ func (c *DBDDL) createDatabase(ctx context.Context, vcursor VCursor, plugin DBDD
break
}
select {
- case <-ctx.Done(): //context cancelled
+ case <-ctx.Done(): // context cancelled
return nil, vterrors.Errorf(vtrpc.Code_DEADLINE_EXCEEDED, "could not validate create database: destination not resolved")
- case <-time.After(500 * time.Millisecond): //timeout
+ case <-time.After(dbDDLDefaultTimeout): // timeout
}
}
var queries []*querypb.BoundQuery
@@ -146,9 +152,9 @@ func (c *DBDDL) createDatabase(ctx context.Context, vcursor VCursor, plugin DBDD
if err != nil {
noErr = false
select {
- case <-ctx.Done(): //context cancelled
+ case <-ctx.Done(): // context cancelled
return nil, vterrors.Errorf(vtrpc.Code_DEADLINE_EXCEEDED, "could not validate create database: tablets not healthy")
- case <-time.After(500 * time.Millisecond): //timeout
+ case <-time.After(dbDDLDefaultTimeout): // timeout
}
break
}
@@ -167,9 +173,9 @@ func (c *DBDDL) dropDatabase(ctx context.Context, vcursor VCursor, plugin DBDDLP
}
for vcursor.KeyspaceAvailable(c.name) {
select {
- case <-ctx.Done(): //context cancelled
+ case <-ctx.Done(): // context cancelled
return nil, vterrors.Errorf(vtrpc.Code_DEADLINE_EXCEEDED, "could not validate drop database: keyspace still available in vschema")
- case <-time.After(500 * time.Millisecond): //timeout
+ case <-time.After(dbDDLDefaultTimeout): // timeout
}
}
diff --git a/go/vt/vtgate/engine/ddl.go b/go/vt/vtgate/engine/ddl.go
index cfdaa5866dc..d7e17eb4f4f 100644
--- a/go/vt/vtgate/engine/ddl.go
+++ b/go/vt/vtgate/engine/ddl.go
@@ -25,6 +25,7 @@ import (
"vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/dynamicconfig"
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
@@ -42,8 +43,7 @@ type DDL struct {
NormalDDL *Send
OnlineDDL *OnlineDDL
- DirectDDLEnabled bool
- OnlineDDLEnabled bool
+ Config dynamicconfig.DDL
CreateTempTable bool
}
@@ -107,12 +107,12 @@ func (ddl *DDL) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[st
switch {
case ddl.isOnlineSchemaDDL():
- if !ddl.OnlineDDLEnabled {
+ if !ddl.Config.OnlineEnabled() {
return nil, schema.ErrOnlineDDLDisabled
}
return vcursor.ExecutePrimitive(ctx, ddl.OnlineDDL, bindVars, wantfields)
default: // non online-ddl
- if !ddl.DirectDDLEnabled {
+ if !ddl.Config.DirectEnabled() {
return nil, schema.ErrDirectDDLDisabled
}
return vcursor.ExecutePrimitive(ctx, ddl.NormalDDL, bindVars, wantfields)
diff --git a/go/vt/vtgate/engine/ddl_test.go b/go/vt/vtgate/engine/ddl_test.go
index 3f7ccb75f70..1d52089bf39 100644
--- a/go/vt/vtgate/engine/ddl_test.go
+++ b/go/vt/vtgate/engine/ddl_test.go
@@ -27,13 +27,23 @@ import (
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
+type ddlConfig struct{}
+
+func (ddlConfig) DirectEnabled() bool {
+ return true
+}
+
+func (ddlConfig) OnlineEnabled() bool {
+ return true
+}
+
func TestDDL(t *testing.T) {
ddl := &DDL{
DDL: &sqlparser.CreateTable{
Table: sqlparser.NewTableName("a"),
},
- DirectDDLEnabled: true,
- OnlineDDL: &OnlineDDL{},
+ Config: ddlConfig{},
+ OnlineDDL: &OnlineDDL{},
NormalDDL: &Send{
Keyspace: &vindexes.Keyspace{
Name: "ks",
diff --git a/go/vt/vtgate/engine/delete.go b/go/vt/vtgate/engine/delete.go
index 6e354aae5f5..91bcca5cf6a 100644
--- a/go/vt/vtgate/engine/delete.go
+++ b/go/vt/vtgate/engine/delete.go
@@ -42,9 +42,6 @@ type Delete struct {
// TryExecute performs a non-streaming exec.
func (del *Delete) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, _ bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, del.QueryTimeout)
- defer cancelFunc()
-
rss, bvs, err := del.findRoute(ctx, vcursor, bindVars)
if err != nil {
return nil, err
diff --git a/go/vt/vtgate/engine/fake_primitive_test.go b/go/vt/vtgate/engine/fake_primitive_test.go
index e992c2a4623..b878c1931c0 100644
--- a/go/vt/vtgate/engine/fake_primitive_test.go
+++ b/go/vt/vtgate/engine/fake_primitive_test.go
@@ -24,6 +24,7 @@ import (
"testing"
"golang.org/x/sync/errgroup"
+ "google.golang.org/protobuf/proto"
"vitess.io/vitess/go/sqltypes"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -80,7 +81,7 @@ func (f *fakePrimitive) TryExecute(ctx context.Context, vcursor VCursor, bindVar
if r == nil {
return nil, f.sendErr
}
- return r, nil
+ return r.Copy(), nil
}
func (f *fakePrimitive) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
@@ -111,7 +112,7 @@ func (f *fakePrimitive) syncCall(wantfields bool, callback func(*sqltypes.Result
}
result := &sqltypes.Result{}
for i := 0; i < len(r.Rows); i++ {
- result.Rows = append(result.Rows, r.Rows[i])
+ result.Rows = append(result.Rows, sqltypes.CopyRow(r.Rows[i]))
// Send only two rows at a time.
if i%2 == 1 {
if err := callback(result); err != nil {
@@ -188,6 +189,15 @@ func wrapStreamExecute(prim Primitive, vcursor VCursor, bindVars map[string]*que
if result == nil {
result = r
} else {
+ if r.Fields != nil {
+ for i, field := range r.Fields {
+ aField := field
+ bField := result.Fields[i]
+ if !proto.Equal(aField, bField) {
+ return fmt.Errorf("fields differ: %s <> %s", aField.String(), bField.String())
+ }
+ }
+ }
result.Rows = append(result.Rows, r.Rows...)
}
return nil
diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go
index 653bcf64576..9ba4fdc6a6b 100644
--- a/go/vt/vtgate/engine/fake_vcursor_test.go
+++ b/go/vt/vtgate/engine/fake_vcursor_test.go
@@ -46,17 +46,25 @@ import (
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
)
-var testMaxMemoryRows = 100
-var testIgnoreMaxMemoryRows = false
+var (
+ testMaxMemoryRows = 100
+ testIgnoreMaxMemoryRows = false
+)
-var _ VCursor = (*noopVCursor)(nil)
-var _ SessionActions = (*noopVCursor)(nil)
+var (
+ _ VCursor = (*noopVCursor)(nil)
+ _ SessionActions = (*noopVCursor)(nil)
+)
// noopVCursor is used to build other vcursors.
type noopVCursor struct {
inTx bool
}
+func (t *noopVCursor) SetExecQueryTimeout(timeout *int) {
+ panic("implement me")
+}
+
// MySQLVersion implements VCursor.
func (t *noopVCursor) Commit(ctx context.Context) error {
return nil
@@ -112,10 +120,22 @@ func (t *noopVCursor) CloneForReplicaWarming(ctx context.Context) VCursor {
panic("implement me")
}
+func (t *noopVCursor) CloneForMirroring(ctx context.Context) VCursor {
+ panic("implement me")
+}
+
func (t *noopVCursor) ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error) {
panic("implement me")
}
+func (t *noopVCursor) UnresolvedTransactions(ctx context.Context, keyspace string) ([]*querypb.TransactionMetadata, error) {
+ panic("implement me")
+}
+
+func (t *noopVCursor) StartPrimitiveTrace() func() Stats {
+ panic("implement me")
+}
+
func (t *noopVCursor) SetExec(ctx context.Context, name string, value string) error {
panic("implement me")
}
@@ -293,10 +313,6 @@ func (t *noopVCursor) SetClientFoundRows(context.Context, bool) error {
func (t *noopVCursor) SetQueryTimeout(maxExecutionTime int64) {
}
-func (t *noopVCursor) GetQueryTimeout(queryTimeoutFromComments int) int {
- return queryTimeoutFromComments
-}
-
func (t *noopVCursor) SetSkipQueryPlanCache(context.Context, bool) error {
panic("implement me")
}
@@ -384,8 +400,10 @@ func (t *noopVCursor) GetDBDDLPluginName() string {
panic("unimplemented")
}
-var _ VCursor = (*loggingVCursor)(nil)
-var _ SessionActions = (*loggingVCursor)(nil)
+var (
+ _ VCursor = (*loggingVCursor)(nil)
+ _ SessionActions = (*loggingVCursor)(nil)
+)
// loggingVCursor logs requests and allows you to verify
// that the correct requests were made.
@@ -402,7 +420,7 @@ type loggingVCursor struct {
resultErr error
warnings []*querypb.QueryWarning
- transactionStatusOutput *querypb.TransactionMetadata
+ transactionStatusOutput []*querypb.TransactionMetadata
// Optional errors that can be returned from nextResult() alongside the results for
// multi-shard queries
@@ -426,6 +444,11 @@ type loggingVCursor struct {
shardSession []*srvtopo.ResolvedShard
parser *sqlparser.Parser
+
+ onMirrorClonesFn func(context.Context) VCursor
+ onExecuteMultiShardFn func(context.Context, Primitive, []*srvtopo.ResolvedShard, []*querypb.BoundQuery, bool, bool)
+ onStreamExecuteMultiFn func(context.Context, Primitive, string, []*srvtopo.ResolvedShard, []map[string]*querypb.BindVariable, bool, bool, func(*sqltypes.Result) error)
+ onRecordMirrorStatsFn func(time.Duration, time.Duration, error)
}
func (f *loggingVCursor) HasCreatedTempTable() {
@@ -541,6 +564,13 @@ func (f *loggingVCursor) CloneForReplicaWarming(ctx context.Context) VCursor {
return f
}
+func (f *loggingVCursor) CloneForMirroring(ctx context.Context) VCursor {
+ if f.onMirrorClonesFn != nil {
+ return f.onMirrorClonesFn(ctx)
+ }
+ panic("no mirror clones available")
+}
+
func (f *loggingVCursor) Execute(ctx context.Context, method string, query string, bindvars map[string]*querypb.BindVariable, rollbackOnError bool, co vtgatepb.CommitOrder) (*sqltypes.Result, error) {
name := "Unknown"
switch co {
@@ -558,7 +588,12 @@ func (f *loggingVCursor) Execute(ctx context.Context, method string, query strin
}
func (f *loggingVCursor) ExecuteMultiShard(ctx context.Context, primitive Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, rollbackOnError, canAutocommit bool) (*sqltypes.Result, []error) {
+ f.mu.Lock()
+ defer f.mu.Unlock()
f.log = append(f.log, fmt.Sprintf("ExecuteMultiShard %v%v %v", printResolvedShardQueries(rss, queries), rollbackOnError, canAutocommit))
+ if f.onExecuteMultiShardFn != nil {
+ f.onExecuteMultiShardFn(ctx, primitive, rss, queries, rollbackOnError, canAutocommit)
+ }
res, err := f.nextResult()
if err != nil {
return nil, []error{err}
@@ -579,6 +614,9 @@ func (f *loggingVCursor) ExecuteStandalone(ctx context.Context, primitive Primit
func (f *loggingVCursor) StreamExecuteMulti(ctx context.Context, primitive Primitive, query string, rss []*srvtopo.ResolvedShard, bindVars []map[string]*querypb.BindVariable, rollbackOnError bool, autocommit bool, callback func(reply *sqltypes.Result) error) []error {
f.mu.Lock()
f.log = append(f.log, fmt.Sprintf("StreamExecuteMulti %s %s", query, printResolvedShardsBindVars(rss, bindVars)))
+ if f.onStreamExecuteMultiFn != nil {
+ f.onStreamExecuteMultiFn(ctx, primitive, query, rss, bindVars, rollbackOnError, autocommit, callback)
+ }
r, err := f.nextResult()
f.mu.Unlock()
if err != nil {
@@ -730,6 +768,8 @@ func (f *loggingVCursor) ResolveDestinationsMultiCol(ctx context.Context, keyspa
func (f *loggingVCursor) ExpectLog(t *testing.T, want []string) {
t.Helper()
+ f.mu.Lock()
+ defer f.mu.Unlock()
if len(f.log) == 0 && len(want) == 0 {
return
}
@@ -747,6 +787,8 @@ func (f *loggingVCursor) ExpectWarnings(t *testing.T, want []*querypb.QueryWarni
}
func (f *loggingVCursor) Rewind() {
+ f.mu.Lock()
+ defer f.mu.Unlock()
f.curShardForKsid = 0
f.curResult = 0
f.log = nil
@@ -820,6 +862,17 @@ func (f *loggingVCursor) CanUseSetVar() bool {
}
func (f *loggingVCursor) ReadTransaction(_ context.Context, _ string) (*querypb.TransactionMetadata, error) {
+ if f.resultErr != nil {
+ return nil, f.resultErr
+ }
+ var out *querypb.TransactionMetadata
+ if len(f.transactionStatusOutput) > 0 {
+ out = f.transactionStatusOutput[0]
+ }
+ return out, nil
+}
+
+func (f *loggingVCursor) UnresolvedTransactions(_ context.Context, _ string) ([]*querypb.TransactionMetadata, error) {
if f.resultErr != nil {
return nil, f.resultErr
}
@@ -834,15 +887,26 @@ func (t *loggingVCursor) SQLParser() *sqlparser.Parser {
return t.parser
}
+func (t *loggingVCursor) RecordMirrorStats(sourceExecTime, targetExecTime time.Duration, targetErr error) {
+ if t.onRecordMirrorStatsFn != nil {
+ t.onRecordMirrorStatsFn(sourceExecTime, targetExecTime, targetErr)
+ }
+}
+
func (t *noopVCursor) VExplainLogging() {}
func (t *noopVCursor) DisableLogging() {}
func (t *noopVCursor) GetVExplainLogs() []ExecuteEntry {
return nil
}
+
func (t *noopVCursor) GetLogs() ([]ExecuteEntry, error) {
return nil, nil
}
+// RecordMirrorStats implements VCursor.
+func (t *noopVCursor) RecordMirrorStats(sourceExecTime, targetExecTime time.Duration, targetErr error) {
+}
+
func expectResult(t *testing.T, result, want *sqltypes.Result) {
t.Helper()
fieldsResult := fmt.Sprintf("%v", result.Fields)
diff --git a/go/vt/vtgate/engine/insert.go b/go/vt/vtgate/engine/insert.go
index af2d290d957..cd462966ccc 100644
--- a/go/vt/vtgate/engine/insert.go
+++ b/go/vt/vtgate/engine/insert.go
@@ -112,9 +112,6 @@ func (ins *Insert) RouteType() string {
// TryExecute performs a non-streaming exec.
func (ins *Insert) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, _ bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, ins.QueryTimeout)
- defer cancelFunc()
-
switch ins.Opcode {
case InsertUnsharded:
return ins.insertIntoUnshardedTable(ctx, vcursor, bindVars)
diff --git a/go/vt/vtgate/engine/insert_select.go b/go/vt/vtgate/engine/insert_select.go
index 88767420508..f8f3936e323 100644
--- a/go/vt/vtgate/engine/insert_select.go
+++ b/go/vt/vtgate/engine/insert_select.go
@@ -22,6 +22,7 @@ import (
"fmt"
"strconv"
"sync"
+ "time"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/key"
@@ -93,9 +94,6 @@ func (ins *InsertSelect) RouteType() string {
// TryExecute performs a non-streaming exec.
func (ins *InsertSelect) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, _ bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, ins.QueryTimeout)
- defer cancelFunc()
-
if ins.Keyspace.Sharded {
return ins.execInsertSharded(ctx, vcursor, bindVars)
}
@@ -111,8 +109,11 @@ func (ins *InsertSelect) TryStreamExecute(ctx context.Context, vcursor VCursor,
}
return callback(res)
}
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, ins.QueryTimeout)
- defer cancelFunc()
+ if ins.QueryTimeout > 0 {
+ var cancel context.CancelFunc
+ ctx, cancel = context.WithTimeout(ctx, time.Duration(ins.QueryTimeout)*time.Millisecond)
+ defer cancel()
+ }
sharded := ins.Keyspace.Sharded
output := &sqltypes.Result{}
diff --git a/go/vt/vtgate/engine/mirror.go b/go/vt/vtgate/engine/mirror.go
new file mode 100644
index 00000000000..6396e4b33ec
--- /dev/null
+++ b/go/vt/vtgate/engine/mirror.go
@@ -0,0 +1,195 @@
+/*
+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 engine
+
+import (
+ "context"
+ "math/rand/v2"
+ "time"
+
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/vterrors"
+)
+
+var errMirrorTargetQueryTookTooLong = vterrors.Errorf(vtrpc.Code_ABORTED, "Mirror target query took too long")
+
+type (
+ // percentBasedMirror represents the instructions to execute an
+ // authoritative primitive and, based on whether a die-roll exceeds a
+ // percentage, to also execute a target Primitive.
+ percentBasedMirror struct {
+ percent float32
+ primitive Primitive
+ target Primitive
+ }
+
+ mirrorResult struct {
+ execTime time.Duration
+ err error
+ }
+)
+
+const (
+ // maxMirrorTargetLag limits how long a mirror target may continue
+ // executing after the main primitive has finished.
+ maxMirrorTargetLag = 100 * time.Millisecond
+)
+
+var _ Primitive = (*percentBasedMirror)(nil)
+
+// NewPercentBasedMirror creates a Mirror.
+func NewPercentBasedMirror(percentage float32, primitive Primitive, target Primitive) Primitive {
+ return &percentBasedMirror{percent: percentage, primitive: primitive, target: target}
+}
+
+func (m *percentBasedMirror) RouteType() string {
+ return "Mirror"
+}
+
+func (m *percentBasedMirror) GetKeyspaceName() string {
+ return m.primitive.GetKeyspaceName()
+}
+
+func (m *percentBasedMirror) GetTableName() string {
+ return m.primitive.GetTableName()
+}
+
+func (m *percentBasedMirror) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) {
+ return m.primitive.GetFields(ctx, vcursor, bindVars)
+}
+
+func (m *percentBasedMirror) NeedsTransaction() bool {
+ return m.primitive.NeedsTransaction()
+}
+
+func (m *percentBasedMirror) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
+ if !m.percentAtLeastDieRoll() {
+ return vcursor.ExecutePrimitive(ctx, m.primitive, bindVars, wantfields)
+ }
+
+ mirrorCh := make(chan mirrorResult, 1)
+ mirrorCtx, mirrorCtxCancel := context.WithCancel(ctx)
+ defer mirrorCtxCancel()
+
+ go func() {
+ mirrorVCursor := vcursor.CloneForMirroring(mirrorCtx)
+ targetStartTime := time.Now()
+ _, targetErr := mirrorVCursor.ExecutePrimitive(mirrorCtx, m.target, bindVars, wantfields)
+ mirrorCh <- mirrorResult{
+ execTime: time.Since(targetStartTime),
+ err: targetErr,
+ }
+ }()
+
+ var (
+ sourceExecTime, targetExecTime time.Duration
+ targetErr error
+ )
+
+ sourceStartTime := time.Now()
+ r, err := vcursor.ExecutePrimitive(ctx, m.primitive, bindVars, wantfields)
+ sourceExecTime = time.Since(sourceStartTime)
+
+ // Cancel the mirror context if it continues executing too long.
+ select {
+ case r := <-mirrorCh:
+ // Mirror target finished on time.
+ targetExecTime = r.execTime
+ targetErr = r.err
+ case <-time.After(maxMirrorTargetLag):
+ // Mirror target took too long.
+ mirrorCtxCancel()
+ targetExecTime = sourceExecTime + maxMirrorTargetLag
+ targetErr = errMirrorTargetQueryTookTooLong
+ }
+
+ vcursor.RecordMirrorStats(sourceExecTime, targetExecTime, targetErr)
+
+ return r, err
+}
+
+func (m *percentBasedMirror) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
+ if !m.percentAtLeastDieRoll() {
+ return vcursor.StreamExecutePrimitive(ctx, m.primitive, bindVars, wantfields, callback)
+ }
+
+ mirrorCh := make(chan mirrorResult, 1)
+ mirrorCtx, mirrorCtxCancel := context.WithCancel(ctx)
+ defer mirrorCtxCancel()
+
+ go func() {
+ mirrorVCursor := vcursor.CloneForMirroring(mirrorCtx)
+ mirrorStartTime := time.Now()
+ targetErr := mirrorVCursor.StreamExecutePrimitive(mirrorCtx, m.target, bindVars, wantfields, func(_ *sqltypes.Result) error {
+ return nil
+ })
+ mirrorCh <- mirrorResult{
+ execTime: time.Since(mirrorStartTime),
+ err: targetErr,
+ }
+ }()
+
+ var (
+ sourceExecTime, targetExecTime time.Duration
+ targetErr error
+ )
+
+ sourceStartTime := time.Now()
+ err := vcursor.StreamExecutePrimitive(ctx, m.primitive, bindVars, wantfields, callback)
+ sourceExecTime = time.Since(sourceStartTime)
+
+ // Cancel the mirror context if it continues executing too long.
+ select {
+ case r := <-mirrorCh:
+ // Mirror target finished on time.
+ targetExecTime = r.execTime
+ targetErr = r.err
+ case <-time.After(maxMirrorTargetLag):
+ // Mirror target took too long.
+ mirrorCtxCancel()
+ targetExecTime = sourceExecTime + maxMirrorTargetLag
+ targetErr = errMirrorTargetQueryTookTooLong
+ }
+
+ vcursor.RecordMirrorStats(sourceExecTime, targetExecTime, targetErr)
+
+ return err
+}
+
+// Inputs is a slice containing the inputs to this Primitive.
+// The returned map has additional information about the inputs, that is used in the description.
+func (m *percentBasedMirror) Inputs() ([]Primitive, []map[string]any) {
+ return []Primitive{m.primitive, m.target}, nil
+}
+
+// description is the description, sans the inputs, of this Primitive.
+// to get the plan description with all children, use PrimitiveToPlanDescription()
+func (m *percentBasedMirror) description() PrimitiveDescription {
+ return PrimitiveDescription{
+ OperatorType: "Mirror",
+ Variant: "PercentBased",
+ Other: map[string]any{
+ "Percent": m.percent,
+ },
+ }
+}
+
+func (m *percentBasedMirror) percentAtLeastDieRoll() bool {
+ return m.percent >= (rand.Float32() * 100.0)
+}
diff --git a/go/vt/vtgate/engine/mirror_test.go b/go/vt/vtgate/engine/mirror_test.go
new file mode 100644
index 00000000000..753b1a26429
--- /dev/null
+++ b/go/vt/vtgate/engine/mirror_test.go
@@ -0,0 +1,549 @@
+/*
+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 engine
+
+import (
+ "context"
+ "fmt"
+ "sync"
+ "sync/atomic"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/srvtopo"
+ "vitess.io/vitess/go/vt/vtgate/evalengine"
+ "vitess.io/vitess/go/vt/vtgate/vindexes"
+)
+
+func TestMirror(t *testing.T) {
+ vindex, _ := vindexes.CreateVindex("xxhash", "xxhash_vdx", nil)
+
+ primitive := NewRoute(
+ Unsharded,
+ &vindexes.Keyspace{
+ Name: "ks1",
+ },
+ "select f.bar from foo f where f.id = 1",
+ "select 1 from foo f where f.id = 1 and 1 != 1",
+ )
+
+ mirrorPrimitive1 := NewRoute(
+ EqualUnique,
+ &vindexes.Keyspace{
+ Name: "ks2",
+ Sharded: true,
+ },
+ "select f.bar from foo f where f.id = 1",
+ "select 1 from foo f where f.id = 1 and 1 != 1",
+ )
+ mirrorPrimitive1.Vindex = vindex.(vindexes.SingleColumn)
+ mirrorPrimitive1.Values = []evalengine.Expr{
+ evalengine.NewLiteralInt(1),
+ }
+
+ mirror := NewPercentBasedMirror(100, primitive, mirrorPrimitive1)
+
+ mirrorVC := &loggingVCursor{
+ shards: []string{"-20", "20-"},
+ ksShardMap: map[string][]string{
+ "ks2": {"-20", "20-"},
+ },
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "bar",
+ "varchar",
+ ),
+ "hello",
+ ),
+ },
+ }
+
+ sourceExecTime := atomic.Pointer[time.Duration]{}
+ targetExecTime := atomic.Pointer[time.Duration]{}
+ targetErr := atomic.Pointer[error]{}
+
+ vc := &loggingVCursor{
+ shards: []string{"0"},
+ ksShardMap: map[string][]string{
+ "ks1": {"0"},
+ },
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "bar",
+ "varchar",
+ ),
+ "hello",
+ ),
+ },
+ onMirrorClonesFn: func(ctx context.Context) VCursor {
+ return mirrorVC
+ },
+ onRecordMirrorStatsFn: func(sourceTime time.Duration, targetTime time.Duration, err error) {
+ sourceExecTime.Store(&sourceTime)
+ targetExecTime.Store(&targetTime)
+ targetErr.Store(&err)
+ },
+ }
+
+ t.Run("TryExecute success", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ mirrorVC.Rewind()
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ want := vc.results[0]
+ res, err := mirror.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, true)
+ require.Equal(t, want, res)
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "ExecuteMultiShard ks1.0: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "ExecuteMultiShard ks2.-20: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ require.NotNil(t, targetExecTime.Load())
+ require.Nil(t, *targetErr.Load())
+ })
+
+ t.Run("TryExecute return primitive error", func(t *testing.T) {
+ results := vc.results
+
+ defer func() {
+ vc.Rewind()
+ vc.results = results
+ vc.resultErr = nil
+ mirrorVC.Rewind()
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ vc.results = nil
+ vc.resultErr = fmt.Errorf("return me")
+
+ ctx := context.Background()
+ res, err := mirror.TryExecute(ctx, vc, map[string]*querypb.BindVariable{}, true)
+ require.Nil(t, res)
+ require.Error(t, err)
+ require.Equal(t, vc.resultErr, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "ExecuteMultiShard ks1.0: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "ExecuteMultiShard ks2.-20: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ require.NotNil(t, targetExecTime.Load())
+ require.Nil(t, *targetErr.Load())
+ })
+
+ t.Run("TryExecute ignore mirror target error", func(t *testing.T) {
+ results := mirrorVC.results
+
+ defer func() {
+ vc.Rewind()
+ mirrorVC.Rewind()
+ mirrorVC.results = results
+ mirrorVC.resultErr = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ mirrorVC.results = nil
+ mirrorVC.resultErr = fmt.Errorf("ignore me")
+
+ want := vc.results[0]
+ res, err := mirror.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, true)
+ require.Equal(t, res, want)
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "ExecuteMultiShard ks1.0: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "ExecuteMultiShard ks2.-20: select f.bar from foo f where f.id = 1 {} false false",
+ })
+
+ require.NotNil(t, targetExecTime.Load())
+ mirrorErr := targetErr.Load()
+ require.ErrorContains(t, *mirrorErr, "ignore me")
+ })
+
+ t.Run("TryExecute fast mirror target", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ vc.onExecuteMultiShardFn = nil
+ mirrorVC.Rewind()
+ mirrorVC.onExecuteMultiShardFn = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ primitiveLatency := 10 * time.Millisecond
+ vc.onExecuteMultiShardFn = func(ctx context.Context, _ Primitive, _ []*srvtopo.ResolvedShard, _ []*querypb.BoundQuery, _ bool, _ bool) {
+ time.Sleep(primitiveLatency)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "primitive context done")
+ default:
+ }
+ }
+
+ var wg sync.WaitGroup
+ defer wg.Wait()
+ mirrorVC.onExecuteMultiShardFn = func(ctx context.Context, _ Primitive, _ []*srvtopo.ResolvedShard, _ []*querypb.BoundQuery, _ bool, _ bool) {
+ wg.Add(1)
+ defer wg.Done()
+ time.Sleep(primitiveLatency / 2)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "mirror target context done")
+ default:
+ }
+ }
+
+ want := vc.results[0]
+ res, err := mirror.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, true)
+ require.Equal(t, res, want)
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "ExecuteMultiShard ks1.0: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "ExecuteMultiShard ks2.-20: select f.bar from foo f where f.id = 1 {} false false",
+ })
+
+ wg.Wait()
+
+ require.Greater(t, *sourceExecTime.Load(), *targetExecTime.Load())
+ })
+
+ t.Run("TryExecute slow mirror target", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ vc.onExecuteMultiShardFn = nil
+ mirrorVC.Rewind()
+ mirrorVC.onExecuteMultiShardFn = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ primitiveLatency := 10 * time.Millisecond
+ vc.onExecuteMultiShardFn = func(ctx context.Context, _ Primitive, _ []*srvtopo.ResolvedShard, _ []*querypb.BoundQuery, _ bool, _ bool) {
+ time.Sleep(primitiveLatency)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "primitive context done")
+ default:
+ }
+ }
+
+ var wg sync.WaitGroup
+ defer wg.Wait()
+ mirrorVC.onExecuteMultiShardFn = func(ctx context.Context, _ Primitive, _ []*srvtopo.ResolvedShard, _ []*querypb.BoundQuery, _ bool, _ bool) {
+ wg.Add(1)
+ defer wg.Done()
+ time.Sleep(primitiveLatency + maxMirrorTargetLag + (5 * time.Millisecond))
+ select {
+ case <-ctx.Done():
+ require.NotNil(t, ctx.Err())
+ require.ErrorContains(t, ctx.Err(), "context canceled")
+ default:
+ require.Fail(t, "mirror target context not done")
+ }
+ }
+
+ want := vc.results[0]
+ res, err := mirror.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, true)
+ require.Equal(t, res, want)
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "ExecuteMultiShard ks1.0: select f.bar from foo f where f.id = 1 {} false false",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "ExecuteMultiShard ks2.-20: select f.bar from foo f where f.id = 1 {} false false",
+ })
+
+ wg.Wait()
+
+ require.Greater(t, *targetExecTime.Load(), *sourceExecTime.Load())
+ require.ErrorContains(t, *targetErr.Load(), "Mirror target query took too long")
+ })
+
+ t.Run("TryStreamExecute success", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ mirrorVC.Rewind()
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ want := vc.results[0]
+ err := mirror.TryStreamExecute(
+ context.Background(),
+ vc,
+ map[string]*querypb.BindVariable{},
+ true,
+ func(result *sqltypes.Result) error {
+ require.Equal(t, want, result)
+ return nil
+ },
+ )
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks1.0: {} ",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks2.-20: {} ",
+ })
+
+ require.NotNil(t, targetExecTime.Load())
+ require.Nil(t, *targetErr.Load())
+ })
+
+ t.Run("TryStreamExecute return primitive error", func(t *testing.T) {
+ results := vc.results
+
+ defer func() {
+ vc.Rewind()
+ vc.results = results
+ vc.resultErr = nil
+ mirrorVC.Rewind()
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ vc.results = nil
+ vc.resultErr = fmt.Errorf("return me")
+
+ err := mirror.TryStreamExecute(
+ context.Background(),
+ vc,
+ map[string]*querypb.BindVariable{},
+ true,
+ func(result *sqltypes.Result) error {
+ require.Nil(t, result)
+ return nil
+ },
+ )
+ require.Error(t, err)
+ require.Equal(t, vc.resultErr, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks1.0: {} ",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks2.-20: {} ",
+ })
+
+ require.NotNil(t, targetExecTime.Load())
+ require.Nil(t, *targetErr.Load())
+ })
+
+ t.Run("TryStreamExecute ignore mirror target error", func(t *testing.T) {
+ results := mirrorVC.results
+
+ defer func() {
+ vc.Rewind()
+ mirrorVC.Rewind()
+ mirrorVC.results = results
+ mirrorVC.resultErr = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ mirrorVC.results = nil
+ mirrorVC.resultErr = fmt.Errorf("ignore me")
+
+ want := vc.results[0]
+ err := mirror.TryStreamExecute(
+ context.Background(),
+ vc,
+ map[string]*querypb.BindVariable{},
+ true,
+ func(result *sqltypes.Result) error {
+ require.Equal(t, want, result)
+ return nil
+ },
+ )
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks1.0: {} ",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks2.-20: {} ",
+ })
+
+ require.NotNil(t, targetExecTime.Load())
+ require.ErrorContains(t, *targetErr.Load(), "ignore me")
+ })
+
+ t.Run("TryStreamExecute fast mirror target", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ vc.onStreamExecuteMultiFn = nil
+ mirrorVC.Rewind()
+ mirrorVC.onStreamExecuteMultiFn = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ primitiveLatency := 10 * time.Millisecond
+ vc.onStreamExecuteMultiFn = func(ctx context.Context, _ Primitive, _ string, _ []*srvtopo.ResolvedShard, _ []map[string]*querypb.BindVariable, _ bool, _ bool, _ func(*sqltypes.Result) error) {
+ time.Sleep(primitiveLatency)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "primitive context done")
+ default:
+ }
+ }
+
+ var wg sync.WaitGroup
+ defer wg.Wait()
+ mirrorVC.onStreamExecuteMultiFn = func(ctx context.Context, _ Primitive, _ string, _ []*srvtopo.ResolvedShard, _ []map[string]*querypb.BindVariable, _ bool, _ bool, _ func(*sqltypes.Result) error) {
+ wg.Add(1)
+ defer wg.Done()
+ time.Sleep(primitiveLatency / 2)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "mirror target context done")
+ default:
+ }
+ }
+
+ want := vc.results[0]
+ err := mirror.TryStreamExecute(
+ context.Background(),
+ vc,
+ map[string]*querypb.BindVariable{},
+ true,
+ func(result *sqltypes.Result) error {
+ require.Equal(t, want, result)
+ return nil
+ },
+ )
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks1.0: {} ",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks2.-20: {} ",
+ })
+
+ require.Greater(t, *sourceExecTime.Load(), *targetExecTime.Load())
+ })
+
+ t.Run("TryStreamExecute slow mirror target", func(t *testing.T) {
+ defer func() {
+ vc.Rewind()
+ vc.onStreamExecuteMultiFn = nil
+ mirrorVC.Rewind()
+ mirrorVC.onStreamExecuteMultiFn = nil
+ sourceExecTime.Store(nil)
+ targetExecTime.Store(nil)
+ targetErr.Store(nil)
+ }()
+
+ primitiveLatency := 10 * time.Millisecond
+ vc.onStreamExecuteMultiFn = func(ctx context.Context, _ Primitive, _ string, _ []*srvtopo.ResolvedShard, _ []map[string]*querypb.BindVariable, _ bool, _ bool, _ func(*sqltypes.Result) error) {
+ time.Sleep(primitiveLatency)
+ select {
+ case <-ctx.Done():
+ require.Fail(t, "primitive context done")
+ default:
+ }
+ }
+
+ var wg sync.WaitGroup
+ defer wg.Wait()
+ mirrorVC.onStreamExecuteMultiFn = func(ctx context.Context, _ Primitive, _ string, _ []*srvtopo.ResolvedShard, _ []map[string]*querypb.BindVariable, _ bool, _ bool, _ func(*sqltypes.Result) error) {
+ wg.Add(1)
+ defer wg.Done()
+ time.Sleep(primitiveLatency + maxMirrorTargetLag + (5 * time.Millisecond))
+ select {
+ case <-ctx.Done():
+ default:
+ require.Fail(t, "mirror target context not done")
+ }
+ }
+
+ want := vc.results[0]
+ err := mirror.TryStreamExecute(
+ context.Background(),
+ vc,
+ map[string]*querypb.BindVariable{},
+ true,
+ func(result *sqltypes.Result) error {
+ require.Equal(t, want, result)
+ return nil
+ },
+ )
+ require.NoError(t, err)
+
+ vc.ExpectLog(t, []string{
+ "ResolveDestinations ks1 [] Destinations:DestinationAllShards()",
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks1.0: {} ",
+ })
+ mirrorVC.ExpectLog(t, []string{
+ `ResolveDestinations ks2 [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(d46405367612b4b7)`,
+ "StreamExecuteMulti select f.bar from foo f where f.id = 1 ks2.-20: {} ",
+ })
+
+ require.Greater(t, *targetExecTime.Load(), *sourceExecTime.Load())
+ require.ErrorContains(t, *targetErr.Load(), "Mirror target query took too long")
+ })
+}
diff --git a/go/vt/vtgate/engine/plan.go b/go/vt/vtgate/engine/plan.go
index 769c69aaa06..9ea9f07655c 100644
--- a/go/vt/vtgate/engine/plan.go
+++ b/go/vt/vtgate/engine/plan.go
@@ -72,7 +72,7 @@ func (p *Plan) Stats() (execCount uint64, execTime time.Duration, shardQueries,
func (p *Plan) MarshalJSON() ([]byte, error) {
var instructions *PrimitiveDescription
if p.Instructions != nil {
- description := PrimitiveToPlanDescription(p.Instructions)
+ description := PrimitiveToPlanDescription(p.Instructions, nil)
instructions = &description
}
diff --git a/go/vt/vtgate/engine/plan_description.go b/go/vt/vtgate/engine/plan_description.go
index a8daa25ecd0..e8e763c1ee1 100644
--- a/go/vt/vtgate/engine/plan_description.go
+++ b/go/vt/vtgate/engine/plan_description.go
@@ -47,6 +47,9 @@ type PrimitiveDescription struct {
InputName string
Inputs []PrimitiveDescription
+
+ RowsReceived RowsReceived
+ ShardsQueried *ShardsQueried
}
// MarshalJSON serializes the PlanDescription into a JSON representation.
@@ -90,6 +93,23 @@ func (pd PrimitiveDescription) MarshalJSON() ([]byte, error) {
return nil, err
}
}
+ if len(pd.RowsReceived) > 0 {
+ if err := marshalAdd(prepend, buf, "NoOfCalls", len(pd.RowsReceived)); err != nil {
+ return nil, err
+ }
+
+ if err := marshalAdd(prepend, buf, "AvgNumberOfRows", average(pd.RowsReceived)); err != nil {
+ return nil, err
+ }
+ if err := marshalAdd(prepend, buf, "MedianNumberOfRows", median(pd.RowsReceived)); err != nil {
+ return nil, err
+ }
+ }
+ if pd.ShardsQueried != nil {
+ if err := marshalAdd(prepend, buf, "ShardsQueried", pd.ShardsQueried); err != nil {
+ return nil, err
+ }
+ }
err := addMap(pd.Other, buf)
if err != nil {
return nil, err
@@ -106,6 +126,155 @@ func (pd PrimitiveDescription) MarshalJSON() ([]byte, error) {
return buf.Bytes(), nil
}
+// PrimitiveDescriptionFromString creates primitive description out of a data string.
+func PrimitiveDescriptionFromString(data string) (pd PrimitiveDescription, err error) {
+ resultMap := make(map[string]any)
+ err = json.Unmarshal([]byte(data), &resultMap)
+ if err != nil {
+ return PrimitiveDescription{}, err
+ }
+ return PrimitiveDescriptionFromMap(resultMap)
+}
+
+// PrimitiveDescriptionFromMap populates the fields of a PrimitiveDescription from a map representation.
+func PrimitiveDescriptionFromMap(data map[string]any) (pd PrimitiveDescription, err error) {
+ if opType, isPresent := data["OperatorType"]; isPresent {
+ pd.OperatorType = opType.(string)
+ }
+ if variant, isPresent := data["Variant"]; isPresent {
+ pd.Variant = variant.(string)
+ }
+ if ksMap, isPresent := data["Keyspace"]; isPresent {
+ ksMap := ksMap.(map[string]any)
+ pd.Keyspace = &vindexes.Keyspace{
+ Name: ksMap["Name"].(string),
+ Sharded: ksMap["Sharded"].(bool),
+ }
+ }
+ if ttt, isPresent := data["TargetTabletType"]; isPresent {
+ pd.TargetTabletType = topodatapb.TabletType(ttt.(int))
+ }
+ if other, isPresent := data["Other"]; isPresent {
+ pd.Other = other.(map[string]any)
+ }
+ if inpName, isPresent := data["InputName"]; isPresent {
+ pd.InputName = inpName.(string)
+ }
+ if avgRows, isPresent := data["AvgNumberOfRows"]; isPresent {
+ pd.RowsReceived = RowsReceived{
+ int(avgRows.(float64)),
+ }
+ }
+ if sq, isPresent := data["ShardsQueried"]; isPresent {
+ sq := int(sq.(float64))
+ pd.ShardsQueried = (*ShardsQueried)(&sq)
+ }
+ if inputs, isPresent := data["Inputs"]; isPresent {
+ inputs := inputs.([]any)
+ for _, input := range inputs {
+ inputMap := input.(map[string]any)
+ inp, err := PrimitiveDescriptionFromMap(inputMap)
+ if err != nil {
+ return PrimitiveDescription{}, err
+ }
+ pd.Inputs = append(pd.Inputs, inp)
+ }
+ }
+ return pd, nil
+}
+
+// WalkPrimitiveDescription walks the primitive description.
+func WalkPrimitiveDescription(pd PrimitiveDescription, f func(PrimitiveDescription)) {
+ f(pd)
+ for _, child := range pd.Inputs {
+ WalkPrimitiveDescription(child, f)
+ }
+}
+
+func (pd PrimitiveDescription) Equals(other PrimitiveDescription) string {
+ if pd.Variant != other.Variant {
+ return fmt.Sprintf("Variant: %v != %v", pd.Variant, other.Variant)
+ }
+
+ if pd.OperatorType != other.OperatorType {
+ return fmt.Sprintf("OperatorType: %v != %v", pd.OperatorType, other.OperatorType)
+ }
+
+ // TODO (harshit): enable this to compare keyspace as well
+ // switch {
+ // case pd.Keyspace == nil && other.Keyspace == nil:
+ // // do nothing
+ // case pd.Keyspace != nil && other.Keyspace != nil:
+ // if pd.Keyspace.Name != other.Keyspace.Name {
+ // return fmt.Sprintf("Keyspace.Name: %v != %v", pd.Keyspace.Name, other.Keyspace.Name)
+ // }
+ // default:
+ // return "Keyspace is nil in one of the descriptions"
+ // }
+
+ switch {
+ case pd.TargetDestination == nil && other.TargetDestination == nil:
+ // do nothing
+ case pd.TargetDestination != nil && other.TargetDestination != nil:
+ if pd.TargetDestination.String() != other.TargetDestination.String() {
+ return fmt.Sprintf("TargetDestination: %v != %v", pd.TargetDestination, other.TargetDestination)
+ }
+ default:
+ return "TargetDestination is nil in one of the descriptions"
+ }
+
+ if pd.TargetTabletType != other.TargetTabletType {
+ return fmt.Sprintf("TargetTabletType: %v != %v", pd.TargetTabletType, other.TargetTabletType)
+ }
+
+ switch {
+ case pd.Other == nil && other.Other == nil:
+ // do nothing
+ case pd.Other != nil && other.Other != nil:
+ if len(pd.Other) != len(other.Other) {
+ return fmt.Sprintf("Other length did not match: %v != %v", pd.Other, other.Other)
+ }
+ for ky, val := range pd.Other {
+ if other.Other[ky] != val {
+ return fmt.Sprintf("Other[%v]: %v != %v", ky, val, other.Other[ky])
+ }
+ }
+ default:
+ return "Other is nil in one of the descriptions"
+ }
+ if len(pd.Inputs) != len(other.Inputs) {
+ return fmt.Sprintf("Inputs length did not match: %v != %v", len(pd.Inputs), len(other.Inputs))
+ }
+ for idx, input := range pd.Inputs {
+ if diff := input.Equals(other.Inputs[idx]); diff != "" {
+ return diff
+ }
+ }
+ return ""
+}
+
+func average(nums []int) float64 {
+ total := 0
+ for _, num := range nums {
+ total += num
+ }
+ return float64(total) / float64(len(nums))
+}
+
+func median(nums []int) float64 {
+ sortedNums := make([]int, len(nums))
+ copy(sortedNums, nums)
+ sort.Ints(sortedNums)
+
+ n := len(sortedNums)
+ if n%2 == 0 {
+ mid1 := sortedNums[n/2-1]
+ mid2 := sortedNums[n/2]
+ return float64(mid1+mid2) / 2.0
+ }
+ return float64(sortedNums[n/2])
+}
+
func (pd PrimitiveDescription) addToGraph(g *graphviz.Graph) (*graphviz.Node, error) {
var nodes []*graphviz.Node
for _, input := range pd.Inputs {
@@ -146,7 +315,7 @@ func (pd PrimitiveDescription) addToGraph(g *graphviz.Graph) (*graphviz.Node, er
func GraphViz(p Primitive) (*graphviz.Graph, error) {
g := graphviz.New()
- description := PrimitiveToPlanDescription(p)
+ description := PrimitiveToPlanDescription(p, nil)
_, err := description.addToGraph(g)
if err != nil {
return nil, err
@@ -182,12 +351,22 @@ func marshalAdd(prepend string, buf *bytes.Buffer, name string, obj any) error {
}
// PrimitiveToPlanDescription transforms a primitive tree into a corresponding PlanDescription tree
-func PrimitiveToPlanDescription(in Primitive) PrimitiveDescription {
+// If stats is not nil, it will be used to populate the stats field of the PlanDescription
+func PrimitiveToPlanDescription(in Primitive, stats *Stats) PrimitiveDescription {
this := in.description()
+ if stats != nil {
+ this.RowsReceived = stats.InterOpStats[in]
+
+ // Only applies to Route primitive
+ v, ok := stats.ShardsStats[in]
+ if ok {
+ this.ShardsQueried = &v
+ }
+ }
inputs, infos := in.Inputs()
for idx, input := range inputs {
- pd := PrimitiveToPlanDescription(input)
+ pd := PrimitiveToPlanDescription(input, stats)
if infos != nil {
for k, v := range infos[idx] {
if k == inputName {
diff --git a/go/vt/vtgate/engine/plan_description_test.go b/go/vt/vtgate/engine/plan_description_test.go
index dfed7d7f675..9f20e37976a 100644
--- a/go/vt/vtgate/engine/plan_description_test.go
+++ b/go/vt/vtgate/engine/plan_description_test.go
@@ -31,7 +31,7 @@ import (
func TestCreateRoutePlanDescription(t *testing.T) {
route := createRoute()
- planDescription := PrimitiveToPlanDescription(route)
+ planDescription := PrimitiveToPlanDescription(route, nil)
expected := PrimitiveDescription{
OperatorType: "Route",
@@ -76,7 +76,7 @@ func TestPlanDescriptionWithInputs(t *testing.T) {
Input: route,
}
- planDescription := PrimitiveToPlanDescription(limit)
+ planDescription := PrimitiveToPlanDescription(limit, nil)
expected := PrimitiveDescription{
OperatorType: "Limit",
diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go
index 0c754a00342..4f3a388d04f 100644
--- a/go/vt/vtgate/engine/primitive.go
+++ b/go/vt/vtgate/engine/primitive.go
@@ -132,8 +132,21 @@ type (
// CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas
CloneForReplicaWarming(ctx context.Context) VCursor
+ // CloneForMirroring clones the VCursor for re-use in mirroring queries to other keyspaces
+ CloneForMirroring(ctx context.Context) VCursor
+ //
// ReadTransaction reads the state of the given transaction from the metadata manager
ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error)
+
+ // UnresolvedTransactions reads the state of all the unresolved atomic transactions in the given keyspace.
+ UnresolvedTransactions(ctx context.Context, keyspace string) ([]*querypb.TransactionMetadata, error)
+
+ // StartPrimitiveTrace starts a trace for the given primitive,
+ // and returns a function to get the trace logs after the primitive execution.
+ StartPrimitiveTrace() func() Stats
+
+ // RecordMirrorStats is used to record stats about a mirror query.
+ RecordMirrorStats(time.Duration, time.Duration, error)
}
// SessionActions gives primitives ability to interact with the session state
@@ -167,6 +180,7 @@ type (
SetConsolidator(querypb.ExecuteOptions_Consolidator)
SetWorkloadName(string)
SetPriority(string)
+ SetExecQueryTimeout(timeout *int)
SetFoundRows(uint64)
SetDDLStrategy(string)
@@ -209,9 +223,6 @@ type (
// This is used to select the right shard session to perform the vindex lookup query.
SetCommitOrder(co vtgatepb.CommitOrder)
- // GetQueryTimeout gets the query timeout and takes in the query timeout from comments
- GetQueryTimeout(queryTimeoutFromComment int) int
-
// SetQueryTimeout sets the query timeout
SetQueryTimeout(queryTimeout int64)
diff --git a/go/vt/vtgate/engine/recurse_cte.go b/go/vt/vtgate/engine/recurse_cte.go
new file mode 100644
index 00000000000..f523883d280
--- /dev/null
+++ b/go/vt/vtgate/engine/recurse_cte.go
@@ -0,0 +1,155 @@
+/*
+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 engine
+
+import (
+ "context"
+
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/vterrors"
+)
+
+// RecurseCTE is used to represent recursive CTEs
+// Seed is used to represent the non-recursive part that initializes the result set.
+// It's result are then used to start the recursion on the Term side
+// The values being sent to the Term side are stored in the Vars map -
+// the key is the bindvar name and the value is the index of the column in the recursive result
+type RecurseCTE struct {
+ Seed, Term Primitive
+
+ Vars map[string]int
+}
+
+var _ Primitive = (*RecurseCTE)(nil)
+
+func (r *RecurseCTE) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
+ res, err := vcursor.ExecutePrimitive(ctx, r.Seed, bindVars, wantfields)
+ if err != nil {
+ return nil, err
+ }
+
+ // recurseRows contains the rows used in the next recursion
+ recurseRows := res.Rows
+ joinVars := make(map[string]*querypb.BindVariable)
+ loops := 0
+ for len(recurseRows) > 0 {
+ // copy over the results from the previous recursion
+ theseRows := recurseRows
+ recurseRows = nil
+ for _, row := range theseRows {
+ for k, col := range r.Vars {
+ joinVars[k] = sqltypes.ValueBindVariable(row[col])
+ }
+ // check if the context is done - we might be in a long running recursion
+ if err := ctx.Err(); err != nil {
+ return nil, err
+ }
+ rresult, err := vcursor.ExecutePrimitive(ctx, r.Term, combineVars(bindVars, joinVars), false)
+ if err != nil {
+ return nil, err
+ }
+ recurseRows = append(recurseRows, rresult.Rows...)
+ res.Rows = append(res.Rows, rresult.Rows...)
+ loops++
+ if loops > 1000 { // TODO: This should be controlled with a system variable setting
+ return nil, vterrors.VT09030("")
+ }
+ }
+ }
+ return res, nil
+}
+
+func (r *RecurseCTE) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
+ if vcursor.Session().InTransaction() {
+ res, err := r.TryExecute(ctx, vcursor, bindVars, wantfields)
+ if err != nil {
+ return err
+ }
+ return callback(res)
+ }
+ return vcursor.StreamExecutePrimitive(ctx, r.Seed, bindVars, wantfields, func(result *sqltypes.Result) error {
+ err := callback(result)
+ if err != nil {
+ return err
+ }
+ return r.recurse(ctx, vcursor, bindVars, result, callback)
+ })
+}
+
+func (r *RecurseCTE) recurse(ctx context.Context, vcursor VCursor, bindvars map[string]*querypb.BindVariable, result *sqltypes.Result, callback func(*sqltypes.Result) error) error {
+ if len(result.Rows) == 0 {
+ return nil
+ }
+ joinVars := make(map[string]*querypb.BindVariable)
+ for _, row := range result.Rows {
+ for k, col := range r.Vars {
+ joinVars[k] = sqltypes.ValueBindVariable(row[col])
+ }
+
+ err := vcursor.StreamExecutePrimitive(ctx, r.Term, combineVars(bindvars, joinVars), false, func(result *sqltypes.Result) error {
+ err := callback(result)
+ if err != nil {
+ return err
+ }
+ return r.recurse(ctx, vcursor, bindvars, result, callback)
+ })
+ if err != nil {
+ return err
+ }
+ }
+ return nil
+}
+
+func (r *RecurseCTE) RouteType() string {
+ return "RecurseCTE"
+}
+
+func (r *RecurseCTE) GetKeyspaceName() string {
+ if r.Seed.GetKeyspaceName() == r.Term.GetKeyspaceName() {
+ return r.Seed.GetKeyspaceName()
+ }
+ return r.Seed.GetKeyspaceName() + "_" + r.Term.GetKeyspaceName()
+}
+
+func (r *RecurseCTE) GetTableName() string {
+ return r.Seed.GetTableName()
+}
+
+func (r *RecurseCTE) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) {
+ return r.Seed.GetFields(ctx, vcursor, bindVars)
+}
+
+func (r *RecurseCTE) NeedsTransaction() bool {
+ return r.Seed.NeedsTransaction() || r.Term.NeedsTransaction()
+}
+
+func (r *RecurseCTE) Inputs() ([]Primitive, []map[string]any) {
+ return []Primitive{r.Seed, r.Term}, nil
+}
+
+func (r *RecurseCTE) description() PrimitiveDescription {
+ other := map[string]interface{}{
+ "JoinVars": orderedStringIntMap(r.Vars),
+ }
+
+ return PrimitiveDescription{
+ OperatorType: "RecurseCTE",
+ Other: other,
+ Inputs: nil,
+ }
+}
diff --git a/go/vt/vtgate/engine/recurse_cte_test.go b/go/vt/vtgate/engine/recurse_cte_test.go
new file mode 100644
index 00000000000..d6826284d21
--- /dev/null
+++ b/go/vt/vtgate/engine/recurse_cte_test.go
@@ -0,0 +1,129 @@
+/*
+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 engine
+
+import (
+ "context"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+)
+
+func TestRecurseDualQuery(t *testing.T) {
+ // Test that the RecurseCTE primitive works as expected.
+ // The test is testing something like this:
+ // WITH RECURSIVE cte AS (SELECT 1 as col1 UNION SELECT col1+1 FROM cte WHERE col1 < 5) SELECT * FROM cte;
+ leftPrim := &fakePrimitive{
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "col1",
+ "int64",
+ ),
+ "1",
+ ),
+ },
+ }
+ rightFields := sqltypes.MakeTestFields(
+ "col4",
+ "int64",
+ )
+
+ rightPrim := &fakePrimitive{
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ rightFields,
+ "2",
+ ),
+ sqltypes.MakeTestResult(
+ rightFields,
+ "3",
+ ),
+ sqltypes.MakeTestResult(
+ rightFields,
+ "4",
+ ), sqltypes.MakeTestResult(
+ rightFields,
+ ),
+ },
+ }
+ bv := map[string]*querypb.BindVariable{}
+
+ cte := &RecurseCTE{
+ Seed: leftPrim,
+ Term: rightPrim,
+ Vars: map[string]int{"col1": 0},
+ }
+
+ r, err := cte.TryExecute(context.Background(), &noopVCursor{}, bv, true)
+ require.NoError(t, err)
+
+ rightPrim.ExpectLog(t, []string{
+ `Execute col1: type:INT64 value:"1" false`,
+ `Execute col1: type:INT64 value:"2" false`,
+ `Execute col1: type:INT64 value:"3" false`,
+ `Execute col1: type:INT64 value:"4" false`,
+ })
+
+ wantRes := sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "col1",
+ "int64",
+ ),
+ "1",
+ "2",
+ "3",
+ "4",
+ )
+ expectResult(t, r, wantRes)
+
+ // testing the streaming mode.
+
+ leftPrim.rewind()
+ rightPrim.rewind()
+
+ r, err = wrapStreamExecute(cte, &noopVCursor{}, bv, true)
+ require.NoError(t, err)
+
+ rightPrim.ExpectLog(t, []string{
+ `StreamExecute col1: type:INT64 value:"1" false`,
+ `StreamExecute col1: type:INT64 value:"2" false`,
+ `StreamExecute col1: type:INT64 value:"3" false`,
+ `StreamExecute col1: type:INT64 value:"4" false`,
+ })
+ expectResult(t, r, wantRes)
+
+ // testing the streaming mode with transaction
+
+ leftPrim.rewind()
+ rightPrim.rewind()
+
+ r, err = wrapStreamExecute(cte, &noopVCursor{inTx: true}, bv, true)
+ require.NoError(t, err)
+
+ rightPrim.ExpectLog(t, []string{
+ `Execute col1: type:INT64 value:"1" false`,
+ `Execute col1: type:INT64 value:"2" false`,
+ `Execute col1: type:INT64 value:"3" false`,
+ `Execute col1: type:INT64 value:"4" false`,
+ })
+ expectResult(t, r, wantRes)
+
+}
diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go
index f28dda01a52..59682dd91fe 100644
--- a/go/vt/vtgate/engine/route.go
+++ b/go/vt/vtgate/engine/route.go
@@ -130,22 +130,12 @@ func (route *Route) GetTableName() string {
// TryExecute performs a non-streaming exec.
func (route *Route) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, route.QueryTimeout)
- defer cancelFunc()
- qr, err := route.executeInternal(ctx, vcursor, bindVars, wantfields)
+ rss, bvs, err := route.findRoute(ctx, vcursor, bindVars)
if err != nil {
return nil, err
}
- return qr.Truncate(route.TruncateColumnCount), nil
-}
-// addQueryTimeout adds a query timeout to the context it receives and returns the modified context along with the cancel function.
-func addQueryTimeout(ctx context.Context, vcursor VCursor, queryTimeout int) (context.Context, context.CancelFunc) {
- timeout := vcursor.Session().GetQueryTimeout(queryTimeout)
- if timeout != 0 {
- return context.WithTimeout(ctx, time.Duration(timeout)*time.Millisecond)
- }
- return ctx, func() {}
+ return route.executeShards(ctx, vcursor, bindVars, wantfields, rss, bvs)
}
type cxtKey int
@@ -154,20 +144,6 @@ const (
IgnoreReserveTxn cxtKey = iota
)
-func (route *Route) executeInternal(
- ctx context.Context,
- vcursor VCursor,
- bindVars map[string]*querypb.BindVariable,
- wantfields bool,
-) (*sqltypes.Result, error) {
- rss, bvs, err := route.findRoute(ctx, vcursor, bindVars)
- if err != nil {
- return nil, err
- }
-
- return route.executeShards(ctx, vcursor, bindVars, wantfields, rss, bvs)
-}
-
func (route *Route) executeShards(
ctx context.Context,
vcursor VCursor,
@@ -223,11 +199,15 @@ func (route *Route) executeShards(
}
}
- if len(route.OrderBy) == 0 {
- return result, nil
+ if len(route.OrderBy) != 0 {
+ var err error
+ result, err = route.sort(result)
+ if err != nil {
+ return nil, err
+ }
}
- return route.sort(result)
+ return result.Truncate(route.TruncateColumnCount), nil
}
func filterOutNilErrors(errs []error) []error {
@@ -384,10 +364,8 @@ func (route *Route) sort(in *sqltypes.Result) (*sqltypes.Result, error) {
// the contents of any row.
out := in.ShallowCopy()
- if err := route.OrderBy.SortResult(out); err != nil {
- return nil, err
- }
- return out.Truncate(route.TruncateColumnCount), nil
+ err := route.OrderBy.SortResult(out)
+ return out, err
}
func (route *Route) description() PrimitiveDescription {
diff --git a/go/vt/vtgate/engine/send.go b/go/vt/vtgate/engine/send.go
index 31c9e9e0eb0..2ebec5c679e 100644
--- a/go/vt/vtgate/engine/send.go
+++ b/go/vt/vtgate/engine/send.go
@@ -47,6 +47,8 @@ type Send struct {
// IsDML specifies how to deal with autocommit behaviour
IsDML bool
+ IsDDL bool
+
// SingleShardOnly specifies that the query must be send to only single shard
SingleShardOnly bool
@@ -91,8 +93,9 @@ func (s *Send) GetTableName() string {
// TryExecute implements Primitive interface
func (s *Send) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, s.QueryTimeout)
- defer cancelFunc()
+ if err := s.commitIfDDL(ctx, vcursor); err != nil {
+ return nil, err
+ }
rss, err := s.checkAndReturnShards(ctx, vcursor)
if err != nil {
@@ -158,6 +161,10 @@ func copyBindVars(in map[string]*querypb.BindVariable) map[string]*querypb.BindV
// TryStreamExecute implements Primitive interface
func (s *Send) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
+ if err := s.commitIfDDL(ctx, vcursor); err != nil {
+ return err
+ }
+
rss, err := s.checkAndReturnShards(ctx, vcursor)
if err != nil {
return err
@@ -204,3 +211,11 @@ func (s *Send) description() PrimitiveDescription {
Other: other,
}
}
+
+// commitIfDDL commits any open transaction before executing the ddl query.
+func (s *Send) commitIfDDL(ctx context.Context, vcursor VCursor) error {
+ if s.IsDDL {
+ return vcursor.Session().Commit(ctx)
+ }
+ return nil
+}
diff --git a/go/vt/vtgate/engine/transaction_status.go b/go/vt/vtgate/engine/transaction_status.go
index 61cc72c08d9..9914031009f 100644
--- a/go/vt/vtgate/engine/transaction_status.go
+++ b/go/vt/vtgate/engine/transaction_status.go
@@ -33,6 +33,7 @@ type TransactionStatus struct {
noInputs
noTxNeeded
+ Keyspace string
TransactionID string
}
@@ -76,25 +77,37 @@ func (t *TransactionStatus) getFields() []*querypb.Field {
}
func (t *TransactionStatus) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- transactionState, err := vcursor.ReadTransaction(ctx, t.TransactionID)
+ var transactionStatuses []*querypb.TransactionMetadata
+ var err error
+ if t.TransactionID != "" {
+ var transactionState *querypb.TransactionMetadata
+ transactionState, err = vcursor.ReadTransaction(ctx, t.TransactionID)
+ transactionStatuses = append(transactionStatuses, transactionState)
+ } else {
+ transactionStatuses, err = vcursor.UnresolvedTransactions(ctx, t.Keyspace)
+ }
if err != nil {
return nil, err
}
+
res := &sqltypes.Result{}
if wantfields {
res.Fields = t.getFields()
}
- if transactionState != nil && transactionState.Dtid != "" {
- var participantString []string
- for _, participant := range transactionState.Participants {
- participantString = append(participantString, fmt.Sprintf("%s:%s", participant.Keyspace, participant.Shard))
+
+ for _, transactionState := range transactionStatuses {
+ if transactionState != nil && transactionState.Dtid != "" {
+ var participantString []string
+ for _, participant := range transactionState.Participants {
+ participantString = append(participantString, fmt.Sprintf("%s:%s", participant.Keyspace, participant.Shard))
+ }
+ res.Rows = append(res.Rows, sqltypes.Row{
+ sqltypes.NewVarChar(transactionState.Dtid),
+ sqltypes.NewVarChar(transactionState.State.String()),
+ sqltypes.NewDatetime(time.Unix(0, transactionState.TimeCreated).UTC().String()),
+ sqltypes.NewVarChar(strings.Join(participantString, ",")),
+ })
}
- res.Rows = append(res.Rows, sqltypes.Row{
- sqltypes.NewVarChar(transactionState.Dtid),
- sqltypes.NewVarChar(transactionState.State.String()),
- sqltypes.NewDatetime(time.Unix(0, transactionState.TimeCreated).UTC().String()),
- sqltypes.NewVarChar(strings.Join(participantString, ",")),
- })
}
return res, nil
}
@@ -108,10 +121,14 @@ func (t *TransactionStatus) TryStreamExecute(ctx context.Context, vcursor VCurso
}
func (t *TransactionStatus) description() PrimitiveDescription {
+ otherMap := map[string]any{}
+ if t.TransactionID == "" {
+ otherMap["Keyspace"] = t.Keyspace
+ } else {
+ otherMap["TransactionID"] = t.TransactionID
+ }
return PrimitiveDescription{
OperatorType: "TransactionStatus",
- Other: map[string]any{
- "TransactionID": t.TransactionID,
- },
+ Other: otherMap,
}
}
diff --git a/go/vt/vtgate/engine/transaction_status_test.go b/go/vt/vtgate/engine/transaction_status_test.go
index 5e4e95be8ef..739a1d32cce 100644
--- a/go/vt/vtgate/engine/transaction_status_test.go
+++ b/go/vt/vtgate/engine/transaction_status_test.go
@@ -32,33 +32,40 @@ import (
func TestTransactionStatusOutput(t *testing.T) {
tests := []struct {
name string
- transactionStatusOutput *querypb.TransactionMetadata
+ transactionStatusOutput []*querypb.TransactionMetadata
resultErr error
expectedRes *sqltypes.Result
+ primitive *TransactionStatus
}{
{
name: "Empty Transaction Status",
transactionStatusOutput: nil,
expectedRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("id|state|record_time|participants", "varchar|varchar|datetime|varchar")),
+ primitive: &TransactionStatus{},
}, {
- name: "Valid Transaction Status",
- transactionStatusOutput: &querypb.TransactionMetadata{
- Dtid: "ks:-80:v24s7843sf78934l3",
- State: querypb.TransactionState_PREPARE,
- TimeCreated: 1257894000000000000,
- Participants: []*querypb.Target{
- {
- Keyspace: "ks",
- Shard: "-80",
- TabletType: topodatapb.TabletType_PRIMARY,
- }, {
- Keyspace: "ks",
- Shard: "80-a0",
- TabletType: topodatapb.TabletType_PRIMARY,
- }, {
- Keyspace: "ks",
- Shard: "a0-",
- TabletType: topodatapb.TabletType_PRIMARY,
+ name: "Valid Transaction Status for a transaction ID",
+ primitive: &TransactionStatus{
+ TransactionID: "ks:-80:v24s7843sf78934l3",
+ },
+ transactionStatusOutput: []*querypb.TransactionMetadata{
+ {
+ Dtid: "ks:-80:v24s7843sf78934l3",
+ State: querypb.TransactionState_PREPARE,
+ TimeCreated: 1257894000000000000,
+ Participants: []*querypb.Target{
+ {
+ Keyspace: "ks",
+ Shard: "-80",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ Keyspace: "ks",
+ Shard: "80-a0",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ Keyspace: "ks",
+ Shard: "a0-",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ },
},
},
},
@@ -66,15 +73,65 @@ func TestTransactionStatusOutput(t *testing.T) {
sqltypes.MakeTestFields("id|state|record_time|participants", "varchar|varchar|datetime|varchar"),
"ks:-80:v24s7843sf78934l3|PREPARE|2009-11-10 23:00:00 +0000 UTC|ks:-80,ks:80-a0,ks:a0-"),
}, {
- name: "Error getting transaction metadata",
+ name: "Error getting transaction metadata",
+ primitive: &TransactionStatus{
+ TransactionID: "ks:-80:v24s7843sf78934l3",
+ },
resultErr: fmt.Errorf("failed reading transaction state"),
+ }, {
+ name: "Valid Transaction Statuses for a keyspace",
+ primitive: &TransactionStatus{
+ Keyspace: "ks",
+ },
+ transactionStatusOutput: []*querypb.TransactionMetadata{
+ {
+ Dtid: "ks:-80:v24s7843sf78934l3",
+ State: querypb.TransactionState_PREPARE,
+ TimeCreated: 1257894000000000000,
+ Participants: []*querypb.Target{
+ {
+ Keyspace: "ks",
+ Shard: "-80",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ Keyspace: "ks",
+ Shard: "80-a0",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ Keyspace: "ks",
+ Shard: "a0-",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ },
+ },
+ },
+ {
+ Dtid: "ks:-80:v34afdfdsfdfd",
+ State: querypb.TransactionState_PREPARE,
+ TimeCreated: 1259894000000000000,
+ Participants: []*querypb.Target{
+ {
+ Keyspace: "ks",
+ Shard: "-80",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ Keyspace: "ks",
+ Shard: "80-",
+ TabletType: topodatapb.TabletType_PRIMARY,
+ },
+ },
+ },
+ },
+ expectedRes: sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields("id|state|record_time|participants", "varchar|varchar|datetime|varchar"),
+ "ks:-80:v24s7843sf78934l3|PREPARE|2009-11-10 23:00:00 +0000 UTC|ks:-80,ks:80-a0,ks:a0-",
+ "ks:-80:v34afdfdsfdfd|PREPARE|2009-12-04 02:33:20 +0000 UTC|ks:-80,ks:80-",
+ ),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
- ts := &TransactionStatus{}
- res, err := ts.TryExecute(context.Background(), &loggingVCursor{
+ res, err := test.primitive.TryExecute(context.Background(), &loggingVCursor{
transactionStatusOutput: test.transactionStatusOutput,
resultErr: test.resultErr,
}, nil, true)
diff --git a/go/vt/vtgate/engine/update.go b/go/vt/vtgate/engine/update.go
index 13c590bbb63..27ca9ad12a1 100644
--- a/go/vt/vtgate/engine/update.go
+++ b/go/vt/vtgate/engine/update.go
@@ -53,9 +53,6 @@ type Update struct {
// TryExecute performs a non-streaming exec.
func (upd *Update) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- ctx, cancelFunc := addQueryTimeout(ctx, vcursor, upd.QueryTimeout)
- defer cancelFunc()
-
rss, bvs, err := upd.findRoute(ctx, vcursor, bindVars)
if err != nil {
return nil, err
diff --git a/go/vt/vtgate/engine/vexplain.go b/go/vt/vtgate/engine/vexplain.go
index 010901021fa..78941e8160f 100644
--- a/go/vt/vtgate/engine/vexplain.go
+++ b/go/vt/vtgate/engine/vexplain.go
@@ -21,6 +21,7 @@ import (
"encoding/json"
"fmt"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
@@ -42,6 +43,14 @@ type (
Input Primitive
Type sqlparser.VExplainType
}
+
+ ShardsQueried int
+ RowsReceived []int
+
+ Stats struct {
+ InterOpStats map[Primitive]RowsReceived
+ ShardsStats map[Primitive]ShardsQueried
+ }
)
var _ Primitive = (*VExplain)(nil)
@@ -62,8 +71,43 @@ func (v *VExplain) GetTableName() string {
}
// GetFields implements the Primitive interface
-func (v *VExplain) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) {
- return v.Input.GetFields(ctx, vcursor, bindVars)
+func (v *VExplain) GetFields(context.Context, VCursor, map[string]*querypb.BindVariable) (*sqltypes.Result, error) {
+ var fields []*querypb.Field
+ switch v.Type {
+ case sqlparser.QueriesVExplainType:
+ fields = getVExplainQueriesFields()
+ case sqlparser.AllVExplainType:
+ fields = getVExplainAllFields()
+ case sqlparser.TraceVExplainType:
+ fields = getVExplainTraceFields()
+ default:
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "Unknown type of VExplain plan")
+ }
+ return &sqltypes.Result{Fields: fields}, nil
+}
+
+func getVExplainTraceFields() []*querypb.Field {
+ return []*querypb.Field{{
+ Name: "Trace",
+ Type: sqltypes.VarChar,
+ Charset: uint32(collations.SystemCollation.Collation),
+ Flags: uint32(querypb.MySqlFlag_NOT_NULL_FLAG),
+ }}
+}
+
+func getVExplainQueriesFields() []*querypb.Field {
+ return []*querypb.Field{
+ {Name: "#", Type: sqltypes.Int32},
+ {Name: "keyspace", Type: sqltypes.VarChar},
+ {Name: "shard", Type: sqltypes.VarChar},
+ {Name: "query", Type: sqltypes.VarChar}}
+
+}
+
+func getVExplainAllFields() []*querypb.Field {
+ return []*querypb.Field{{
+ Name: "VExplain", Type: sqltypes.VarChar,
+ }}
}
// NeedsTransaction implements the Primitive interface
@@ -73,42 +117,75 @@ func (v *VExplain) NeedsTransaction() bool {
// TryExecute implements the Primitive interface
func (v *VExplain) TryExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- vcursor.Session().VExplainLogging()
+ var stats func() Stats
+ if v.Type == sqlparser.TraceVExplainType {
+ stats = vcursor.StartPrimitiveTrace()
+ } else {
+ vcursor.Session().VExplainLogging()
+ }
_, err := vcursor.ExecutePrimitive(ctx, v.Input, bindVars, wantfields)
if err != nil {
return nil, err
}
- return v.convertToResult(ctx, vcursor)
+ return v.convertToResult(ctx, vcursor, stats)
+}
+
+func noOpCallback(*sqltypes.Result) error {
+ return nil
}
// TryStreamExecute implements the Primitive interface
func (v *VExplain) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
- vcursor.Session().VExplainLogging()
- err := vcursor.StreamExecutePrimitive(ctx, v.Input, bindVars, wantfields, func(result *sqltypes.Result) error {
- return nil
- })
+ var stats func() Stats
+ if v.Type == sqlparser.TraceVExplainType {
+ stats = vcursor.StartPrimitiveTrace()
+ } else {
+ vcursor.Session().VExplainLogging()
+ }
+
+ err := vcursor.StreamExecutePrimitive(ctx, v.Input, bindVars, wantfields, noOpCallback)
if err != nil {
return err
}
- result, err := v.convertToResult(ctx, vcursor)
+ result, err := v.convertToResult(ctx, vcursor, stats)
if err != nil {
return err
}
return callback(result)
}
-func (v *VExplain) convertToResult(ctx context.Context, vcursor VCursor) (*sqltypes.Result, error) {
+func (v *VExplain) convertToResult(ctx context.Context, vcursor VCursor, stats func() Stats) (*sqltypes.Result, error) {
switch v.Type {
case sqlparser.QueriesVExplainType:
result := convertToVExplainQueriesResult(vcursor.Session().GetVExplainLogs())
return result, nil
case sqlparser.AllVExplainType:
return v.convertToVExplainAllResult(ctx, vcursor)
+ case sqlparser.TraceVExplainType:
+ return v.getExplainTraceOutput(stats)
+
default:
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "Unknown type of VExplain plan")
}
}
+func (v *VExplain) getExplainTraceOutput(getOpStats func() Stats) (*sqltypes.Result, error) {
+ stats := getOpStats()
+ description := PrimitiveToPlanDescription(v.Input, &stats)
+
+ output, err := json.MarshalIndent(description, "", "\t")
+ if err != nil {
+ return nil, err
+ }
+
+ return &sqltypes.Result{
+ Fields: getVExplainTraceFields(),
+ Rows: []sqltypes.Row{{
+ sqltypes.NewVarChar(string(output)),
+ }},
+ }, nil
+}
+
func (v *VExplain) convertToVExplainAllResult(ctx context.Context, vcursor VCursor) (*sqltypes.Result, error) {
logEntries := vcursor.Session().GetVExplainLogs()
explainResults := make(map[Primitive]string)
@@ -144,18 +221,14 @@ func (v *VExplain) convertToVExplainAllResult(ctx context.Context, vcursor VCurs
}
result := string(resultBytes)
- fields := []*querypb.Field{
- {
- Name: "VExplain", Type: sqltypes.VarChar,
- },
- }
+
rows := []sqltypes.Row{
{
sqltypes.NewVarChar(result),
},
}
qr := &sqltypes.Result{
- Fields: fields,
+ Fields: getVExplainAllFields(),
Rows: rows,
}
return qr, nil
@@ -193,17 +266,8 @@ func primitiveToPlanDescriptionWithSQLResults(in Primitive, res map[Primitive]st
}
func convertToVExplainQueriesResult(logs []ExecuteEntry) *sqltypes.Result {
- fields := []*querypb.Field{{
- Name: "#", Type: sqltypes.Int32,
- }, {
- Name: "keyspace", Type: sqltypes.VarChar,
- }, {
- Name: "shard", Type: sqltypes.VarChar,
- }, {
- Name: "query", Type: sqltypes.VarChar,
- }}
qr := &sqltypes.Result{
- Fields: fields,
+ Fields: getVExplainQueriesFields(),
}
for _, line := range logs {
qr.Rows = append(qr.Rows, sqltypes.Row{
diff --git a/go/vt/vtgate/engine/vindex_lookup.go b/go/vt/vtgate/engine/vindex_lookup.go
index 8bf8755c40e..2e0e2857498 100644
--- a/go/vt/vtgate/engine/vindex_lookup.go
+++ b/go/vt/vtgate/engine/vindex_lookup.go
@@ -252,7 +252,7 @@ func (vr *VindexLookup) generateIds(ctx context.Context, vcursor VCursor, bindVa
switch vr.Opcode {
case Equal, EqualUnique:
return []sqltypes.Value{value.Value(vcursor.ConnCollation())}, nil
- case IN:
+ case IN, MultiEqual:
return value.TupleValues(), nil
}
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "opcode %s not supported for VindexLookup", vr.Opcode.String())
diff --git a/go/vt/vtgate/engine/vindex_lookup_test.go b/go/vt/vtgate/engine/vindex_lookup_test.go
new file mode 100644
index 00000000000..d734bf12080
--- /dev/null
+++ b/go/vt/vtgate/engine/vindex_lookup_test.go
@@ -0,0 +1,135 @@
+/*
+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 engine
+
+import (
+ "context"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/vtgate/evalengine"
+ "vitess.io/vitess/go/vt/vtgate/vindexes"
+)
+
+var (
+ vindex, _ = vindexes.CreateVindex("lookup_unique", "", map[string]string{
+ "table": "lkp",
+ "from": "from",
+ "to": "toc",
+ "write_only": "true",
+ })
+ ks = &vindexes.Keyspace{Name: "ks", Sharded: true}
+)
+
+func TestVindexLookup(t *testing.T) {
+ planableVindex, ok := vindex.(vindexes.LookupPlanable)
+ require.True(t, ok, "not a lookup vindex")
+ _, args := planableVindex.Query()
+
+ fp := &fakePrimitive{
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields("id|keyspace_id", "int64|varbinary"),
+ "1|\x10"),
+ },
+ }
+ route := NewRoute(ByDestination, ks, "dummy_select", "dummy_select_field")
+ vdxLookup := &VindexLookup{
+ Opcode: EqualUnique,
+ Keyspace: ks,
+ Vindex: planableVindex,
+ Arguments: args,
+ Values: []evalengine.Expr{evalengine.NewLiteralInt(1)},
+ Lookup: fp,
+ SendTo: route,
+ }
+
+ vc := &loggingVCursor{results: []*sqltypes.Result{defaultSelectResult}}
+
+ result, err := vdxLookup.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, false)
+ require.NoError(t, err)
+ fp.ExpectLog(t, []string{`Execute from: type:TUPLE values:{type:INT64 value:"1"} false`})
+ vc.ExpectLog(t, []string{
+ `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(10)`,
+ `ExecuteMultiShard ks.-20: dummy_select {} false false`,
+ })
+ expectResult(t, result, defaultSelectResult)
+
+ fp.rewind()
+ vc.Rewind()
+ result, err = wrapStreamExecute(vdxLookup, vc, map[string]*querypb.BindVariable{}, false)
+ require.NoError(t, err)
+ vc.ExpectLog(t, []string{
+ `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(10)`,
+ `StreamExecuteMulti dummy_select ks.-20: {} `,
+ })
+ expectResult(t, result, defaultSelectResult)
+}
+
+func TestVindexLookupTruncate(t *testing.T) {
+ planableVindex, ok := vindex.(vindexes.LookupPlanable)
+ require.True(t, ok, "not a lookup vindex")
+ _, args := planableVindex.Query()
+
+ fp := &fakePrimitive{
+ results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields("id|keyspace_id", "int64|varbinary"),
+ "1|\x10"),
+ },
+ }
+ route := NewRoute(ByDestination, ks, "dummy_select", "dummy_select_field")
+ route.TruncateColumnCount = 1
+ vdxLookup := &VindexLookup{
+ Opcode: EqualUnique,
+ Keyspace: ks,
+ Vindex: planableVindex,
+ Arguments: args,
+ Values: []evalengine.Expr{evalengine.NewLiteralInt(1)},
+ Lookup: fp,
+ SendTo: route,
+ }
+
+ vc := &loggingVCursor{results: []*sqltypes.Result{
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("name|morecol", "varchar|int64"),
+ "foo|1", "bar|2", "baz|3"),
+ }}
+
+ wantRes := sqltypes.MakeTestResult(sqltypes.MakeTestFields("name", "varchar"),
+ "foo", "bar", "baz")
+ result, err := vdxLookup.TryExecute(context.Background(), vc, map[string]*querypb.BindVariable{}, false)
+ require.NoError(t, err)
+ fp.ExpectLog(t, []string{`Execute from: type:TUPLE values:{type:INT64 value:"1"} false`})
+ vc.ExpectLog(t, []string{
+ `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(10)`,
+ `ExecuteMultiShard ks.-20: dummy_select {} false false`,
+ })
+ expectResult(t, result, wantRes)
+
+ fp.rewind()
+ vc.Rewind()
+ result, err = wrapStreamExecute(vdxLookup, vc, map[string]*querypb.BindVariable{}, false)
+ require.NoError(t, err)
+ vc.ExpectLog(t, []string{
+ `ResolveDestinations ks [type:INT64 value:"1"] Destinations:DestinationKeyspaceID(10)`,
+ `StreamExecuteMulti dummy_select ks.-20: {} `,
+ })
+ expectResult(t, result, wantRes)
+}
diff --git a/go/vt/vtgate/evalengine/compiler_test.go b/go/vt/vtgate/evalengine/compiler_test.go
index 04eb72ad4f2..cb9b99e7776 100644
--- a/go/vt/vtgate/evalengine/compiler_test.go
+++ b/go/vt/vtgate/evalengine/compiler_test.go
@@ -24,6 +24,8 @@ import (
"testing"
"time"
+ "github.com/stretchr/testify/assert"
+
"github.com/olekukonko/tablewriter"
"github.com/stretchr/testify/require"
@@ -94,7 +96,18 @@ func (s *Tracker) String() string {
return s.buf.String()
}
+func TestOneCase(t *testing.T) {
+ query := ``
+ if query == "" {
+ t.Skip("no query to test")
+ }
+ venv := vtenv.NewTestEnv()
+ env := evalengine.EmptyExpressionEnv(venv)
+ testCompilerCase(t, query, venv, nil, env)
+}
+
func TestCompilerReference(t *testing.T) {
+ // This test runs a lot of queries and compares the results of the evalengine in eval mode to the results of the compiler.
now := time.Now()
evalengine.SystemTime = func() time.Time { return now }
defer func() { evalengine.SystemTime = time.Now }()
@@ -108,52 +121,11 @@ func TestCompilerReference(t *testing.T) {
tc.Run(func(query string, row []sqltypes.Value) {
env.Row = row
-
- stmt, err := venv.Parser().ParseExpr(query)
- if err != nil {
- // no need to test un-parseable queries
- return
- }
-
- fields := evalengine.FieldResolver(tc.Schema)
- cfg := &evalengine.Config{
- ResolveColumn: fields.Column,
- ResolveType: fields.Type,
- Collation: collations.CollationUtf8mb4ID,
- Environment: venv,
- NoConstantFolding: true,
- }
-
- converted, err := evalengine.Translate(stmt, cfg)
- if err != nil {
- return
- }
-
- expected, evalErr := env.EvaluateAST(converted)
total++
-
- res, vmErr := env.Evaluate(converted)
- if vmErr != nil {
- switch {
- case evalErr == nil:
- t.Errorf("failed evaluation from compiler:\nSQL: %s\nError: %s", query, vmErr)
- case evalErr.Error() != vmErr.Error():
- t.Errorf("error mismatch:\nSQL: %s\nError eval: %s\nError comp: %s", query, evalErr, vmErr)
- default:
- supported++
- }
- return
+ testCompilerCase(t, query, venv, tc.Schema, env)
+ if !t.Failed() {
+ supported++
}
-
- eval := expected.String()
- comp := res.String()
-
- if eval != comp {
- t.Errorf("bad evaluation from compiler:\nSQL: %s\nEval: %s\nComp: %s", query, eval, comp)
- return
- }
-
- supported++
})
track.Add(tc.Name(), supported, total)
@@ -163,6 +135,51 @@ func TestCompilerReference(t *testing.T) {
t.Logf("\n%s", track.String())
}
+func testCompilerCase(t *testing.T, query string, venv *vtenv.Environment, schema []*querypb.Field, env *evalengine.ExpressionEnv) {
+ stmt, err := venv.Parser().ParseExpr(query)
+ if err != nil {
+ // no need to test un-parseable queries
+ return
+ }
+
+ fields := evalengine.FieldResolver(schema)
+ cfg := &evalengine.Config{
+ ResolveColumn: fields.Column,
+ ResolveType: fields.Type,
+ Collation: collations.CollationUtf8mb4ID,
+ Environment: venv,
+ NoConstantFolding: true,
+ }
+
+ converted, err := evalengine.Translate(stmt, cfg)
+ if err != nil {
+ return
+ }
+
+ var expected evalengine.EvalResult
+ var evalErr error
+ assert.NotPanics(t, func() {
+ expected, evalErr = env.EvaluateAST(converted)
+ })
+ var res evalengine.EvalResult
+ var vmErr error
+ assert.NotPanics(t, func() {
+ res, vmErr = env.Evaluate(converted)
+ })
+ switch {
+ case vmErr == nil && evalErr == nil:
+ eval := expected.String()
+ comp := res.String()
+ assert.Equalf(t, eval, comp, "bad evaluation from compiler:\nSQL: %s\nEval: %s\nComp: %s", query, eval, comp)
+ case vmErr == nil:
+ t.Errorf("failed evaluation from evalengine:\nSQL: %s\nError: %s", query, evalErr)
+ case evalErr == nil:
+ t.Errorf("failed evaluation from compiler:\nSQL: %s\nError: %s", query, vmErr)
+ case evalErr.Error() != vmErr.Error():
+ t.Errorf("error mismatch:\nSQL: %s\nError eval: %s\nError comp: %s", query, evalErr, vmErr)
+ }
+}
+
func TestCompilerSingle(t *testing.T) {
var testCases = []struct {
expression string
diff --git a/go/vt/vtgate/evalengine/eval_result.go b/go/vt/vtgate/evalengine/eval_result.go
index d9916af03be..5c1973d8eb1 100644
--- a/go/vt/vtgate/evalengine/eval_result.go
+++ b/go/vt/vtgate/evalengine/eval_result.go
@@ -62,6 +62,7 @@ func (er EvalResult) String() string {
// TupleValues allows for retrieval of the value we expose for public consumption
func (er EvalResult) TupleValues() []sqltypes.Value {
+ // TODO: Make this collation-aware
switch v := er.v.(type) {
case *evalTuple:
result := make([]sqltypes.Value, 0, len(v.t))
diff --git a/go/vt/vtgate/evalengine/expr.go b/go/vt/vtgate/evalengine/expr.go
index 44026f97e69..b90390e1ba8 100644
--- a/go/vt/vtgate/evalengine/expr.go
+++ b/go/vt/vtgate/evalengine/expr.go
@@ -56,7 +56,7 @@ func (expr *BinaryExpr) arguments(env *ExpressionEnv) (eval, eval, error) {
}
right, err := expr.Right.eval(env)
if err != nil {
- return nil, nil, err
+ return left, nil, err
}
return left, right, nil
}
diff --git a/go/vt/vtgate/evalengine/expr_compare.go b/go/vt/vtgate/evalengine/expr_compare.go
index ce1e16af787..6e6c888ecf6 100644
--- a/go/vt/vtgate/evalengine/expr_compare.go
+++ b/go/vt/vtgate/evalengine/expr_compare.go
@@ -592,13 +592,18 @@ func (l *LikeExpr) matchWildcard(left, right []byte, coll collations.ID) bool {
}
fullColl := colldata.Lookup(coll)
wc := fullColl.Wildcard(right, 0, 0, 0)
- return wc.Match(left)
+ return wc.Match(left) == !l.Negate
}
func (l *LikeExpr) eval(env *ExpressionEnv) (eval, error) {
- left, right, err := l.arguments(env)
- if left == nil || right == nil || err != nil {
- return nil, err
+ left, err := l.Left.eval(env)
+ if err != nil || left == nil {
+ return left, err
+ }
+
+ right, err := l.Right.eval(env)
+ if err != nil || right == nil {
+ return right, err
}
var col collations.TypedCollation
@@ -607,18 +612,9 @@ func (l *LikeExpr) eval(env *ExpressionEnv) (eval, error) {
return nil, err
}
- var matched bool
- switch {
- case typeIsTextual(left.SQLType()) && typeIsTextual(right.SQLType()):
- matched = l.matchWildcard(left.(*evalBytes).bytes, right.(*evalBytes).bytes, col.Collation)
- case typeIsTextual(right.SQLType()):
- matched = l.matchWildcard(left.ToRawBytes(), right.(*evalBytes).bytes, col.Collation)
- case typeIsTextual(left.SQLType()):
- matched = l.matchWildcard(left.(*evalBytes).bytes, right.ToRawBytes(), col.Collation)
- default:
- matched = l.matchWildcard(left.ToRawBytes(), right.ToRawBytes(), collations.CollationBinaryID)
- }
- return newEvalBool(matched == !l.Negate), nil
+ matched := l.matchWildcard(left.ToRawBytes(), right.ToRawBytes(), col.Collation)
+
+ return newEvalBool(matched), nil
}
func (expr *LikeExpr) compile(c *compiler) (ctype, error) {
@@ -627,12 +623,14 @@ func (expr *LikeExpr) compile(c *compiler) (ctype, error) {
return ctype{}, err
}
+ skip1 := c.compileNullCheck1(lt)
+
rt, err := expr.Right.compile(c)
if err != nil {
return ctype{}, err
}
- skip := c.compileNullCheck2(lt, rt)
+ skip2 := c.compileNullCheck1(rt)
if !lt.isTextual() {
c.asm.Convert_xc(2, sqltypes.VarChar, c.collation, nil)
@@ -684,6 +682,6 @@ func (expr *LikeExpr) compile(c *compiler) (ctype, error) {
})
}
- c.asm.jumpDestination(skip)
+ c.asm.jumpDestination(skip1, skip2)
return ctype{Type: sqltypes.Int64, Col: collationNumeric, Flag: flagIsBoolean | flagNullable}, nil
}
diff --git a/go/vt/vtgate/evalengine/fn_regexp.go b/go/vt/vtgate/evalengine/fn_regexp.go
index a94b9a83aee..10aa3f6b6c6 100644
--- a/go/vt/vtgate/evalengine/fn_regexp.go
+++ b/go/vt/vtgate/evalengine/fn_regexp.go
@@ -167,43 +167,43 @@ func compileRegex(pat eval, c colldata.Charset, flags icuregex.RegexpFlag) (*icu
var compileErr *icuregex.CompileError
if errors.Is(err, icuerrors.ErrUnsupported) {
- err = vterrors.NewErrorf(vtrpcpb.Code_UNIMPLEMENTED, vterrors.RegexpUnimplemented, err.Error())
+ err = vterrors.NewError(vtrpcpb.Code_UNIMPLEMENTED, vterrors.RegexpUnimplemented, err.Error())
} else if errors.Is(err, icuerrors.ErrIllegalArgument) {
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpIllegalArgument, err.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpIllegalArgument, err.Error())
} else if errors.As(err, &compileErr) {
switch compileErr.Code {
case icuregex.InternalError:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInternal, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInternal, compileErr.Error())
case icuregex.RuleSyntax:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpRuleSyntax, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpRuleSyntax, compileErr.Error())
case icuregex.BadEscapeSequence:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpBadEscapeSequence, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpBadEscapeSequence, compileErr.Error())
case icuregex.PropertySyntax:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpRuleSyntax, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpRuleSyntax, compileErr.Error())
case icuregex.Unimplemented:
- err = vterrors.NewErrorf(vtrpcpb.Code_UNIMPLEMENTED, vterrors.RegexpUnimplemented, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_UNIMPLEMENTED, vterrors.RegexpUnimplemented, compileErr.Error())
case icuregex.MismatchedParen:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMismatchParen, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMismatchParen, compileErr.Error())
case icuregex.BadInterval:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpBadInterval, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpBadInterval, compileErr.Error())
case icuregex.MaxLtMin:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMaxLtMin, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMaxLtMin, compileErr.Error())
case icuregex.InvalidBackRef:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidBackRef, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidBackRef, compileErr.Error())
case icuregex.InvalidFlag:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidFlag, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidFlag, compileErr.Error())
case icuregex.LookBehindLimit:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpLookBehindLimit, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpLookBehindLimit, compileErr.Error())
case icuregex.MissingCloseBracket:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMissingCloseBracket, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpMissingCloseBracket, compileErr.Error())
case icuregex.InvalidRange:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidRange, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidRange, compileErr.Error())
case icuregex.PatternTooBig:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpPatternTooBig, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpPatternTooBig, compileErr.Error())
case icuregex.InvalidCaptureGroupName:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidCaptureGroup, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInvalidCaptureGroup, compileErr.Error())
default:
- err = vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInternal, compileErr.Error())
+ err = vterrors.NewError(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.RegexpInternal, compileErr.Error())
}
}
diff --git a/go/vt/vtgate/evalengine/fn_string.go b/go/vt/vtgate/evalengine/fn_string.go
index 6d83d36412d..1cca7a94c8d 100644
--- a/go/vt/vtgate/evalengine/fn_string.go
+++ b/go/vt/vtgate/evalengine/fn_string.go
@@ -1685,24 +1685,17 @@ func (call *builtinLocate) compile(c *compiler) (ctype, error) {
return ctype{}, err
}
+ skip1 := c.compileNullCheck1(substr)
str, err := call.Arguments[1].compile(c)
if err != nil {
return ctype{}, err
}
- skip1 := c.compileNullCheck2(substr, str)
- var skip2 *jump
- if len(call.Arguments) > 2 {
- l, err := call.Arguments[2].compile(c)
- if err != nil {
- return ctype{}, err
- }
- skip2 = c.compileNullCheck2(str, l)
- _ = c.compileToInt64(l, 1)
- }
+ skip2 := c.compileNullCheck1(str)
+ var skip3 *jump
if !str.isTextual() {
- c.asm.Convert_xce(len(call.Arguments)-1, sqltypes.VarChar, c.collation)
+ c.asm.Convert_xce(1, sqltypes.VarChar, c.collation)
str.Col = collations.TypedCollation{
Collation: c.collation,
Coercibility: collations.CoerceCoercible,
@@ -1713,7 +1706,7 @@ func (call *builtinLocate) compile(c *compiler) (ctype, error) {
fromCharset := colldata.Lookup(substr.Col.Collation).Charset()
toCharset := colldata.Lookup(str.Col.Collation).Charset()
if !substr.isTextual() || (fromCharset != toCharset && !toCharset.IsSuperset(fromCharset)) {
- c.asm.Convert_xce(len(call.Arguments), sqltypes.VarChar, str.Col.Collation)
+ c.asm.Convert_xce(2, sqltypes.VarChar, str.Col.Collation)
substr.Col = collations.TypedCollation{
Collation: str.Col.Collation,
Coercibility: collations.CoerceCoercible,
@@ -1721,6 +1714,15 @@ func (call *builtinLocate) compile(c *compiler) (ctype, error) {
}
}
+ if len(call.Arguments) > 2 {
+ l, err := call.Arguments[2].compile(c)
+ if err != nil {
+ return ctype{}, err
+ }
+ skip3 = c.compileNullCheck1(l)
+ _ = c.compileToInt64(l, 1)
+ }
+
var coll colldata.Collation
if typeIsTextual(substr.Type) && typeIsTextual(str.Type) {
coll = colldata.Lookup(str.Col.Collation)
@@ -1734,7 +1736,7 @@ func (call *builtinLocate) compile(c *compiler) (ctype, error) {
c.asm.Locate2(coll)
}
- c.asm.jumpDestination(skip1, skip2)
+ c.asm.jumpDestination(skip1, skip2, skip3)
return ctype{Type: sqltypes.Int64, Col: collationNumeric, Flag: flagNullable}, nil
}
diff --git a/go/vt/vtgate/evalengine/integration/fuzz_test.go b/go/vt/vtgate/evalengine/integration/fuzz_test.go
index 17a721edde9..7372d6fd731 100644
--- a/go/vt/vtgate/evalengine/integration/fuzz_test.go
+++ b/go/vt/vtgate/evalengine/integration/fuzz_test.go
@@ -20,6 +20,7 @@ package integration
import (
"encoding/json"
+ "errors"
"fmt"
"math/rand/v2"
"os"
@@ -205,11 +206,11 @@ func TestGenerateFuzzCases(t *testing.T) {
remote, remoteErr := conn.ExecuteFetch(query, 1, false)
if localErr != nil && strings.Contains(localErr.Error(), "syntax error at position") {
- localErr = fmt.Errorf(localSyntaxErr)
+ localErr = errors.New(localSyntaxErr)
}
if remoteErr != nil && strings.Contains(remoteErr.Error(), "You have an error in your SQL syntax") {
- remoteErr = fmt.Errorf(syntaxErr)
+ remoteErr = errors.New(syntaxErr)
}
res := mismatch{
diff --git a/go/vt/vtgate/evalengine/testcases/cases.go b/go/vt/vtgate/evalengine/testcases/cases.go
index ed1c5ed1f76..ff6c0c0f311 100644
--- a/go/vt/vtgate/evalengine/testcases/cases.go
+++ b/go/vt/vtgate/evalengine/testcases/cases.go
@@ -1097,24 +1097,26 @@ func CollationOperations(yield Query) {
}
func LikeComparison(yield Query) {
- var left = []string{
+ var left = append(inputConversions,
`'foobar'`, `'FOOBAR'`,
`'1234'`, `1234`,
`_utf8mb4 'foobar' COLLATE utf8mb4_0900_as_cs`,
- `_utf8mb4 'FOOBAR' COLLATE utf8mb4_0900_as_cs`,
- }
- var right = append([]string{
+ `_utf8mb4 'FOOBAR' COLLATE utf8mb4_0900_as_cs`)
+
+ var right = append(left,
+ `NULL`, `1`, `0`,
`'foo%'`, `'FOO%'`, `'foo_ar'`, `'FOO_AR'`,
`'12%'`, `'12_4'`,
`_utf8mb4 'foo%' COLLATE utf8mb4_0900_as_cs`,
`_utf8mb4 'FOO%' COLLATE utf8mb4_0900_as_cs`,
`_utf8mb4 'foo_ar' COLLATE utf8mb4_0900_as_cs`,
- `_utf8mb4 'FOO_AR' COLLATE utf8mb4_0900_as_cs`,
- }, left...)
+ `_utf8mb4 'FOO_AR' COLLATE utf8mb4_0900_as_cs`)
for _, lhs := range left {
for _, rhs := range right {
- yield(fmt.Sprintf("%s LIKE %s", lhs, rhs), nil)
+ for _, op := range []string{"LIKE", "NOT LIKE"} {
+ yield(fmt.Sprintf("%s %s %s", lhs, op, rhs), nil)
+ }
}
}
}
diff --git a/go/vt/vtgate/evalengine/translate.go b/go/vt/vtgate/evalengine/translate.go
index 0091f06a633..e93d338952c 100644
--- a/go/vt/vtgate/evalengine/translate.go
+++ b/go/vt/vtgate/evalengine/translate.go
@@ -87,7 +87,7 @@ func (ast *astCompiler) translateComparisonExpr2(op sqlparser.ComparisonExprOper
Negate: op == sqlparser.NotRegexpOp,
}, nil
default:
- return nil, vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, op.ToString())
+ return nil, vterrors.New(vtrpcpb.Code_UNIMPLEMENTED, op.ToString())
}
}
@@ -309,10 +309,6 @@ func (ast *astCompiler) translateBinaryExpr(binary *sqlparser.BinaryExpr) (IR, e
return &BitwiseExpr{BinaryExpr: binaryExpr, Op: &opBitShl{}}, nil
case sqlparser.ShiftRightOp:
return &BitwiseExpr{BinaryExpr: binaryExpr, Op: &opBitShr{}}, nil
- case sqlparser.JSONExtractOp:
- return builtinJSONExtractRewrite(left, right)
- case sqlparser.JSONUnquoteExtractOp:
- return builtinJSONExtractUnquoteRewrite(left, right)
default:
return nil, translateExprNotSupported(binary)
}
diff --git a/go/vt/vtgate/evalengine/translate_test.go b/go/vt/vtgate/evalengine/translate_test.go
index 3702230e22e..2bebae548e1 100644
--- a/go/vt/vtgate/evalengine/translate_test.go
+++ b/go/vt/vtgate/evalengine/translate_test.go
@@ -89,8 +89,8 @@ func TestTranslateSimplification(t *testing.T) {
{"coalesce(NULL, 2, NULL, 4)", ok("coalesce(null, 2, null, 4)"), ok("2")},
{"coalesce(NULL, NULL)", ok("coalesce(null, null)"), ok("null")},
{"coalesce(NULL)", ok("coalesce(null)"), ok("null")},
- {"weight_string('foobar')", ok(`weight_string('foobar')`), ok("'\x1c\xe5\x1d\xdd\x1d\xdd\x1c`\x1cG\x1e3'")},
- {"weight_string('foobar' as char(12))", ok(`weight_string('foobar' as char(12))`), ok("'\x1c\xe5\x1d\xdd\x1d\xdd\x1c`\x1cG\x1e3'")},
+ {"weight_string('foobar')", ok(`weight_string('foobar')`), ok("_binary'\x1c\xe5\x1d\xdd\x1d\xdd\x1c`\x1cG\x1e3'")},
+ {"weight_string('foobar' as char(12))", ok(`weight_string('foobar' as char(12))`), ok("_binary'\x1c\xe5\x1d\xdd\x1d\xdd\x1c`\x1cG\x1e3'")},
{"case when 1 = 1 then 2 else 3 end", ok("case when 1 = 1 then 2 else 3"), ok("2")},
{"case when null then 2 when 12 = 4 then 'ohnoes' else 42 end", ok(`case when null then 2 when 12 = 4 then 'ohnoes' else 42`), ok(`'42'`)},
{"convert('a', char(2) character set utf8mb4)", ok(`convert('a', CHAR(2) character set utf8mb4_0900_ai_ci)`), ok(`'a'`)},
diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go
index bc492dd0335..0bb47361f55 100644
--- a/go/vt/vtgate/executor.go
+++ b/go/vt/vtgate/executor.go
@@ -30,6 +30,8 @@ import (
"github.com/spf13/pflag"
+ vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/cache/theine"
"vitess.io/vitess/go/mysql/capabilities"
@@ -57,6 +59,7 @@ import (
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/evalengine"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vtgate/logstats"
"vitess.io/vitess/go/vt/vtgate/planbuilder"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
@@ -67,7 +70,6 @@ import (
)
var (
- errNoKeyspace = vterrors.VT09005()
defaultTabletType = topodatapb.TabletType_PRIMARY
// TODO: @rafael - These two counters should be deprecated in favor of the ByTable ones in v17+. They are kept for now for backwards compatibility.
@@ -77,6 +79,11 @@ var (
queriesProcessedByTable = stats.NewCountersWithMultiLabels("QueriesProcessedByTable", "Queries processed at vtgate by plan type, keyspace and table", []string{"Plan", "Keyspace", "Table"})
queriesRoutedByTable = stats.NewCountersWithMultiLabels("QueriesRoutedByTable", "Queries routed from vtgate to vttablet by plan type, keyspace and table", []string{"Plan", "Keyspace", "Table"})
+ // commitMode records the timing of the commit phase of a transaction.
+ // It also tracks between different transaction mode i.e. Single, Multi and TwoPC
+ commitMode = stats.NewTimings("CommitModeTimings", "Commit Mode Time", "mode")
+ commitUnresolved = stats.NewCounter("CommitUnresolved", "Atomic Commit failed to conclude after commit decision is made")
+
exceedMemoryRowsLogger = logutil.NewThrottledLogger("ExceedMemoryRows", 1*time.Minute)
errorTransform errorTransformer = nullErrorTransformer{}
@@ -106,7 +113,6 @@ type Executor struct {
resolver *Resolver
scatterConn *ScatterConn
txConn *TxConn
- pv plancontext.PlannerVersion
mu sync.Mutex
vschema *vindexes.VSchema
@@ -116,8 +122,7 @@ type Executor struct {
plans *PlanCache
epoch atomic.Uint32
- normalize bool
- warnShardedOnly bool
+ normalize bool
vm *VSchemaManager
schemaTracker SchemaInfo
@@ -130,6 +135,8 @@ type Executor struct {
warmingReadsPercent int
warmingReadsChannel chan bool
+
+ vConfig econtext.VCursorConfig
}
var executorOnce sync.Once
@@ -170,17 +177,16 @@ func NewExecutor(
scatterConn: resolver.scatterConn,
txConn: resolver.scatterConn.txConn,
normalize: normalize,
- warnShardedOnly: warnOnShardedOnly,
streamSize: streamSize,
schemaTracker: schemaTracker,
allowScatter: !noScatter,
- pv: pv,
plans: plans,
warmingReadsPercent: warmingReadsPercent,
warmingReadsChannel: make(chan bool, warmingReadsConcurrency),
}
+ // setting the vcursor config.
+ e.initVConfig(warnOnShardedOnly, pv)
- vschemaacl.Init()
// we subscribe to update from the VSchemaManager
e.vm = &VSchemaManager{
subscriber: e.SaveVSchema,
@@ -208,7 +214,7 @@ func NewExecutor(
return e.plans.Metrics.Hits()
})
stats.NewCounterFunc("QueryPlanCacheMisses", "Query plan cache misses", func() int64 {
- return e.plans.Metrics.Hits()
+ return e.plans.Metrics.Misses()
})
servenv.HTTPHandle(pathQueryPlans, e)
servenv.HTTPHandle(pathScatterStats, e)
@@ -218,7 +224,7 @@ func NewExecutor(
}
// Execute executes a non-streaming query.
-func (e *Executor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, safeSession *SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (result *sqltypes.Result, err error) {
+func (e *Executor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (result *sqltypes.Result, err error) {
span, ctx := trace.NewSpan(ctx, "executor.Execute")
span.Annotate("method", method)
trace.AnnotateSQL(span, sqlparser.Preview(sql))
@@ -281,7 +287,7 @@ func (e *Executor) StreamExecute(
ctx context.Context,
mysqlCtx vtgateservice.MySQLConnection,
method string,
- safeSession *SafeSession,
+ safeSession *econtext.SafeSession,
sql string,
bindVars map[string]*querypb.BindVariable,
callback func(*sqltypes.Result) error,
@@ -295,7 +301,7 @@ func (e *Executor) StreamExecute(
srr := &streaminResultReceiver{callback: callback}
var err error
- resultHandler := func(ctx context.Context, plan *engine.Plan, vc *vcursorImpl, bindVars map[string]*querypb.BindVariable, execStart time.Time) error {
+ resultHandler := func(ctx context.Context, plan *engine.Plan, vc *econtext.VCursorImpl, bindVars map[string]*querypb.BindVariable, execStart time.Time) error {
var seenResults atomic.Bool
var resultMu sync.Mutex
result := &sqltypes.Result{}
@@ -363,7 +369,7 @@ func (e *Executor) StreamExecute(
logStats.TablesUsed = plan.TablesUsed
logStats.TabletType = vc.TabletType().String()
logStats.ExecuteTime = time.Since(execStart)
- logStats.ActiveKeyspace = vc.keyspace
+ logStats.ActiveKeyspace = vc.GetKeyspace()
e.updateQueryCounts(plan.Instructions.RouteType(), plan.Instructions.GetKeyspaceName(), plan.Instructions.GetTableName(), int64(logStats.ShardQueries))
@@ -406,12 +412,12 @@ func canReturnRows(stmtType sqlparser.StatementType) bool {
}
}
-func saveSessionStats(safeSession *SafeSession, stmtType sqlparser.StatementType, rowsAffected, insertID uint64, rowsReturned int, err error) {
+func saveSessionStats(safeSession *econtext.SafeSession, stmtType sqlparser.StatementType, rowsAffected, insertID uint64, rowsReturned int, err error) {
safeSession.RowCount = -1
if err != nil {
return
}
- if !safeSession.foundRowsHandled {
+ if !safeSession.IsFoundRowsHandled() {
safeSession.FoundRows = uint64(rowsReturned)
}
if insertID > 0 {
@@ -425,11 +431,11 @@ func saveSessionStats(safeSession *SafeSession, stmtType sqlparser.StatementType
}
}
-func (e *Executor) execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, safeSession *SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) (sqlparser.StatementType, *sqltypes.Result, error) {
+func (e *Executor) execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) (sqlparser.StatementType, *sqltypes.Result, error) {
var err error
var qr *sqltypes.Result
var stmtType sqlparser.StatementType
- err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, logStats, func(ctx context.Context, plan *engine.Plan, vc *vcursorImpl, bindVars map[string]*querypb.BindVariable, time time.Time) error {
+ err = e.newExecute(ctx, mysqlCtx, safeSession, sql, bindVars, logStats, func(ctx context.Context, plan *engine.Plan, vc *econtext.VCursorImpl, bindVars map[string]*querypb.BindVariable, time time.Time) error {
stmtType = plan.Type
qr, err = e.executePlan(ctx, safeSession, plan, vc, bindVars, logStats, time)
return err
@@ -443,7 +449,7 @@ func (e *Executor) execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConn
}
// addNeededBindVars adds bind vars that are needed by the plan
-func (e *Executor) addNeededBindVars(vcursor *vcursorImpl, bindVarNeeds *sqlparser.BindVarNeeds, bindVars map[string]*querypb.BindVariable, session *SafeSession) error {
+func (e *Executor) addNeededBindVars(vcursor *econtext.VCursorImpl, bindVarNeeds *sqlparser.BindVarNeeds, bindVars map[string]*querypb.BindVariable, session *econtext.SafeSession) error {
for _, funcName := range bindVarNeeds.NeedFunctionResult {
switch funcName {
case sqlparser.DBVarName:
@@ -536,7 +542,7 @@ func (e *Executor) addNeededBindVars(vcursor *vcursorImpl, bindVarNeeds *sqlpars
}
evalExpr, err := evalengine.Translate(expr, &evalengine.Config{
- Collation: vcursor.collation,
+ Collation: vcursor.ConnCollation(),
Environment: e.env,
SQLMode: evalengine.ParseSQLMode(vcursor.SQLMode()),
})
@@ -547,7 +553,7 @@ func (e *Executor) addNeededBindVars(vcursor *vcursorImpl, bindVarNeeds *sqlpars
if err != nil {
return err
}
- bindVars[key] = sqltypes.ValueBindVariable(evaluated.Value(vcursor.collation))
+ bindVars[key] = sqltypes.ValueBindVariable(evaluated.Value(vcursor.ConnCollation()))
}
}
}
@@ -567,21 +573,21 @@ func (e *Executor) addNeededBindVars(vcursor *vcursorImpl, bindVarNeeds *sqlpars
return nil
}
-func ifOptionsExist(session *SafeSession, f func(*querypb.ExecuteOptions)) {
+func ifOptionsExist(session *econtext.SafeSession, f func(*querypb.ExecuteOptions)) {
options := session.GetOptions()
if options != nil {
f(options)
}
}
-func ifReadAfterWriteExist(session *SafeSession, f func(*vtgatepb.ReadAfterWrite)) {
+func ifReadAfterWriteExist(session *econtext.SafeSession, f func(*vtgatepb.ReadAfterWrite)) {
raw := session.ReadAfterWrite
if raw != nil {
f(raw)
}
}
-func (e *Executor) handleBegin(ctx context.Context, safeSession *SafeSession, logStats *logstats.LogStats, stmt sqlparser.Statement) (*sqltypes.Result, error) {
+func (e *Executor) handleBegin(ctx context.Context, safeSession *econtext.SafeSession, logStats *logstats.LogStats, stmt sqlparser.Statement) (*sqltypes.Result, error) {
execStart := time.Now()
logStats.PlanTime = execStart.Sub(logStats.StartTime)
@@ -594,7 +600,7 @@ func (e *Executor) handleBegin(ctx context.Context, safeSession *SafeSession, lo
return &sqltypes.Result{}, err
}
-func (e *Executor) handleCommit(ctx context.Context, safeSession *SafeSession, logStats *logstats.LogStats) (*sqltypes.Result, error) {
+func (e *Executor) handleCommit(ctx context.Context, safeSession *econtext.SafeSession, logStats *logstats.LogStats) (*sqltypes.Result, error) {
execStart := time.Now()
logStats.PlanTime = execStart.Sub(logStats.StartTime)
logStats.ShardQueries = uint64(len(safeSession.ShardSessions))
@@ -606,11 +612,11 @@ func (e *Executor) handleCommit(ctx context.Context, safeSession *SafeSession, l
}
// Commit commits the existing transactions
-func (e *Executor) Commit(ctx context.Context, safeSession *SafeSession) error {
+func (e *Executor) Commit(ctx context.Context, safeSession *econtext.SafeSession) error {
return e.txConn.Commit(ctx, safeSession)
}
-func (e *Executor) handleRollback(ctx context.Context, safeSession *SafeSession, logStats *logstats.LogStats) (*sqltypes.Result, error) {
+func (e *Executor) handleRollback(ctx context.Context, safeSession *econtext.SafeSession, logStats *logstats.LogStats) (*sqltypes.Result, error) {
execStart := time.Now()
logStats.PlanTime = execStart.Sub(logStats.StartTime)
logStats.ShardQueries = uint64(len(safeSession.ShardSessions))
@@ -620,7 +626,7 @@ func (e *Executor) handleRollback(ctx context.Context, safeSession *SafeSession,
return &sqltypes.Result{}, err
}
-func (e *Executor) handleSavepoint(ctx context.Context, safeSession *SafeSession, sql string, planType string, logStats *logstats.LogStats, nonTxResponse func(query string) (*sqltypes.Result, error), ignoreMaxMemoryRows bool) (*sqltypes.Result, error) {
+func (e *Executor) handleSavepoint(ctx context.Context, safeSession *econtext.SafeSession, sql string, planType string, logStats *logstats.LogStats, nonTxResponse func(query string) (*sqltypes.Result, error), ignoreMaxMemoryRows bool) (*sqltypes.Result, error) {
execStart := time.Now()
logStats.PlanTime = execStart.Sub(logStats.StartTime)
logStats.ShardQueries = uint64(len(safeSession.ShardSessions))
@@ -632,7 +638,7 @@ func (e *Executor) handleSavepoint(ctx context.Context, safeSession *SafeSession
// If no transaction exists on any of the shard sessions,
// then savepoint does not need to be executed, it will be only stored in the session
// and later will be executed when a transaction is started.
- if !safeSession.isTxOpen() {
+ if !safeSession.IsTxOpen() {
if safeSession.InTransaction() {
// Storing, as this needs to be executed just after starting transaction on the shard.
safeSession.StoreSavepoint(sql)
@@ -640,7 +646,7 @@ func (e *Executor) handleSavepoint(ctx context.Context, safeSession *SafeSession
}
return nonTxResponse(sql)
}
- orig := safeSession.commitOrder
+ orig := safeSession.GetCommitOrder()
qr, err := e.executeSPInAllSessions(ctx, safeSession, sql, ignoreMaxMemoryRows)
safeSession.SetCommitOrder(orig)
if err != nil {
@@ -652,7 +658,7 @@ func (e *Executor) handleSavepoint(ctx context.Context, safeSession *SafeSession
// executeSPInAllSessions function executes the savepoint query in all open shard sessions (pre, normal and post)
// which has non-zero transaction id (i.e. an open transaction on the shard connection).
-func (e *Executor) executeSPInAllSessions(ctx context.Context, safeSession *SafeSession, sql string, ignoreMaxMemoryRows bool) (*sqltypes.Result, error) {
+func (e *Executor) executeSPInAllSessions(ctx context.Context, safeSession *econtext.SafeSession, sql string, ignoreMaxMemoryRows bool) (*sqltypes.Result, error) {
var qr *sqltypes.Result
var errs []error
for _, co := range []vtgatepb.CommitOrder{vtgatepb.CommitOrder_PRE, vtgatepb.CommitOrder_NORMAL, vtgatepb.CommitOrder_POST} {
@@ -660,7 +666,7 @@ func (e *Executor) executeSPInAllSessions(ctx context.Context, safeSession *Safe
var rss []*srvtopo.ResolvedShard
var queries []*querypb.BoundQuery
- for _, shardSession := range safeSession.getSessions() {
+ for _, shardSession := range safeSession.GetSessions() {
// This will avoid executing savepoint on reserved connections
// which has no open transaction.
if shardSession.TransactionId == 0 {
@@ -672,7 +678,7 @@ func (e *Executor) executeSPInAllSessions(ctx context.Context, safeSession *Safe
})
queries = append(queries, &querypb.BoundQuery{Sql: sql})
}
- qr, errs = e.ExecuteMultiShard(ctx, nil, rss, queries, safeSession, false /*autocommit*/, ignoreMaxMemoryRows)
+ qr, errs = e.ExecuteMultiShard(ctx, nil, rss, queries, safeSession, false /*autocommit*/, ignoreMaxMemoryRows, nullResultsObserver{})
err := vterrors.Aggregate(errs)
if err != nil {
return nil, err
@@ -713,11 +719,11 @@ func (e *Executor) handleKill(ctx context.Context, mysqlCtx vtgateservice.MySQLC
// CloseSession releases the current connection, which rollbacks open transactions and closes reserved connections.
// It is called then the MySQL servers closes the connection to its client.
-func (e *Executor) CloseSession(ctx context.Context, safeSession *SafeSession) error {
+func (e *Executor) CloseSession(ctx context.Context, safeSession *econtext.SafeSession) error {
return e.txConn.ReleaseAll(ctx, safeSession)
}
-func (e *Executor) setVitessMetadata(ctx context.Context, name, value string) error {
+func (e *Executor) SetVitessMetadata(ctx context.Context, name, value string) error {
// TODO(kalfonso): move to its own acl check and consolidate into an acl component that can handle multiple operations (vschema, metadata)
user := callerid.ImmediateCallerIDFromContext(ctx)
allowed := vschemaacl.Authorized(user)
@@ -736,7 +742,7 @@ func (e *Executor) setVitessMetadata(ctx context.Context, name, value string) er
return ts.UpsertMetadata(ctx, name, value)
}
-func (e *Executor) showVitessMetadata(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+func (e *Executor) ShowVitessMetadata(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
ts, err := e.serv.GetTopoServer()
if err != nil {
return nil, err
@@ -769,7 +775,7 @@ func (e *Executor) showVitessMetadata(ctx context.Context, filter *sqlparser.Sho
type tabletFilter func(tablet *topodatapb.Tablet, servingState string, primaryTermStartTime int64) bool
-func (e *Executor) showShards(ctx context.Context, filter *sqlparser.ShowFilter, destTabletType topodatapb.TabletType) (*sqltypes.Result, error) {
+func (e *Executor) ShowShards(ctx context.Context, filter *sqlparser.ShowFilter, destTabletType topodatapb.TabletType) (*sqltypes.Result, error) {
showVitessShardsFilters := func(filter *sqlparser.ShowFilter) ([]func(string) bool, []func(string, *topodatapb.ShardReference) bool) {
keyspaceFilters := []func(string) bool{}
shardFilters := []func(string, *topodatapb.ShardReference) bool{}
@@ -853,7 +859,7 @@ func (e *Executor) showShards(ctx context.Context, filter *sqlparser.ShowFilter,
}, nil
}
-func (e *Executor) showTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+func (e *Executor) ShowTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
getTabletFilters := func(filter *sqlparser.ShowFilter) []tabletFilter {
var filters []tabletFilter
@@ -926,7 +932,7 @@ func (e *Executor) showTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result,
}, nil
}
-func (e *Executor) showVitessReplicationStatus(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+func (e *Executor) ShowVitessReplicationStatus(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
ctx, cancel := context.WithTimeout(ctx, healthCheckTimeout)
defer cancel()
rows := [][]sqltypes.Value{}
@@ -1073,26 +1079,14 @@ func (e *Executor) SaveVSchema(vschema *vindexes.VSchema, stats *VSchemaStats) {
// ParseDestinationTarget parses destination target string and sets default keyspace if possible.
func (e *Executor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) {
- destKeyspace, destTabletType, dest, err := topoproto.ParseDestination(targetString, defaultTabletType)
- // Set default keyspace
- if destKeyspace == "" && len(e.VSchema().Keyspaces) == 1 {
- for k := range e.VSchema().Keyspaces {
- destKeyspace = k
- }
- }
- return destKeyspace, destTabletType, dest, err
-}
-
-type iQueryOption interface {
- cachePlan() bool
- getSelectLimit() int
+ return econtext.ParseDestinationTarget(targetString, defaultTabletType, e.VSchema())
}
// getPlan computes the plan for the given query. If one is in
// the cache, it reuses it.
func (e *Executor) getPlan(
ctx context.Context,
- vcursor *vcursorImpl,
+ vcursor *econtext.VCursorImpl,
query string,
stmt sqlparser.Statement,
comments sqlparser.MarginComments,
@@ -1105,15 +1099,16 @@ func (e *Executor) getPlan(
return nil, vterrors.VT13001("vschema not initialized")
}
- vcursor.SetIgnoreMaxMemoryRows(sqlparser.IgnoreMaxMaxMemoryRowsDirective(stmt))
- vcursor.SetConsolidator(sqlparser.Consolidator(stmt))
- vcursor.SetWorkloadName(sqlparser.GetWorkloadNameFromStatement(stmt))
- vcursor.UpdateForeignKeyChecksState(sqlparser.ForeignKeyChecksState(stmt))
- priority, err := sqlparser.GetPriorityFromStatement(stmt)
+ qh, err := sqlparser.BuildQueryHints(stmt)
if err != nil {
return nil, err
}
- vcursor.SetPriority(priority)
+ vcursor.SetIgnoreMaxMemoryRows(qh.IgnoreMaxMemoryRows)
+ vcursor.SetConsolidator(qh.Consolidator)
+ vcursor.SetWorkloadName(qh.Workload)
+ vcursor.UpdateForeignKeyChecksState(qh.ForeignKeyChecks)
+ vcursor.SetPriority(qh.Priority)
+ vcursor.SetExecQueryTimeout(qh.Timeout)
setVarComment, err := prepareSetVarComment(vcursor, stmt)
if err != nil {
@@ -1129,10 +1124,10 @@ func (e *Executor) getPlan(
reservedVars,
bindVars,
parameterize,
- vcursor.keyspace,
- vcursor.safeSession.getSelectLimit(),
+ vcursor.GetKeyspace(),
+ vcursor.SafeSession.GetSelectLimit(),
setVarComment,
- vcursor.safeSession.SystemVariables,
+ vcursor.GetSystemVariablesCopy(),
vcursor.GetForeignKeyChecksState(),
vcursor,
)
@@ -1151,9 +1146,9 @@ func (e *Executor) getPlan(
return e.cacheAndBuildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds, logStats)
}
-func (e *Executor) hashPlan(ctx context.Context, vcursor *vcursorImpl, query string) PlanCacheKey {
+func (e *Executor) hashPlan(ctx context.Context, vcursor *econtext.VCursorImpl, query string) PlanCacheKey {
hasher := vthash.New256()
- vcursor.keyForPlan(ctx, query, hasher)
+ vcursor.KeyForPlan(ctx, query, hasher)
var planKey PlanCacheKey
hasher.Sum(planKey[:0])
@@ -1162,19 +1157,22 @@ func (e *Executor) hashPlan(ctx context.Context, vcursor *vcursorImpl, query str
func (e *Executor) buildStatement(
ctx context.Context,
- vcursor *vcursorImpl,
+ vcursor *econtext.VCursorImpl,
query string,
stmt sqlparser.Statement,
reservedVars *sqlparser.ReservedVars,
bindVarNeeds *sqlparser.BindVarNeeds,
) (*engine.Plan, error) {
- plan, err := planbuilder.BuildFromStmt(ctx, query, stmt, reservedVars, vcursor, bindVarNeeds, enableOnlineDDL, enableDirectDDL)
+ cfg := &dynamicViperConfig{
+ onlineDDL: enableOnlineDDL,
+ directDDL: enableDirectDDL,
+ }
+ plan, err := planbuilder.BuildFromStmt(ctx, query, stmt, reservedVars, vcursor, bindVarNeeds, cfg)
if err != nil {
return nil, err
}
- plan.Warnings = vcursor.warnings
- vcursor.warnings = nil
+ plan.Warnings = vcursor.GetAndEmptyWarnings()
err = e.checkThatPlanIsValid(stmt, plan)
return plan, err
@@ -1182,14 +1180,14 @@ func (e *Executor) buildStatement(
func (e *Executor) cacheAndBuildStatement(
ctx context.Context,
- vcursor *vcursorImpl,
+ vcursor *econtext.VCursorImpl,
query string,
stmt sqlparser.Statement,
reservedVars *sqlparser.ReservedVars,
bindVarNeeds *sqlparser.BindVarNeeds,
logStats *logstats.LogStats,
) (*engine.Plan, error) {
- planCachable := sqlparser.CachePlan(stmt) && vcursor.safeSession.cachePlan()
+ planCachable := sqlparser.CachePlan(stmt) && vcursor.CachePlan()
if planCachable {
planKey := e.hashPlan(ctx, vcursor, query)
@@ -1207,7 +1205,7 @@ func (e *Executor) canNormalizeStatement(stmt sqlparser.Statement, setVarComment
return sqlparser.CanNormalize(stmt) || setVarComment != ""
}
-func prepareSetVarComment(vcursor *vcursorImpl, stmt sqlparser.Statement) (string, error) {
+func prepareSetVarComment(vcursor *econtext.VCursorImpl, stmt sqlparser.Statement) (string, error) {
if vcursor == nil || vcursor.Session().InReservedConn() {
return "", nil
}
@@ -1348,7 +1346,7 @@ func isValidPayloadSize(query string) bool {
}
// Prepare executes a prepare statements.
-func (e *Executor) Prepare(ctx context.Context, method string, safeSession *SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (fld []*querypb.Field, err error) {
+func (e *Executor) Prepare(ctx context.Context, method string, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable) (fld []*querypb.Field, err error) {
logStats := logstats.NewLogStats(ctx, method, sql, safeSession.GetSessionUUID(), bindVars)
fld, err = e.prepare(ctx, safeSession, sql, bindVars, logStats)
logStats.Error = err
@@ -1367,7 +1365,7 @@ func (e *Executor) Prepare(ctx context.Context, method string, safeSession *Safe
return fld, err
}
-func (e *Executor) prepare(ctx context.Context, safeSession *SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) {
+func (e *Executor) prepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) {
// Start an implicit transaction if necessary.
if !safeSession.Autocommit && !safeSession.InTransaction() {
if err := e.txConn.Begin(ctx, safeSession, nil); err != nil {
@@ -1403,9 +1401,41 @@ func (e *Executor) prepare(ctx context.Context, safeSession *SafeSession, sql st
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unrecognized prepare statement: %s", sql)
}
-func (e *Executor) handlePrepare(ctx context.Context, safeSession *SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) {
+func (e *Executor) initVConfig(warnOnShardedOnly bool, pv plancontext.PlannerVersion) {
+ connCollation := collations.Unknown
+ if gw, isTabletGw := e.resolver.resolver.GetGateway().(*TabletGateway); isTabletGw {
+ connCollation = gw.DefaultConnCollation()
+ }
+ if connCollation == collations.Unknown {
+ connCollation = e.env.CollationEnv().DefaultConnectionCharset()
+ }
+
+ e.vConfig = econtext.VCursorConfig{
+ Collation: connCollation,
+ DefaultTabletType: defaultTabletType,
+ PlannerVersion: pv,
+
+ QueryTimeout: queryTimeout,
+ MaxMemoryRows: maxMemoryRows,
+
+ SetVarEnabled: sysVarSetEnabled,
+ EnableViews: enableViews,
+ ForeignKeyMode: fkMode(foreignKeyMode),
+ EnableShardRouting: enableShardRouting,
+ WarnShardedOnly: warnOnShardedOnly,
+
+ DBDDLPlugin: dbDDLPlugin,
+
+ WarmingReadsPercent: e.warmingReadsPercent,
+ WarmingReadsTimeout: warmingReadsQueryTimeout,
+ WarmingReadsChannel: e.warmingReadsChannel,
+ }
+}
+
+func (e *Executor) handlePrepare(ctx context.Context, safeSession *econtext.SafeSession, sql string, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) ([]*querypb.Field, error) {
query, comments := sqlparser.SplitMarginComments(sql)
- vcursor, _ := newVCursorImpl(safeSession, comments, e, logStats, e.vm, e.VSchema(), e.resolver.resolver, e.serv, e.warnShardedOnly, e.pv)
+
+ vcursor, _ := econtext.NewVCursorImpl(safeSession, comments, e, logStats, e.vm, e.VSchema(), e.resolver.resolver, e.serv, nullResultsObserver{}, e.vConfig)
stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser())
if err != nil {
@@ -1454,17 +1484,17 @@ func parseAndValidateQuery(query string, parser *sqlparser.Parser) (sqlparser.St
}
// ExecuteMultiShard implements the IExecutor interface
-func (e *Executor) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *SafeSession, autocommit bool, ignoreMaxMemoryRows bool) (qr *sqltypes.Result, errs []error) {
- return e.scatterConn.ExecuteMultiShard(ctx, primitive, rss, queries, session, autocommit, ignoreMaxMemoryRows)
+func (e *Executor) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *econtext.SafeSession, autocommit bool, ignoreMaxMemoryRows bool, resultsObserver econtext.ResultsObserver) (qr *sqltypes.Result, errs []error) {
+ return e.scatterConn.ExecuteMultiShard(ctx, primitive, rss, queries, session, autocommit, ignoreMaxMemoryRows, resultsObserver)
}
// StreamExecuteMulti implements the IExecutor interface
-func (e *Executor) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error) []error {
- return e.scatterConn.StreamExecuteMulti(ctx, primitive, query, rss, vars, session, autocommit, callback)
+func (e *Executor) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *econtext.SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error, resultsObserver econtext.ResultsObserver) []error {
+ return e.scatterConn.StreamExecuteMulti(ctx, primitive, query, rss, vars, session, autocommit, callback, resultsObserver)
}
// ExecuteLock implements the IExecutor interface
-func (e *Executor) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
+func (e *Executor) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *econtext.SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
return e.scatterConn.ExecuteLock(ctx, rs, query, session, lockFuncType)
}
@@ -1575,25 +1605,25 @@ func getTabletThrottlerStatus(tabletHostPort string) (string, error) {
}
// ReleaseLock implements the IExecutor interface
-func (e *Executor) ReleaseLock(ctx context.Context, session *SafeSession) error {
+func (e *Executor) ReleaseLock(ctx context.Context, session *econtext.SafeSession) error {
return e.txConn.ReleaseLock(ctx, session)
}
-// planPrepareStmt implements the IExecutor interface
-func (e *Executor) planPrepareStmt(ctx context.Context, vcursor *vcursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) {
+// PlanPrepareStmt implements the IExecutor interface
+func (e *Executor) PlanPrepareStmt(ctx context.Context, vcursor *econtext.VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) {
stmt, reservedVars, err := parseAndValidateQuery(query, e.env.Parser())
if err != nil {
return nil, nil, err
}
// creating this log stats to not interfere with the original log stats.
- lStats := logstats.NewLogStats(ctx, "prepare", query, vcursor.safeSession.SessionUUID, nil)
+ lStats := logstats.NewLogStats(ctx, "prepare", query, vcursor.Session().GetSessionUUID(), nil)
plan, err := e.getPlan(
ctx,
vcursor,
query,
sqlparser.Clone(stmt),
- vcursor.marginComments,
+ vcursor.GetMarginComments(),
map[string]*querypb.BindVariable{},
reservedVars, /* normalize */
false,
@@ -1615,7 +1645,7 @@ func (e *Executor) Close() {
e.plans.Close()
}
-func (e *Executor) environment() *vtenv.Environment {
+func (e *Executor) Environment() *vtenv.Environment {
return e.env
}
@@ -1623,6 +1653,14 @@ func (e *Executor) ReadTransaction(ctx context.Context, transactionID string) (*
return e.txConn.ReadTransaction(ctx, transactionID)
}
+func (e *Executor) UnresolvedTransactions(ctx context.Context, targets []*querypb.Target) ([]*querypb.TransactionMetadata, error) {
+ return e.txConn.UnresolvedTransactions(ctx, targets)
+}
+
+func (e *Executor) AddWarningCount(name string, count int64) {
+ warnings.Add(name, count)
+}
+
type (
errorTransformer interface {
TransformError(err error) error
@@ -1633,3 +1671,16 @@ type (
func (nullErrorTransformer) TransformError(err error) error {
return err
}
+
+func fkMode(foreignkey string) vschemapb.Keyspace_ForeignKeyMode {
+ switch foreignkey {
+ case "disallow":
+ return vschemapb.Keyspace_disallow
+ case "managed":
+ return vschemapb.Keyspace_managed
+ case "unmanaged":
+ return vschemapb.Keyspace_unmanaged
+
+ }
+ return vschemapb.Keyspace_unspecified
+}
diff --git a/go/vt/vtgate/executor_ddl_test.go b/go/vt/vtgate/executor_ddl_test.go
index 3274fd94475..bf117856e08 100644
--- a/go/vt/vtgate/executor_ddl_test.go
+++ b/go/vt/vtgate/executor_ddl_test.go
@@ -21,14 +21,15 @@ import (
"testing"
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"github.com/stretchr/testify/require"
)
func TestDDLFlags(t *testing.T) {
defer func() {
- enableOnlineDDL = true
- enableDirectDDL = true
+ enableOnlineDDL.Set(true)
+ enableDirectDDL.Set(true)
}()
testcases := []struct {
enableDirectDDL bool
@@ -56,9 +57,9 @@ func TestDDLFlags(t *testing.T) {
for _, testcase := range testcases {
t.Run(fmt.Sprintf("%s-%v-%v", testcase.sql, testcase.enableDirectDDL, testcase.enableOnlineDDL), func(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
- enableDirectDDL = testcase.enableDirectDDL
- enableOnlineDDL = testcase.enableOnlineDDL
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
+ enableDirectDDL.Set(testcase.enableDirectDDL)
+ enableOnlineDDL.Set(testcase.enableOnlineDDL)
_, err := executor.Execute(ctx, nil, "TestDDLFlags", session, testcase.sql, nil)
if testcase.wantErr {
require.EqualError(t, err, testcase.err)
diff --git a/go/vt/vtgate/executor_dml_test.go b/go/vt/vtgate/executor_dml_test.go
index 3dce4e212ef..792e197f48d 100644
--- a/go/vt/vtgate/executor_dml_test.go
+++ b/go/vt/vtgate/executor_dml_test.go
@@ -25,6 +25,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/mysql/config"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
@@ -135,7 +137,6 @@ func TestUpdateEqual(t *testing.T) {
func TestUpdateFromSubQuery(t *testing.T) {
executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
@@ -234,7 +235,7 @@ func TestUpdateInTransactionLookupDefaultReadLock(t *testing.T) {
)}
executor, sbc1, sbc2, sbcLookup, ctx := createCustomExecutorSetValues(t, executorVSchema, res)
- safeSession := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
_, err := executorExecSession(ctx,
executor,
"update t2_lookup set lu_col = 5 where nv_lu_col = 2",
@@ -296,7 +297,7 @@ func TestUpdateInTransactionLookupExclusiveReadLock(t *testing.T) {
)}
executor, sbc1, sbc2, sbcLookup, ctx := createCustomExecutorSetValues(t, executorVSchema, res)
- safeSession := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
_, err := executorExecSession(ctx,
executor,
"update t2_lookup set lu_col = 5 where erl_lu_col = 2",
@@ -358,7 +359,7 @@ func TestUpdateInTransactionLookupSharedReadLock(t *testing.T) {
)}
executor, sbc1, sbc2, sbcLookup, ctx := createCustomExecutorSetValues(t, executorVSchema, res)
- safeSession := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
_, err := executorExecSession(ctx,
executor,
"update t2_lookup set lu_col = 5 where srl_lu_col = 2",
@@ -420,7 +421,7 @@ func TestUpdateInTransactionLookupNoReadLock(t *testing.T) {
)}
executor, sbc1, sbc2, sbcLookup, ctx := createCustomExecutorSetValues(t, executorVSchema, res)
- safeSession := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
_, err := executorExecSession(ctx,
executor,
"update t2_lookup set lu_col = 5 where nrl_lu_col = 2",
@@ -2066,7 +2067,7 @@ func TestInsertPartialFail1(t *testing.T) {
context.Background(),
nil,
"TestExecute",
- NewSafeSession(&vtgatepb.Session{InTransaction: true}),
+ econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}),
"insert into user(id, v, name) values (1, 2, 'myname')",
nil,
)
@@ -2082,7 +2083,7 @@ func TestInsertPartialFail2(t *testing.T) {
// Make the second DML fail, it should result in a rollback.
sbc1.MustFailExecute[sqlparser.StmtInsert] = 1
- safeSession := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
_, err := executor.Execute(
context.Background(),
nil,
@@ -2656,7 +2657,7 @@ func TestReservedConnDML(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestReservedConnDML")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true})
_, err := executor.Execute(ctx, nil, "TestReservedConnDML", session, "use "+KsTestUnsharded, nil)
require.NoError(t, err)
@@ -2708,7 +2709,7 @@ func TestStreamingDML(t *testing.T) {
logChan := executor.queryLogger.Subscribe(method)
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
tcases := []struct {
query string
@@ -2792,7 +2793,7 @@ func TestPartialVindexInsertQueryFailure(t *testing.T) {
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
require.True(t, session.GetAutocommit())
require.False(t, session.InTransaction())
@@ -2845,7 +2846,7 @@ func TestPartialVindexInsertQueryFailureAutoCommit(t *testing.T) {
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
require.True(t, session.GetAutocommit())
require.False(t, session.InTransaction())
@@ -2886,7 +2887,7 @@ func TestPartialVindexInsertQueryFailureAutoCommit(t *testing.T) {
func TestMultiInternalSavepoint(t *testing.T) {
executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
_, err := executorExecSession(ctx, executor, "begin", nil, session.Session)
require.NoError(t, err)
@@ -2935,7 +2936,7 @@ func TestInsertSelectFromDual(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestInsertSelect")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
query := "insert into user(id, v, name) select 1, 2, 'myname' from dual"
wantQueries := []*querypb.BoundQuery{{
@@ -2990,7 +2991,7 @@ func TestInsertSelectFromTable(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestInsertSelect")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
query := "insert into user(id, name) select c1, c2 from music"
wantQueries := []*querypb.BoundQuery{{
@@ -3140,3 +3141,62 @@ func TestDeleteMultiTable(t *testing.T) {
// delete from `user` where (`user`.id) in ::dml_vals - 1 shard
testQueryLog(t, executor, logChan, "TestExecute", "DELETE", "delete `user` from `user` join music on `user`.col = music.col where music.user_id = 1", 18)
}
+
+// TestSessionRowsAffected test that rowsAffected is set correctly for each shard session.
+func TestSessionRowsAffected(t *testing.T) {
+ method := t.Name()
+ executor, _, sbc4060, _, ctx := createExecutorEnv(t)
+
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
+
+ // start the transaction
+ _, err := executor.Execute(ctx, nil, method, session, "begin", nil)
+ require.NoError(t, err)
+
+ // -20 - select query
+ _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 1", nil)
+ require.NoError(t, err)
+ require.Len(t, session.ShardSessions, 1)
+ require.False(t, session.ShardSessions[0].RowsAffected)
+
+ // -20 - update query (rows affected)
+ _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 41 where id = 1", nil)
+ require.NoError(t, err)
+ require.True(t, session.ShardSessions[0].RowsAffected)
+
+ // e0- - select query
+ _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 7", nil)
+ require.NoError(t, err)
+ assert.Len(t, session.ShardSessions, 2)
+ require.False(t, session.ShardSessions[1].RowsAffected)
+
+ // c0-e0 - update query (rows affected)
+ _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 42 where id = 5", nil)
+ require.NoError(t, err)
+ require.Len(t, session.ShardSessions, 3)
+ require.True(t, session.ShardSessions[2].RowsAffected)
+
+ // 40-60 - update query (no rows affected)
+ sbc4060.SetResults([]*sqltypes.Result{{RowsAffected: 0}})
+ _, err = executor.Execute(ctx, nil, method, session, "update user set foo = 42 where id = 3", nil)
+ require.NoError(t, err)
+ assert.Len(t, session.ShardSessions, 4)
+ require.False(t, session.ShardSessions[3].RowsAffected)
+
+ // 40-60 - select query
+ _, err = executor.Execute(ctx, nil, method, session, "select * from user where id = 3", nil)
+ require.NoError(t, err)
+ require.False(t, session.ShardSessions[3].RowsAffected)
+
+ // 40-60 - delete query (rows affected)
+ _, err = executor.Execute(ctx, nil, method, session, "delete from user where id = 3", nil)
+ require.NoError(t, err)
+ require.True(t, session.ShardSessions[0].RowsAffected)
+ require.False(t, session.ShardSessions[1].RowsAffected)
+ require.True(t, session.ShardSessions[2].RowsAffected)
+ require.True(t, session.ShardSessions[3].RowsAffected)
+
+ _, err = executor.Execute(ctx, nil, method, session, "commit", nil)
+ require.NoError(t, err)
+ require.Zero(t, session.ShardSessions)
+}
diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go
index 332139c4a78..2ee3425209f 100644
--- a/go/vt/vtgate/executor_framework_test.go
+++ b/go/vt/vtgate/executor_framework_test.go
@@ -28,6 +28,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/cache/theine"
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/sqltypes"
@@ -307,7 +309,7 @@ func executorExecSession(ctx context.Context, executor *Executor, sql string, bv
ctx,
nil,
"TestExecute",
- NewSafeSession(session),
+ econtext.NewSafeSession(session),
sql,
bv)
}
@@ -320,7 +322,7 @@ func executorPrepare(ctx context.Context, executor *Executor, session *vtgatepb.
return executor.Prepare(
ctx,
"TestExecute",
- NewSafeSession(session),
+ econtext.NewSafeSession(session),
sql,
bv)
}
@@ -331,7 +333,7 @@ func executorStream(ctx context.Context, executor *Executor, sql string) (qr *sq
ctx,
nil,
"TestExecuteStream",
- NewSafeSession(nil),
+ econtext.NewSafeSession(nil),
sql,
nil,
func(qr *sqltypes.Result) error {
diff --git a/go/vt/vtgate/executor_scatter_stats_test.go b/go/vt/vtgate/executor_scatter_stats_test.go
index 84dd2744e8b..b665f850a23 100644
--- a/go/vt/vtgate/executor_scatter_stats_test.go
+++ b/go/vt/vtgate/executor_scatter_stats_test.go
@@ -24,12 +24,13 @@ import (
"github.com/stretchr/testify/require"
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
)
func TestScatterStatsWithNoScatterQuery(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
_, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from main1", nil)
require.NoError(t, err)
@@ -41,7 +42,7 @@ func TestScatterStatsWithNoScatterQuery(t *testing.T) {
func TestScatterStatsWithSingleScatterQuery(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
_, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user", nil)
require.NoError(t, err)
@@ -53,7 +54,7 @@ func TestScatterStatsWithSingleScatterQuery(t *testing.T) {
func TestScatterStatsHttpWriting(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
_, err := executor.Execute(ctx, nil, "TestExecutorResultsExceeded", session, "select * from user", nil)
require.NoError(t, err)
diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go
index 6e3bcdd3eda..86aafaefba4 100644
--- a/go/vt/vtgate/executor_select_test.go
+++ b/go/vt/vtgate/executor_select_test.go
@@ -30,6 +30,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
_flag "vitess.io/vitess/go/internal/flag"
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
@@ -59,7 +61,7 @@ func TestSelectNext(t *testing.T) {
}}
// Autocommit
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
_, err := executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv)
require.NoError(t, err)
@@ -69,7 +71,7 @@ func TestSelectNext(t *testing.T) {
sbclookup.Queries = nil
// Txn
- session = NewAutocommitSession(&vtgatepb.Session{})
+ session = econtext.NewAutocommitSession(&vtgatepb.Session{})
session.Session.InTransaction = true
_, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv)
require.NoError(t, err)
@@ -80,7 +82,7 @@ func TestSelectNext(t *testing.T) {
sbclookup.Queries = nil
// Reserve
- session = NewAutocommitSession(&vtgatepb.Session{})
+ session = econtext.NewAutocommitSession(&vtgatepb.Session{})
session.Session.InReservedConn = true
_, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv)
require.NoError(t, err)
@@ -91,7 +93,7 @@ func TestSelectNext(t *testing.T) {
sbclookup.Queries = nil
// Reserve and Txn
- session = NewAutocommitSession(&vtgatepb.Session{})
+ session = econtext.NewAutocommitSession(&vtgatepb.Session{})
session.Session.InReservedConn = true
session.Session.InTransaction = true
_, err = executor.Execute(context.Background(), nil, "TestSelectNext", session, query, bv)
@@ -107,7 +109,7 @@ func TestSelectDBA(t *testing.T) {
query := "select * from INFORMATION_SCHEMA.foo"
_, err := executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
@@ -117,42 +119,48 @@ func TestSelectDBA(t *testing.T) {
sbc1.Queries = nil
query = "SELECT COUNT(*) FROM INFORMATION_SCHEMA.TABLES ist WHERE ist.table_schema = 'performance_schema' AND ist.table_name = 'foo'"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"__vtschemaname": sqltypes.StringBindVariable("performance_schema"),
"ist_table_name": sqltypes.StringBindVariable("foo"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
sbc1.Queries = nil
query = "select 1 from information_schema.table_constraints where constraint_schema = 'vt_ks' and table_name = 'user'"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"__vtschemaname": sqltypes.StringBindVariable("vt_ks"),
"table_name": sqltypes.StringBindVariable("user"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
sbc1.Queries = nil
query = "select 1 from information_schema.table_constraints where constraint_schema = 'vt_ks'"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select 1 from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"__vtschemaname": sqltypes.StringBindVariable("vt_ks"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
}
@@ -161,7 +169,7 @@ func TestSystemVariablesMySQLBelow80(t *testing.T) {
executor.normalize = true
setVarEnabled = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
sbc1.SetResults([]*sqltypes.Result{{
Fields: []*querypb.Field{
@@ -193,12 +201,8 @@ func TestSystemVariablesMySQLBelow80(t *testing.T) {
func TestSystemVariablesWithSetVarDisabled(t *testing.T) {
executor, sbc1, _, _, _ := createCustomExecutor(t, "{}", "8.0.0")
executor.normalize = true
-
- setVarEnabled = false
- defer func() {
- setVarEnabled = true
- }()
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
+ executor.vConfig.SetVarEnabled = false
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
sbc1.SetResults([]*sqltypes.Result{{
Fields: []*querypb.Field{
@@ -231,7 +235,7 @@ func TestSetSystemVariablesTx(t *testing.T) {
executor, sbc1, _, _, _ := createCustomExecutor(t, "{}", "8.0.1")
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: "TestExecutor"})
_, err := executor.Execute(context.Background(), nil, "TestBegin", session, "begin", map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -277,7 +281,7 @@ func TestSetSystemVariables(t *testing.T) {
executor, _, _, lookup, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded, SystemVariables: map[string]string{}})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded, SystemVariables: map[string]string{}})
// Set @@sql_mode and execute a select statement. We should have SET_VAR in the select statement
@@ -388,7 +392,7 @@ func TestSetSystemVariablesWithReservedConnection(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, SystemVariables: map[string]string{}})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, SystemVariables: map[string]string{}})
sbc1.SetResults([]*sqltypes.Result{{
Fields: []*querypb.Field{
@@ -431,7 +435,7 @@ func TestSelectVindexFunc(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
query := "select * from hash_index where id = 1"
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
_, err := executor.Execute(context.Background(), nil, "TestSelectVindexFunc", session, query, nil)
require.ErrorContains(t, err, "VT09005: no database selected")
@@ -444,7 +448,7 @@ func TestCreateTableValidTimestamp(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor", SystemVariables: map[string]string{"sql_mode": "ALLOW_INVALID_DATES"}})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor", SystemVariables: map[string]string{"sql_mode": "ALLOW_INVALID_DATES"}})
query := "create table aa(t timestamp default 0)"
_, err := executor.Execute(context.Background(), nil, "TestSelect", session, query, map[string]*querypb.BindVariable{})
@@ -462,11 +466,10 @@ func TestCreateTableValidTimestamp(t *testing.T) {
func TestGen4SelectDBA(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
query := "select * from INFORMATION_SCHEMA.TABLE_CONSTRAINTS"
_, err := executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
@@ -477,60 +480,68 @@ func TestGen4SelectDBA(t *testing.T) {
sbc1.Queries = nil
query = "SELECT COUNT(*) FROM INFORMATION_SCHEMA.TABLES ist WHERE ist.table_schema = 'performance_schema' AND ist.table_name = 'foo'"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name1 /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select count(*) from INFORMATION_SCHEMA.`TABLES` as ist where ist.table_schema = :__vtschemaname /* VARCHAR */ and ist.table_name = :ist_table_name1 /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"ist_table_schema": sqltypes.StringBindVariable("performance_schema"),
"__vtschemaname": sqltypes.StringBindVariable("performance_schema"),
"ist_table_name": sqltypes.StringBindVariable("foo"),
"ist_table_name1": sqltypes.StringBindVariable("foo"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
sbc1.Queries = nil
query = "select 1 from information_schema.table_constraints where constraint_schema = 'vt_ks' and table_name = 'user'"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name1 /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */ and table_name = :table_name1 /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"vtg1": sqltypes.Int64BindVariable(1),
"constraint_schema": sqltypes.StringBindVariable("vt_ks"),
"table_name": sqltypes.StringBindVariable("user"),
"__vtschemaname": sqltypes.StringBindVariable("vt_ks"),
"table_name1": sqltypes.StringBindVariable("user"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
sbc1.Queries = nil
query = "select 1 from information_schema.table_constraints where constraint_schema = 'vt_ks'"
- _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{})
+ _, err = executor.Execute(context.Background(), nil, "TestSelectDBA", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"}), query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select :vtg1 /* INT64 */ from information_schema.table_constraints where constraint_schema = :__vtschemaname /* VARCHAR */",
BindVariables: map[string]*querypb.BindVariable{
"vtg1": sqltypes.Int64BindVariable(1),
"constraint_schema": sqltypes.StringBindVariable("vt_ks"),
"__vtschemaname": sqltypes.StringBindVariable("vt_ks"),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
sbc1.Queries = nil
query = "select t.table_schema,t.table_name,c.column_name,c.column_type from tables t join columns c on c.table_schema = t.table_schema and c.table_name = t.table_name where t.table_schema = 'TestExecutor' and c.table_schema = 'TestExecutor' order by t.table_schema,t.table_name,c.column_name"
_, err = executor.Execute(context.Background(), nil, "TestSelectDBA",
- NewSafeSession(&vtgatepb.Session{TargetString: "information_schema"}),
+ econtext.NewSafeSession(&vtgatepb.Session{TargetString: "information_schema"}),
query, map[string]*querypb.BindVariable{},
)
require.NoError(t, err)
- wantQueries = []*querypb.BoundQuery{{Sql: "select t.table_schema, t.table_name, c.column_name, c.column_type from information_schema.`tables` as t, information_schema.`columns` as c where t.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = t.table_schema and c.table_name = t.table_name order by t.table_schema asc, t.table_name asc, c.column_name asc",
+ wantQueries = []*querypb.BoundQuery{{
+ Sql: "select t.table_schema, t.table_name, c.column_name, c.column_type from information_schema.`tables` as t, information_schema.`columns` as c where t.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = :__vtschemaname /* VARCHAR */ and c.table_schema = t.table_schema and c.table_name = t.table_name order by t.table_schema asc, t.table_name asc, c.column_name asc",
BindVariables: map[string]*querypb.BindVariable{
"t_table_schema": sqltypes.StringBindVariable("TestExecutor"),
"__replacevtschemaname": sqltypes.Int64BindVariable(1),
- }}}
+ },
+ }}
utils.MustMatch(t, wantQueries, sbc1.Queries)
}
@@ -637,7 +648,7 @@ func TestStreamBuffering(t *testing.T) {
context.Background(),
nil,
"TestStreamBuffering",
- NewSafeSession(session),
+ econtext.NewSafeSession(session),
"select id from music_user_map where id = 1",
nil,
func(qr *sqltypes.Result) error {
@@ -709,7 +720,7 @@ func TestStreamLimitOffset(t *testing.T) {
context.Background(),
nil,
"TestStreamLimitOffset",
- NewSafeSession(session),
+ econtext.NewSafeSession(session),
"select id, textcol from user order by id limit 2 offset 2",
nil,
func(qr *sqltypes.Result) error {
@@ -1069,7 +1080,7 @@ func TestSelectDatabase(t *testing.T) {
newSession := &vtgatepb.Session{
TargetString: "@primary",
}
- session := NewSafeSession(newSession)
+ session := econtext.NewSafeSession(newSession)
session.TargetString = "TestExecutor@primary"
result, err := executor.Execute(
context.Background(),
@@ -1088,7 +1099,6 @@ func TestSelectDatabase(t *testing.T) {
}
require.NoError(t, err)
utils.MustMatch(t, wantResult, result, "Mismatch")
-
}
func TestSelectBindvars(t *testing.T) {
@@ -1270,7 +1280,6 @@ func TestSelectEqual(t *testing.T) {
func TestSelectINFromOR(t *testing.T) {
executor, sbc1, _, _, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
session := &vtgatepb.Session{
TargetString: "@primary",
@@ -2938,7 +2947,7 @@ func TestSubQueryAndQueryWithLimit(t *testing.T) {
sbc1.SetResults(result1)
sbc2.SetResults(result2)
- exec(executor, NewSafeSession(&vtgatepb.Session{
+ exec(executor, econtext.NewSafeSession(&vtgatepb.Session{
TargetString: "@primary",
}), "select id1, id2 from t1 where id1 >= ( select id1 from t1 order by id1 asc limit 1) limit 100")
require.Equal(t, 2, len(sbc1.Queries))
@@ -2953,6 +2962,67 @@ func TestSubQueryAndQueryWithLimit(t *testing.T) {
assert.Equal(t, `type:INT64 value:"100"`, sbc2.Queries[1].BindVariables["__upper_limit"].String())
}
+func TestSelectUsingMultiEqualOnLookupColumn(t *testing.T) {
+ executor, sbc1, sbc2, sbclookup, _ := createExecutorEnv(t)
+
+ // No results on shard `-20` (`sbc1`), but some lookup results on shard `40-60` (`sbc2`)
+ sbclookup.SetResults([]*sqltypes.Result{{
+ Fields: []*querypb.Field{
+ {Name: "lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
+ {Name: "keyspace_id", Type: sqltypes.VarBinary, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_BINARY_FLAG)},
+ },
+ Rows: [][]sqltypes.Value{{
+ sqltypes.NewInt32(2),
+ sqltypes.MakeTrusted(sqltypes.VarBinary, []byte("\x45")),
+ }},
+ }})
+
+ sbc1.SetResults([]*sqltypes.Result{{
+ Fields: []*querypb.Field{
+ {Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
+ {Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
+ },
+ Rows: [][]sqltypes.Value{},
+ }})
+
+ sbc2.SetResults([]*sqltypes.Result{{
+ Fields: []*querypb.Field{
+ {Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
+ {Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
+ },
+ Rows: [][]sqltypes.Value{{
+ sqltypes.NewInt32(2),
+ sqltypes.NewVarChar("baz"),
+ }},
+ }})
+
+ result, err := exec(executor, econtext.NewSafeSession(&vtgatepb.Session{
+ TargetString: KsTestSharded,
+ }), "select nv_lu_col, other from t2_lookup WHERE (nv_lu_col = 1 AND other = 'bar') OR (nv_lu_col = 2 AND other = 'baz') OR (nv_lu_col = 3 AND other = 'qux') OR (nv_lu_col = 4 AND other = 'brz') OR (nv_lu_col = 5 AND other = 'brz')")
+
+ require.NoError(t, err)
+
+ require.Len(t, sbc1.Queries, 0)
+ require.Len(t, sbc2.Queries, 1)
+
+ require.Equal(t, []*querypb.BoundQuery{{
+ Sql: "select nv_lu_col, other from t2_lookup where nv_lu_col = 1 and other = 'bar' or nv_lu_col = 2 and other = 'baz' or nv_lu_col = 3 and other = 'qux' or nv_lu_col = 4 and other = 'brz' or nv_lu_col = 5 and other = 'brz'",
+ BindVariables: map[string]*querypb.BindVariable{},
+ }}, sbc2.Queries)
+
+ wantResult := &sqltypes.Result{
+ Fields: []*querypb.Field{
+ {Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
+ {Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
+ },
+ Rows: [][]sqltypes.Value{{
+ sqltypes.NewInt32(2),
+ sqltypes.NewVarChar("baz"),
+ }},
+ }
+ require.Equal(t, wantResult, result)
+}
+
func TestCrossShardSubqueryStream(t *testing.T) {
executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
result1 := []*sqltypes.Result{{
@@ -3123,7 +3193,7 @@ func TestSelectWithUnionAll(t *testing.T) {
func TestSelectLock(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
- session := NewSafeSession(nil)
+ session := econtext.NewSafeSession(nil)
session.Session.InTransaction = true
session.ShardSessions = []*vtgatepb.Session_ShardSession{{
Target: &querypb.Target{
@@ -3191,7 +3261,7 @@ func TestLockReserve(t *testing.T) {
"select release_lock('lock name') from dual",
}
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
for _, sql := range tcases {
t.Run(sql, func(t *testing.T) {
@@ -3205,12 +3275,11 @@ func TestLockReserve(t *testing.T) {
_, err := exec(executor, session, "select get_lock('lock name', 10) from dual")
require.NoError(t, err)
require.NotNil(t, session.LockSession)
-
}
func TestSelectFromInformationSchema(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
- session := NewSafeSession(nil)
+ session := econtext.NewSafeSession(nil)
// check failure when trying to query two keyspaces
_, err := exec(executor, session, "SELECT B.TABLE_NAME FROM INFORMATION_SCHEMA.TABLES AS A, INFORMATION_SCHEMA.COLUMNS AS B WHERE A.TABLE_SCHEMA = 'TestExecutor' AND A.TABLE_SCHEMA = 'TestXBadSharding'")
@@ -3337,8 +3406,8 @@ func TestSelectScatterFails(t *testing.T) {
func TestGen4SelectStraightJoin(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select u.id from user u straight_join user2 u2 on u.id = u2.id"
_, err := executor.Execute(context.Background(), nil,
"TestGen4SelectStraightJoin",
@@ -3359,9 +3428,8 @@ func TestGen4SelectStraightJoin(t *testing.T) {
func TestGen4MultiColumnVindexEqual(t *testing.T) {
executor, sbc1, sbc2, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select * from user_region where cola = 1 and colb = 2"
_, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -3398,9 +3466,8 @@ func TestGen4MultiColumnVindexEqual(t *testing.T) {
func TestGen4MultiColumnVindexIn(t *testing.T) {
executor, sbc1, sbc2, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select * from user_region where cola IN (1,17984) and colb IN (2,3,4)"
_, err := executor.Execute(context.Background(), nil, "TestGen4MultiColumnVindex", session, query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -3437,9 +3504,8 @@ func TestGen4MultiColumnVindexIn(t *testing.T) {
func TestGen4MultiColMixedColComparision(t *testing.T) {
executor, sbc1, sbc2, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select * from user_region where colb = 2 and cola IN (1,17984)"
_, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMixedColComparision", session, query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -3474,9 +3540,8 @@ func TestGen4MultiColMixedColComparision(t *testing.T) {
func TestGen4MultiColBestVindexSel(t *testing.T) {
executor, sbc1, sbc2, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select * from user_region where colb = 2 and cola IN (1,17984) and cola = 1"
_, err := executor.Execute(context.Background(), nil, "TestGen4MultiColBestVindexSel", session, query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -3520,9 +3585,8 @@ func TestGen4MultiColBestVindexSel(t *testing.T) {
func TestGen4MultiColMultiEqual(t *testing.T) {
executor, sbc1, sbc2, _, _ := createExecutorEnv(t)
executor.normalize = true
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
query := "select * from user_region where (cola,colb) in ((17984,2),(17984,3))"
_, err := executor.Execute(context.Background(), nil, "TestGen4MultiColMultiEqual", session, query, map[string]*querypb.BindVariable{})
require.NoError(t, err)
@@ -3542,7 +3606,6 @@ func TestGen4MultiColMultiEqual(t *testing.T) {
func TestGen4SelectUnqualifiedReferenceTable(t *testing.T) {
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
query := "select * from zip_detail"
session := &vtgatepb.Session{
@@ -3563,7 +3626,6 @@ func TestGen4SelectUnqualifiedReferenceTable(t *testing.T) {
func TestGen4SelectQualifiedReferenceTable(t *testing.T) {
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
query := fmt.Sprintf("select * from %s.zip_detail", KsTestSharded)
session := &vtgatepb.Session{
@@ -3584,7 +3646,6 @@ func TestGen4SelectQualifiedReferenceTable(t *testing.T) {
func TestGen4JoinUnqualifiedReferenceTable(t *testing.T) {
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
query := "select * from user join zip_detail on user.zip_detail_id = zip_detail.id"
session := &vtgatepb.Session{
@@ -3621,7 +3682,6 @@ func TestGen4JoinUnqualifiedReferenceTable(t *testing.T) {
func TestGen4CrossShardJoinQualifiedReferenceTable(t *testing.T) {
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
query := "select user.id from user join TestUnsharded.zip_detail on user.zip_detail_id = TestUnsharded.zip_detail.id"
session := &vtgatepb.Session{
@@ -3678,7 +3738,6 @@ func TestRegionRange(t *testing.T) {
}
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
tcases := []struct {
regionID int
@@ -3696,7 +3755,7 @@ func TestRegionRange(t *testing.T) {
for _, tcase := range tcases {
t.Run(strconv.Itoa(tcase.regionID), func(t *testing.T) {
sql := fmt.Sprintf("select * from user_region where cola = %d", tcase.regionID)
- _, err := executor.Execute(context.Background(), nil, "TestRegionRange", NewAutocommitSession(&vtgatepb.Session{}), sql, nil)
+ _, err := executor.Execute(context.Background(), nil, "TestRegionRange", econtext.NewAutocommitSession(&vtgatepb.Session{}), sql, nil)
require.NoError(t, err)
count := 0
for _, sbc := range conns {
@@ -3728,7 +3787,6 @@ func TestMultiCol(t *testing.T) {
}
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
tcases := []struct {
cola, colb, colc int
@@ -3744,7 +3802,7 @@ func TestMultiCol(t *testing.T) {
shards: []string{"20a0-"},
}}
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
for _, tcase := range tcases {
t.Run(fmt.Sprintf("%d_%d_%d", tcase.cola, tcase.colb, tcase.colc), func(t *testing.T) {
@@ -3809,7 +3867,6 @@ func TestMultiColPartial(t *testing.T) {
}
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
tcases := []struct {
where string
@@ -3834,7 +3891,7 @@ func TestMultiColPartial(t *testing.T) {
shards: []string{"20a0c0-"},
}}
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
for _, tcase := range tcases {
t.Run(tcase.where, func(t *testing.T) {
@@ -3873,7 +3930,6 @@ func TestSelectAggregationNoData(t *testing.T) {
}
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
tcases := []struct {
sql string
@@ -3913,15 +3969,15 @@ func TestSelectAggregationNoData(t *testing.T) {
{
sql: `select count(*) from (select col1, col2 from user limit 2) x`,
sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1", "int64|int64|int64")),
- expSandboxQ: "select x.col1, x.col2, 1 from (select col1, col2 from `user`) as x limit 2",
- expField: `[name:"count(*)" type:INT64]`,
+ expSandboxQ: "select 1 from (select col1, col2 from `user`) as x limit 2",
+ expField: `[name:"count(*)" type:INT64 charset:63 flags:32769]`,
expRow: `[[INT64(0)]]`,
},
{
sql: `select col2, count(*) from (select col1, col2 from user limit 2) x group by col2`,
sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1|weight_string(col2)", "int64|int64|int64|varbinary")),
- expSandboxQ: "select x.col1, x.col2, 1, weight_string(x.col2) from (select col1, col2 from `user`) as x limit 2",
- expField: `[name:"col2" type:INT64 name:"count(*)" type:INT64]`,
+ expSandboxQ: "select x.col1, x.col2, weight_string(x.col2) from (select col1, col2 from `user`) as x limit 2",
+ expField: `[name:"col2" type:INT64 charset:63 flags:32768 name:"count(*)" type:INT64 charset:63 flags:32769]`,
expRow: `[]`,
},
}
@@ -3965,7 +4021,6 @@ func TestSelectAggregationData(t *testing.T) {
}
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
tcases := []struct {
sql string
@@ -4005,15 +4060,15 @@ func TestSelectAggregationData(t *testing.T) {
{
sql: `select count(*) from (select col1, col2 from user limit 2) x`,
sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1", "int64|int64|int64"), "100|200|1", "200|300|1"),
- expSandboxQ: "select x.col1, x.col2, 1 from (select col1, col2 from `user`) as x limit 2",
- expField: `[name:"count(*)" type:INT64]`,
+ expSandboxQ: "select 1 from (select col1, col2 from `user`) as x limit 2",
+ expField: `[name:"count(*)" type:INT64 charset:63 flags:32769]`,
expRow: `[[INT64(2)]]`,
},
{
sql: `select col2, count(*) from (select col1, col2 from user limit 9) x group by col2`,
sandboxRes: sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1|col2|1|weight_string(col2)", "int64|int64|int64|varbinary"), "100|3|1|NULL", "200|2|1|NULL"),
- expSandboxQ: "select x.col1, x.col2, 1, weight_string(x.col2) from (select col1, col2 from `user`) as x limit 9",
- expField: `[name:"col2" type:INT64 name:"count(*)" type:INT64]`,
+ expSandboxQ: "select x.col1, x.col2, weight_string(x.col2) from (select col1, col2 from `user`) as x limit 9",
+ expField: `[name:"col2" type:INT64 charset:63 flags:32768 name:"count(*)" type:INT64 charset:63 flags:32769]`,
expRow: `[[INT64(2) INT64(4)] [INT64(3) INT64(5)]]`,
},
{
@@ -4123,8 +4178,7 @@ func TestSelectAggregationRandom(t *testing.T) {
executor := createExecutor(ctx, serv, cell, resolver)
defer executor.Close()
- executor.pv = querypb.ExecuteOptions_Gen4
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
rs, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ A.a, A.b, (A.a / A.b) as c from (select sum(a) as a, sum(b) as b from user) A", nil)
require.NoError(t, err)
@@ -4134,7 +4188,7 @@ func TestSelectAggregationRandom(t *testing.T) {
func TestSelectDateTypes(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
qr, err := executor.Execute(context.Background(), nil, "TestSelectDateTypes", session, "select '2020-01-01' + interval month(date_sub(FROM_UNIXTIME(1234), interval 1 month))-1 month", nil)
require.NoError(t, err)
@@ -4145,7 +4199,7 @@ func TestSelectDateTypes(t *testing.T) {
func TestSelectHexAndBit(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
qr, err := executor.Execute(context.Background(), nil, "TestSelectHexAndBit", session, "select 0b1001, b'1001', 0x9, x'09'", nil)
require.NoError(t, err)
@@ -4161,7 +4215,7 @@ func TestSelectHexAndBit(t *testing.T) {
func TestSelectCFC(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
_, err := executor.Execute(context.Background(), nil, "TestSelectCFC", session, "select /*vt+ PLANNER=gen4 */ c2 from tbl_cfc where c1 like 'A%'", nil)
require.NoError(t, err)
@@ -4190,7 +4244,7 @@ func TestSelectView(t *testing.T) {
require.NoError(t, err)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
_, err = executor.Execute(context.Background(), nil, "TestSelectView", session, "select * from user_details_view", nil)
require.NoError(t, err)
@@ -4231,7 +4285,7 @@ func TestWarmingReads(t *testing.T) {
executor, primary, replica := createExecutorEnvWithPrimaryReplicaConn(t, ctx, 100)
executor.normalize = true
- session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
// Since queries on the replica will run in a separate go-routine, we need synchronization for the Queries field in the sandboxconn.
replica.RequireQueriesLocking()
@@ -4295,6 +4349,7 @@ func TestWarmingReads(t *testing.T) {
// waitUntilQueryCount waits until the number of queries run on the tablet reach the specified count.
func waitUntilQueryCount(t *testing.T, tab *sandboxconn.SandboxConn, count int) {
+ t.Helper()
timeout := time.After(1 * time.Second)
for {
select {
@@ -4355,7 +4410,7 @@ func TestStreamJoinQuery(t *testing.T) {
func TestSysVarGlobalAndSession(t *testing.T) {
executor, sbc1, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, SystemVariables: map[string]string{}})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, SystemVariables: map[string]string{}})
sbc1.SetResults([]*sqltypes.Result{
sqltypes.MakeTestResult(sqltypes.MakeTestFields("innodb_lock_wait_timeout", "uint64"), "20"),
@@ -4395,3 +4450,60 @@ func TestSysVarGlobalAndSession(t *testing.T) {
require.NoError(t, err)
require.Equal(t, `[[UINT64(20)]]`, fmt.Sprintf("%v", qr.Rows))
}
+
+func BenchmarkSelectMirror(b *testing.B) {
+ ctx := context.Background()
+ cell := "aa"
+ sql := fmt.Sprintf("select id from %s.user where id = 1", KsTestUnsharded)
+
+ currentSandboxMirrorRules := sandboxMirrorRules
+ b.Cleanup(func() {
+ setSandboxMirrorRules(currentSandboxMirrorRules)
+ })
+
+ // Don't use createExecutorEnv. Doesn't work with benchmarks because of
+ // utils.EnsureNoLeak.
+ createBenchmarkExecutor := func(b *testing.B) (context.Context, *Executor) {
+ ctx, cancel := context.WithCancel(ctx)
+ b.Cleanup(cancel)
+ hc := discovery.NewFakeHealthCheck(nil)
+ u := createSandbox(KsTestUnsharded)
+ s := createSandbox(KsTestSharded)
+ s.VSchema = executorVSchema
+ u.VSchema = unshardedVSchema
+ serv := newSandboxForCells(ctx, []string{cell})
+ resolver := newTestResolver(ctx, hc, serv, cell)
+ shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"}
+ for _, shard := range shards {
+ hc.AddTestTablet(cell, shard, 1, KsTestSharded, shard, topodatapb.TabletType_PRIMARY, true, 1, nil)
+ }
+ hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil)
+ return ctx, createExecutor(ctx, serv, cell, resolver)
+ }
+
+ for _, percent := range []float32{0, 1, 5, 10, 25, 50, 100} {
+ b.Run(fmt.Sprintf("mirror %.2f%%", percent), func(b *testing.B) {
+ setSandboxMirrorRules(fmt.Sprintf(`{
+ "rules": [
+ {
+ "from_table": "%s.user",
+ "to_table": "%s.user",
+ "percent": %.2f
+ }
+ ]
+ }`, KsTestUnsharded, KsTestSharded, percent))
+
+ ctx, executor := createBenchmarkExecutor(b)
+ session := &vtgatepb.Session{
+ TargetString: "@primary",
+ }
+
+ b.ReportAllocs()
+ b.ResetTimer()
+ for i := 0; i < b.N; i++ {
+ executorExec(ctx, executor, session, sql, nil)
+ }
+ b.StopTimer()
+ })
+ }
+}
diff --git a/go/vt/vtgate/executor_set_test.go b/go/vt/vtgate/executor_set_test.go
index 2792c957edd..f8ed0b558c3 100644
--- a/go/vt/vtgate/executor_set_test.go
+++ b/go/vt/vtgate/executor_set_test.go
@@ -22,6 +22,7 @@ import (
"vitess.io/vitess/go/mysql/sqlerror"
querypb "vitess.io/vitess/go/vt/proto/query"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/test/utils"
@@ -266,7 +267,7 @@ func TestExecutorSet(t *testing.T) {
}}
for i, tcase := range testcases {
t.Run(fmt.Sprintf("%d-%s", i, tcase.in), func(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true})
_, err := executorEnv.Execute(ctx, nil, "TestExecute", session, tcase.in, nil)
if tcase.err == "" {
require.NoError(t, err)
@@ -374,7 +375,7 @@ func TestExecutorSetOp(t *testing.T) {
}}
for _, tcase := range testcases {
t.Run(tcase.in, func(t *testing.T) {
- session := NewAutocommitSession(&vtgatepb.Session{
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{
TargetString: "@primary",
})
session.TargetString = KsTestUnsharded
@@ -392,7 +393,7 @@ func TestExecutorSetMetadata(t *testing.T) {
t.Run("Session 1", func(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
set := "set @@vitess_metadata.app_keyspace_v1= '1'"
_, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil)
@@ -400,21 +401,21 @@ func TestExecutorSetMetadata(t *testing.T) {
})
t.Run("Session 2", func(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
set := "set @@vitess_metadata.app_keyspace_v1= '1'"
_, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil)
- assert.NoError(t, err, "%s error: %v", set, err)
+ require.NoError(t, err, "%s error: %v", set, err)
show := `show vitess_metadata variables like 'app\\_keyspace\\_v_'`
result, err := executor.Execute(ctx, nil, "TestExecute", session, show, nil)
- assert.NoError(t, err)
+ require.NoError(t, err)
want := "1"
got := result.Rows[0][1].ToString()
@@ -423,11 +424,11 @@ func TestExecutorSetMetadata(t *testing.T) {
// Update metadata
set = "set @@vitess_metadata.app_keyspace_v2='2'"
_, err = executor.Execute(ctx, nil, "TestExecute", session, set, nil)
- assert.NoError(t, err, "%s error: %v", set, err)
+ require.NoError(t, err, "%s error: %v", set, err)
show = `show vitess_metadata variables like 'app\\_keyspace\\_v%'`
gotqr, err := executor.Execute(ctx, nil, "TestExecute", session, show, nil)
- assert.NoError(t, err)
+ require.NoError(t, err)
wantqr := &sqltypes.Result{
Fields: buildVarCharFields("Key", "Value"),
@@ -469,7 +470,7 @@ func TestPlanExecutorSetUDV(t *testing.T) {
}}
for _, tcase := range testcases {
t.Run(tcase.in, func(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true})
_, err := executor.Execute(ctx, nil, "TestExecute", session, tcase.in, nil)
if err != nil {
require.EqualError(t, err, tcase.err)
@@ -515,7 +516,7 @@ func TestSetVar(t *testing.T) {
executor, _, _, sbc, ctx := createCustomExecutor(t, "{}", "8.0.0")
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded})
sbc.SetResults([]*sqltypes.Result{sqltypes.MakeTestResult(
sqltypes.MakeTestFields("orig|new", "varchar|varchar"),
@@ -554,7 +555,7 @@ func TestSetVarShowVariables(t *testing.T) {
executor, _, _, sbc, ctx := createCustomExecutor(t, "{}", "8.0.0")
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{EnableSystemSettings: true, TargetString: KsTestUnsharded})
sbc.SetResults([]*sqltypes.Result{
// select query result for checking any change in system settings
@@ -597,7 +598,7 @@ func TestExecutorSetAndSelect(t *testing.T) {
sysVar: "tx_isolation",
exp: `[[VARCHAR("READ-UNCOMMITTED")]]`, // this returns the value set in previous query.
}}
- session := NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded, EnableSystemSettings: true})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded, EnableSystemSettings: true})
for _, tcase := range testcases {
t.Run(fmt.Sprintf("%s-%s", tcase.sysVar, tcase.val), func(t *testing.T) {
sbc.ExecCount.Store(0) // reset the value
@@ -625,3 +626,22 @@ func TestExecutorSetAndSelect(t *testing.T) {
})
}
}
+
+// TestTimeZone verifies that setting different time zones in the session
+// results in different outputs for the `now()` function.
+func TestExecutorTimeZone(t *testing.T) {
+ e, _, _, _, ctx := createExecutorEnv(t)
+
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded, EnableSystemSettings: true})
+ session.SetSystemVariable("time_zone", "'+08:00'")
+
+ qr, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, "select now()", nil)
+
+ require.NoError(t, err)
+ session.SetSystemVariable("time_zone", "'+02:00'")
+
+ qrWith, err := e.Execute(ctx, nil, "TestExecutorSetAndSelect", session, "select now()", nil)
+ require.NoError(t, err)
+
+ assert.False(t, qr.Rows[0][0].Equal(qrWith.Rows[0][0]), "%v vs %v", qr.Rows[0][0].ToString(), qrWith.Rows[0][0].ToString())
+}
diff --git a/go/vt/vtgate/executor_stream_test.go b/go/vt/vtgate/executor_stream_test.go
index b8cfeaf3cd5..a8500dd59c4 100644
--- a/go/vt/vtgate/executor_stream_test.go
+++ b/go/vt/vtgate/executor_stream_test.go
@@ -31,6 +31,7 @@ import (
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
"vitess.io/vitess/go/vt/vtenv"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vtgate/logstats"
_ "vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vttablet/sandboxconn"
@@ -102,7 +103,7 @@ func executorStreamMessages(executor *Executor, sql string) (qr *sqltypes.Result
ctx,
nil,
"TestExecuteStream",
- NewSafeSession(session),
+ econtext.NewSafeSession(session),
sql,
nil,
func(qr *sqltypes.Result) error {
diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go
index b8e2b996780..74bfb710582 100644
--- a/go/vt/vtgate/executor_test.go
+++ b/go/vt/vtgate/executor_test.go
@@ -36,12 +36,19 @@ import (
"github.com/stretchr/testify/require"
"google.golang.org/protobuf/proto"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/discovery"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vtgate/buffer"
@@ -50,12 +57,6 @@ import (
"vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vtgate/vschemaacl"
"vitess.io/vitess/go/vt/vtgate/vtgateservice"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vschemapb "vitess.io/vitess/go/vt/proto/vschema"
- vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
func TestExecutorResultsExceeded(t *testing.T) {
@@ -65,7 +66,7 @@ func TestExecutorResultsExceeded(t *testing.T) {
warnMemoryRows = 3
defer func() { warnMemoryRows = save }()
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
initial := warnings.Counts()["ResultsExceeded"]
@@ -89,7 +90,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) {
maxMemoryRows = 3
defer func() { maxMemoryRows = save }()
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
result := sqltypes.MakeTestResult(sqltypes.MakeTestFields("col", "int64"), "1", "2", "3", "4")
fn := func(r *sqltypes.Result) error {
return nil
@@ -123,7 +124,7 @@ func TestExecutorMaxMemoryRowsExceeded(t *testing.T) {
func TestExecutorTransactionsNoAutoCommit(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", SessionUUID: "suuid"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", SessionUUID: "suuid"})
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
@@ -189,7 +190,7 @@ func TestExecutorTransactionsNoAutoCommit(t *testing.T) {
}
// Prevent use of non-primary if in_transaction is on.
- session = NewSafeSession(&vtgatepb.Session{TargetString: "@primary", InTransaction: true})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", InTransaction: true})
_, err = executor.Execute(ctx, nil, "TestExecute", session, "use @replica", nil)
require.EqualError(t, err, `can't execute the given command because you have an active transaction`)
}
@@ -206,7 +207,7 @@ func TestDirectTargetRewrites(t *testing.T) {
}
sql := "select database()"
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(session), sql, map[string]*querypb.BindVariable{})
require.NoError(t, err)
assertQueries(t, sbclookup, []*querypb.BoundQuery{{
Sql: "select :__vtdbname as `database()` from dual",
@@ -217,7 +218,7 @@ func TestDirectTargetRewrites(t *testing.T) {
func TestExecutorTransactionsAutoCommit(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true, SessionUUID: "suuid"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true, SessionUUID: "suuid"})
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
@@ -271,7 +272,7 @@ func TestExecutorTransactionsAutoCommitStreaming(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
oltpOptions := &querypb.ExecuteOptions{Workload: querypb.ExecuteOptions_OLTP}
- session := NewSafeSession(&vtgatepb.Session{
+ session := econtext.NewSafeSession(&vtgatepb.Session{
TargetString: "@primary",
Autocommit: true,
Options: oltpOptions,
@@ -334,13 +335,13 @@ func TestExecutorTransactionsAutoCommitStreaming(t *testing.T) {
}
func TestExecutorDeleteMetadata(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
set := "set @@vitess_metadata.app_v1= '1'"
_, err := executor.Execute(ctx, nil, "TestExecute", session, set, nil)
@@ -368,7 +369,7 @@ func TestExecutorDeleteMetadata(t *testing.T) {
func TestExecutorAutocommit(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
logChan := executor.queryLogger.Subscribe("Test")
defer executor.queryLogger.Unsubscribe(logChan)
@@ -447,7 +448,7 @@ func TestExecutorAutocommit(t *testing.T) {
// transition autocommit from 0 to 1 in the middle of a transaction.
startCount = sbclookup.CommitCount.Load()
- session = NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
_, err = executor.Execute(ctx, nil, "TestExecute", session, "begin", nil)
require.NoError(t, err)
_, err = executor.Execute(ctx, nil, "TestExecute", session, "update main1 set id=1", nil)
@@ -469,7 +470,7 @@ func TestExecutorAutocommit(t *testing.T) {
func TestExecutorShowColumns(t *testing.T) {
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: ""})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ""})
queries := []string{
"SHOW COLUMNS FROM `user` in `TestExecutor`",
@@ -521,7 +522,7 @@ func assertMatchesNoOrder(t *testing.T, expected, got string) {
func TestExecutorShow(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
for _, query := range []string{"show vitess_keyspaces", "show keyspaces"} {
qr, err := executor.Execute(ctx, nil, "TestExecute", session, query, nil)
@@ -546,7 +547,7 @@ func TestExecutorShow(t *testing.T) {
_, err = executor.Execute(ctx, nil, "TestExecute", session, "use @primary", nil)
require.NoError(t, err)
_, err = executor.Execute(ctx, nil, "TestExecute", session, "show tables", nil)
- assert.EqualError(t, err, errNoKeyspace.Error(), "'show tables' should fail without a keyspace")
+ assert.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "'show tables' should fail without a keyspace")
assert.Empty(t, sbclookup.Queries, "sbclookup unexpectedly has queries already")
showResults := &sqltypes.Result{
@@ -921,7 +922,7 @@ func TestExecutorShow(t *testing.T) {
query = "show vschema vindexes on user"
_, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil)
- wantErr := errNoKeyspace.Error()
+ wantErr := econtext.ErrNoKeyspace.Error()
assert.EqualError(t, err, wantErr, query)
query = "show vschema vindexes on TestExecutor.garbage"
@@ -1025,7 +1026,7 @@ func TestExecutorShow(t *testing.T) {
utils.MustMatch(t, wantqr, qr, fmt.Sprintf("%s, with a bad keyspace", query))
query = "show vschema tables"
- session = NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
qr, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil)
require.NoError(t, err)
wantqr = &sqltypes.Result{
@@ -1051,9 +1052,9 @@ func TestExecutorShow(t *testing.T) {
utils.MustMatch(t, wantqr, qr, query)
query = "show vschema tables"
- session = NewSafeSession(&vtgatepb.Session{})
+ session = econtext.NewSafeSession(&vtgatepb.Session{})
_, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil)
- want = errNoKeyspace.Error()
+ want = econtext.ErrNoKeyspace.Error()
assert.EqualError(t, err, want, query)
query = "show 10"
@@ -1062,7 +1063,7 @@ func TestExecutorShow(t *testing.T) {
assert.EqualError(t, err, want, query)
query = "show vschema tables"
- session = NewSafeSession(&vtgatepb.Session{TargetString: "no_such_keyspace"})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "no_such_keyspace"})
_, err = executor.Execute(ctx, nil, "TestExecute", session, query, nil)
want = "VT05003: unknown database 'no_such_keyspace' in vschema"
assert.EqualError(t, err, want, query)
@@ -1081,7 +1082,7 @@ func TestExecutorShow(t *testing.T) {
func TestExecutorShowTargeted(t *testing.T) {
executor, _, sbc2, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor/40-60"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor/40-60"})
queries := []string{
"show databases",
@@ -1108,7 +1109,7 @@ func TestExecutorShowTargeted(t *testing.T) {
func TestExecutorShowFromSystemSchema(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "mysql"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "mysql"})
_, err := executor.Execute(ctx, nil, "TestExecutorShowFromSystemSchema", session, "show tables", nil)
require.NoError(t, err)
@@ -1117,7 +1118,7 @@ func TestExecutorShowFromSystemSchema(t *testing.T) {
func TestExecutorUse(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "@primary"})
stmts := []string{
"use TestExecutor",
@@ -1136,13 +1137,13 @@ func TestExecutorUse(t *testing.T) {
utils.MustMatch(t, wantSession, session.Session, "session does not match")
}
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{}), "use 1", nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "use 1", nil)
wantErr := "syntax error at position 6 near '1'"
if err == nil || err.Error() != wantErr {
t.Errorf("got: %v, want %v", err, wantErr)
}
- _, err = executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{}), "use UnexistentKeyspace", nil)
+ _, err = executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "use UnexistentKeyspace", nil)
require.EqualError(t, err, "VT05003: unknown database 'UnexistentKeyspace' in vschema")
}
@@ -1156,7 +1157,7 @@ func TestExecutorComment(t *testing.T) {
wantResult := &sqltypes.Result{}
for _, stmt := range stmts {
- gotResult, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil)
+ gotResult, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil)
if err != nil {
t.Error(err)
}
@@ -1241,9 +1242,9 @@ func TestExecutorDDL(t *testing.T) {
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
stmtType := "DDL"
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
if tc.hasNoKeyspaceErr {
- require.EqualError(t, err, errNoKeyspace.Error(), "expect query to fail: %q", stmt)
+ require.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "expect query to fail: %q", stmt)
stmtType = "" // For error case, plan is not generated to query log will not contain any stmtType.
} else {
require.NoError(t, err, "did not expect error for query: %q", stmt)
@@ -1279,9 +1280,9 @@ func TestExecutorDDL(t *testing.T) {
sbc1.ExecCount.Store(0)
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: ""}), stmt.input, nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: ""}), stmt.input, nil)
if stmt.hasErr {
- require.EqualError(t, err, errNoKeyspace.Error(), "expect query to fail")
+ require.EqualError(t, err, econtext.ErrNoKeyspace.Error(), "expect query to fail")
testQueryLog(t, executor, logChan, "TestExecute", "", stmt.input, 0)
} else {
require.NoError(t, err)
@@ -1298,13 +1299,13 @@ func TestExecutorDDLFk(t *testing.T) {
}
for _, stmt := range stmts {
- for _, fkMode := range []string{"allow", "disallow"} {
- t.Run(stmt+fkMode, func(t *testing.T) {
+ for _, mode := range []string{"allow", "disallow"} {
+ t.Run(stmt+mode, func(t *testing.T) {
executor, _, _, sbc, ctx := createExecutorEnv(t)
sbc.ExecCount.Store(0)
- foreignKeyMode = fkMode
- _, err := executor.Execute(ctx, nil, mName, NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil)
- if fkMode == "allow" {
+ executor.vConfig.ForeignKeyMode = fkMode(mode)
+ _, err := executor.Execute(ctx, nil, mName, econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), stmt, nil)
+ if mode == "allow" {
require.NoError(t, err)
require.EqualValues(t, 1, sbc.ExecCount.Load())
} else {
@@ -1317,13 +1318,13 @@ func TestExecutorDDLFk(t *testing.T) {
}
func TestExecutorAlterVSchemaKeyspace(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
vschemaUpdates := make(chan *vschemapb.SrvVSchema, 2)
executor.serv.WatchSrvVSchema(ctx, executor.cell, func(vschema *vschemapb.SrvVSchema, err error) bool {
@@ -1346,9 +1347,9 @@ func TestExecutorAlterVSchemaKeyspace(t *testing.T) {
}
func TestExecutorCreateVindexDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
ks := "TestExecutor"
@@ -1365,7 +1366,7 @@ func TestExecutorCreateVindexDDL(t *testing.T) {
t.Fatalf("test_vindex should not exist in original vschema")
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema create vindex test_vindex using hash"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.NoError(t, err)
@@ -1389,7 +1390,7 @@ func TestExecutorCreateVindexDDL(t *testing.T) {
// Create a new vschema keyspace implicitly by creating a vindex with a different
// target in the session
// ksNew := "test_new_keyspace"
- session = NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt = "alter vschema create vindex test_vindex2 using hash"
_, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
if err != nil {
@@ -1416,9 +1417,9 @@ func TestExecutorCreateVindexDDL(t *testing.T) {
}
func TestExecutorAddDropVschemaTableDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
ks := KsTestUnsharded
@@ -1440,7 +1441,7 @@ func TestExecutorAddDropVschemaTableDDL(t *testing.T) {
vschemaTables = append(vschemaTables, t)
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema add table test_table"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.NoError(t, err)
@@ -1452,7 +1453,7 @@ func TestExecutorAddDropVschemaTableDDL(t *testing.T) {
_ = waitForVschemaTables(t, ks, append([]string{"test_table", "test_table2"}, vschemaTables...), executor)
// Should fail adding a table on a sharded keyspace
- session = NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
stmt = "alter vschema add table test_table"
_, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.EqualError(t, err, "add vschema table: unsupported on sharded keyspace TestExecutor")
@@ -1471,7 +1472,7 @@ func TestExecutorVindexDDLACL(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
ks := "TestExecutor"
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
ctxRedUser := callerid.NewContext(ctx, &vtrpcpb.CallerID{}, &querypb.VTGateCallerID{Username: "redUser"})
ctxBlueUser := callerid.NewContext(ctx, &vtrpcpb.CallerID{}, &querypb.VTGateCallerID{Username: "blueUser"})
@@ -1485,8 +1486,7 @@ func TestExecutorVindexDDLACL(t *testing.T) {
require.EqualError(t, err, `User 'blueUser' is not authorized to perform vschema operations`)
// test when all users are enabled
- vschemaacl.AuthorizedDDLUsers = "%"
- vschemaacl.Init()
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
_, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil)
if err != nil {
t.Errorf("unexpected error '%v'", err)
@@ -1498,8 +1498,7 @@ func TestExecutorVindexDDLACL(t *testing.T) {
}
// test when only one user is enabled
- vschemaacl.AuthorizedDDLUsers = "orangeUser, blueUser, greenUser"
- vschemaacl.Init()
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("orangeUser, blueUser, greenUser"))
_, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil)
require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`)
@@ -1510,16 +1509,30 @@ func TestExecutorVindexDDLACL(t *testing.T) {
}
// restore the disallowed state
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}
func TestExecutorUnrecognized(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{}), "invalid statement", nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{}), "invalid statement", nil)
require.Error(t, err, "unrecognized statement: invalid statement'")
}
+func TestExecutorDeniedErrorNoBuffer(t *testing.T) {
+ executor, sbc1, _, _, ctx := createExecutorEnv(t)
+ sbc1.EphemeralShardErr = errors.New("enforce denied tables")
+
+ vschemaWaitTimeout = 500 * time.Millisecond
+
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: "@primary"})
+ startExec := time.Now()
+ _, err := executor.Execute(ctx, nil, "TestExecutorDeniedErrorNoBuffer", session, "select * from user", nil)
+ require.NoError(t, err, "enforce denied tables not buffered")
+ endExec := time.Now()
+ require.GreaterOrEqual(t, endExec.Sub(startExec).Milliseconds(), int64(500))
+}
+
// TestVSchemaStats makes sure the building and displaying of the
// VSchemaStats works.
func TestVSchemaStats(t *testing.T) {
@@ -1546,9 +1559,8 @@ var pv = querypb.ExecuteOptions_Gen4
func TestGetPlanUnnormalized(t *testing.T) {
r, _, _, _, ctx := createExecutorEnv(t)
-
- emptyvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
- unshardedvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
+ unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
query1 := "select * from music_user_map where id = 1"
plan1, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
@@ -1591,7 +1603,7 @@ func assertCacheSize(t *testing.T, c *PlanCache, expected int) {
}
}
-func assertCacheContains(t *testing.T, e *Executor, vc *vcursorImpl, sql string) *engine.Plan {
+func assertCacheContains(t *testing.T, e *Executor, vc *econtext.VCursorImpl, sql string) *engine.Plan {
t.Helper()
var plan *engine.Plan
@@ -1610,9 +1622,9 @@ func assertCacheContains(t *testing.T, e *Executor, vc *vcursorImpl, sql string)
return plan
}
-func getPlanCached(t *testing.T, ctx context.Context, e *Executor, vcursor *vcursorImpl, sql string, comments sqlparser.MarginComments, bindVars map[string]*querypb.BindVariable, skipQueryPlanCache bool) (*engine.Plan, *logstats.LogStats) {
+func getPlanCached(t *testing.T, ctx context.Context, e *Executor, vcursor *econtext.VCursorImpl, sql string, comments sqlparser.MarginComments, bindVars map[string]*querypb.BindVariable, skipQueryPlanCache bool) (*engine.Plan, *logstats.LogStats) {
logStats := logstats.NewLogStats(ctx, "Test", "", "", nil)
- vcursor.safeSession = &SafeSession{
+ vcursor.SafeSession = &econtext.SafeSession{
Session: &vtgatepb.Session{
Options: &querypb.ExecuteOptions{SkipQueryPlanCache: skipQueryPlanCache}},
}
@@ -1631,7 +1643,7 @@ func TestGetPlanCacheUnnormalized(t *testing.T) {
t.Run("Cache", func(t *testing.T) {
r, _, _, _, ctx := createExecutorEnv(t)
- emptyvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
query1 := "select * from music_user_map where id = 1"
_, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true)
@@ -1655,7 +1667,7 @@ func TestGetPlanCacheUnnormalized(t *testing.T) {
// Skip cache using directive
r, _, _, _, ctx := createExecutorEnv(t)
- unshardedvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
query1 := "insert /*vt+ SKIP_QUERY_PLAN_CACHE=1 */ into user(id) values (1), (2)"
getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
@@ -1666,12 +1678,12 @@ func TestGetPlanCacheUnnormalized(t *testing.T) {
assertCacheSize(t, r.plans, 1)
// the target string will be resolved and become part of the plan cache key, which adds a new entry
- ksIDVc1, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ ksIDVc1, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
getPlanCached(t, ctx, r, ksIDVc1, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
assertCacheSize(t, r.plans, 2)
// the target string will be resolved and become part of the plan cache key, as it's an unsharded ks, it will be the same entry as above
- ksIDVc2, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ ksIDVc2, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
getPlanCached(t, ctx, r, ksIDVc2, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
assertCacheSize(t, r.plans, 2)
})
@@ -1681,7 +1693,7 @@ func TestGetPlanCacheNormalized(t *testing.T) {
t.Run("Cache", func(t *testing.T) {
r, _, _, _, ctx := createExecutorEnv(t)
r.normalize = true
- emptyvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
query1 := "select * from music_user_map where id = 1"
_, logStats1 := getPlanCached(t, ctx, r, emptyvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, true /* skipQueryPlanCache */)
@@ -1698,7 +1710,7 @@ func TestGetPlanCacheNormalized(t *testing.T) {
// Skip cache using directive
r, _, _, _, ctx := createExecutorEnv(t)
r.normalize = true
- unshardedvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
query1 := "insert /*vt+ SKIP_QUERY_PLAN_CACHE=1 */ into user(id) values (1), (2)"
getPlanCached(t, ctx, r, unshardedvc, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
@@ -1709,12 +1721,12 @@ func TestGetPlanCacheNormalized(t *testing.T) {
assertCacheSize(t, r.plans, 1)
// the target string will be resolved and become part of the plan cache key, which adds a new entry
- ksIDVc1, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ ksIDVc1, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[deadbeef]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
getPlanCached(t, ctx, r, ksIDVc1, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
assertCacheSize(t, r.plans, 2)
// the target string will be resolved and become part of the plan cache key, as it's an unsharded ks, it will be the same entry as above
- ksIDVc2, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ ksIDVc2, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "[beefdead]"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, r.vConfig)
getPlanCached(t, ctx, r, ksIDVc2, query1, makeComments(" /* comment */"), map[string]*querypb.BindVariable{}, false)
assertCacheSize(t, r.plans, 2)
})
@@ -1724,8 +1736,8 @@ func TestGetPlanNormalized(t *testing.T) {
r, _, _, _, ctx := createExecutorEnv(t)
r.normalize = true
- emptyvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
- unshardedvc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ emptyvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
+ unshardedvc, _ := econtext.NewVCursorImpl(econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded + "@unknown"}), makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
query1 := "select * from music_user_map where id = 1"
query2 := "select * from music_user_map where id = 2"
@@ -1772,7 +1784,7 @@ func TestGetPlanPriority(t *testing.T) {
{name: "empty priority", sql: "select * from music_user_map", expectedPriority: "", expectedError: nil},
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@unknown", Options: &querypb.ExecuteOptions{}})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@unknown", Options: &querypb.ExecuteOptions{}})
for _, aTestCase := range testCases {
testCase := aTestCase
@@ -1782,20 +1794,21 @@ func TestGetPlanPriority(t *testing.T) {
r.normalize = true
logStats := logstats.NewLogStats(ctx, "Test", "", "", nil)
- vCursor, err := newVCursorImpl(session, makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, false, pv)
+ vCursor, err := econtext.NewVCursorImpl(session, makeComments(""), r, nil, r.vm, r.VSchema(), r.resolver.resolver, nil, nullResultsObserver{}, econtext.VCursorConfig{})
assert.NoError(t, err)
stmt, err := sqlparser.NewTestParser().Parse(testCase.sql)
assert.NoError(t, err)
- crticalityFromStatement, _ := sqlparser.GetPriorityFromStatement(stmt)
+ qh, _ := sqlparser.BuildQueryHints(stmt)
+ priorityFromStatement := qh.Priority
_, err = r.getPlan(context.Background(), vCursor, testCase.sql, stmt, makeComments("/* some comment */"), map[string]*querypb.BindVariable{}, nil, true, logStats)
if testCase.expectedError != nil {
assert.ErrorIs(t, err, testCase.expectedError)
} else {
assert.NoError(t, err)
- assert.Equal(t, testCase.expectedPriority, crticalityFromStatement)
- assert.Equal(t, testCase.expectedPriority, vCursor.safeSession.Options.Priority)
+ assert.Equal(t, testCase.expectedPriority, priorityFromStatement)
+ assert.Equal(t, testCase.expectedPriority, vCursor.SafeSession.Options.Priority)
}
})
}
@@ -1952,7 +1965,7 @@ func TestExecutorMaxPayloadSizeExceeded(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
warningCount := warnings.Counts()["WarnPayloadSizeExceeded"]
testMaxPayloadSizeExceeded := []string{
"select * from main1",
@@ -2000,7 +2013,7 @@ func TestOlapSelectDatabase(t *testing.T) {
cbInvoked = true
return nil
}
- err := executor.StreamExecute(context.Background(), nil, "TestExecute", NewSafeSession(session), sql, nil, cb)
+ err := executor.StreamExecute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(session), sql, nil, cb)
assert.NoError(t, err)
assert.True(t, cbInvoked)
}
@@ -2008,7 +2021,7 @@ func TestOlapSelectDatabase(t *testing.T) {
func TestExecutorClearsWarnings(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{
+ session := econtext.NewSafeSession(&vtgatepb.Session{
Warnings: []*querypb.QueryWarning{{Code: 234, Message: "oh noes"}},
})
_, err := executor.Execute(context.Background(), nil, "TestExecute", session, "select 42", nil)
@@ -2025,7 +2038,6 @@ func TestServingKeyspaces(t *testing.T) {
executor, sbc1, _, sbclookup, ctx := createExecutorEnv(t)
- executor.pv = querypb.ExecuteOptions_Gen4
gw, ok := executor.resolver.resolver.GetGateway().(*TabletGateway)
require.True(t, ok)
hc := gw.hc.(*discovery.FakeHealthCheck)
@@ -2044,7 +2056,7 @@ func TestServingKeyspaces(t *testing.T) {
})
require.ElementsMatch(t, []string{"TestExecutor", "TestUnsharded"}, gw.GetServingKeyspaces())
- result, err := executor.Execute(ctx, nil, "TestServingKeyspaces", NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil)
+ result, err := executor.Execute(ctx, nil, "TestServingKeyspaces", econtext.NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil)
require.NoError(t, err)
require.Equal(t, `[[VARCHAR("TestExecutor")]]`, fmt.Sprintf("%v", result.Rows))
@@ -2060,7 +2072,7 @@ func TestServingKeyspaces(t *testing.T) {
// Clear plan cache, to force re-planning of the query.
executor.ClearPlans()
require.ElementsMatch(t, []string{"TestUnsharded"}, gw.GetServingKeyspaces())
- result, err = executor.Execute(ctx, nil, "TestServingKeyspaces", NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil)
+ result, err = executor.Execute(ctx, nil, "TestServingKeyspaces", econtext.NewSafeSession(&vtgatepb.Session{}), "select keyspace_name from dual", nil)
require.NoError(t, err)
require.Equal(t, `[[VARCHAR("TestUnsharded")]]`, fmt.Sprintf("%v", result.Rows))
}
@@ -2136,9 +2148,9 @@ func TestExecutorOther(t *testing.T) {
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
if tc.hasNoKeyspaceErr {
- assert.Error(t, err, errNoKeyspace)
+ assert.Error(t, err, econtext.ErrNoKeyspace.Error())
} else if tc.hasDestinationShardErr {
assert.Errorf(t, err, "Destination can only be a single shard for statement: %s", stmt)
} else {
@@ -2192,7 +2204,7 @@ func TestExecutorAnalyze(t *testing.T) {
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
require.NoError(t, err)
utils.MustMatch(t, tc.wantCnts, cnts{
@@ -2256,7 +2268,7 @@ func TestExecutorExplainStmt(t *testing.T) {
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
- _, err := executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
+ _, err := executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
assert.NoError(t, err)
utils.MustMatch(t, tc.wantCnts, cnts{
@@ -2289,7 +2301,7 @@ func TestExecutorVExplain(t *testing.T) {
result, err = executorExec(ctx, executor, session, "vexplain plan select 42", bindVars)
require.NoError(t, err)
- expected := `[[VARCHAR("{\n\t\"OperatorType\": \"Projection\",\n\t\"Expressions\": [\n\t\t\"42 as 42\"\n\t],\n\t\"Inputs\": [\n\t\t{\n\t\t\t\"OperatorType\": \"SingleRow\"\n\t\t}\n\t]\n}")]]`
+ expected := `[[VARCHAR("{\n\t\"OperatorType\": \"Projection\",\n\t\"Expressions\": [\n\t\t\":vtg1 as :vtg1 /* INT64 */\"\n\t],\n\t\"Inputs\": [\n\t\t{\n\t\t\t\"OperatorType\": \"SingleRow\"\n\t\t}\n\t]\n}")]]`
require.Equal(t, expected, fmt.Sprintf("%v", result.Rows))
}
@@ -2346,9 +2358,9 @@ func TestExecutorOtherAdmin(t *testing.T) {
sbc2.ExecCount.Store(0)
sbclookup.ExecCount.Store(0)
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), stmt, nil)
if tc.hasNoKeyspaceErr {
- assert.Error(t, err, errNoKeyspace)
+ assert.Error(t, err, econtext.ErrNoKeyspace.Error())
} else if tc.hasDestinationShardErr {
assert.Errorf(t, err, "Destination can only be a single shard for statement: %s, got: DestinationExactKeyRange(-)", stmt)
} else {
@@ -2373,7 +2385,7 @@ func TestExecutorSavepointInTx(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestExecutorSavepoint")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewSafeSession(&vtgatepb.Session{Autocommit: false, TargetString: "@primary"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: false, TargetString: "@primary"})
_, err := exec(executor, session, "savepoint a")
require.NoError(t, err)
_, err = exec(executor, session, "rollback to a")
@@ -2456,7 +2468,7 @@ func TestExecutorSavepointInTxWithReservedConn(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestExecutorSavepoint")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "TestExecutor", EnableSystemSettings: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "TestExecutor", EnableSystemSettings: true})
sbc1.SetResults([]*sqltypes.Result{
sqltypes.MakeTestResult(sqltypes.MakeTestFields("orig|new", "varchar|varchar"), "a|"),
})
@@ -2523,7 +2535,7 @@ func TestExecutorSavepointWithoutTx(t *testing.T) {
logChan := executor.queryLogger.Subscribe("TestExecutorSavepoint")
defer executor.queryLogger.Unsubscribe(logChan)
- session := NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "@primary", InTransaction: false})
+ session := econtext.NewSafeSession(&vtgatepb.Session{Autocommit: true, TargetString: "@primary", InTransaction: false})
_, err := exec(executor, session, "savepoint a")
require.NoError(t, err)
_, err = exec(executor, session, "rollback to a")
@@ -2608,9 +2620,9 @@ func TestExecutorCallProc(t *testing.T) {
sbc2.ExecCount.Store(0)
sbcUnsharded.ExecCount.Store(0)
- _, err := executor.Execute(context.Background(), nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), "CALL proc()", nil)
+ _, err := executor.Execute(context.Background(), nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), "CALL proc()", nil)
if tc.hasNoKeyspaceErr {
- assert.EqualError(t, err, errNoKeyspace.Error())
+ assert.EqualError(t, err, econtext.ErrNoKeyspace.Error())
} else if tc.unshardedOnlyErr {
require.EqualError(t, err, "CALL is not supported for sharded keyspace")
} else {
@@ -2630,9 +2642,9 @@ func TestExecutorTempTable(t *testing.T) {
executor, _, _, sbcUnsharded, ctx := createExecutorEnv(t)
initialWarningsCount := warnings.Counts()["WarnUnshardedOnly"]
- executor.warnShardedOnly = true
+ executor.vConfig.WarnShardedOnly = true
creatQuery := "create temporary table temp_t(id bigint primary key)"
- session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded})
_, err := executor.Execute(ctx, nil, "TestExecutorTempTable", session, creatQuery, nil)
require.NoError(t, err)
assert.EqualValues(t, 1, sbcUnsharded.ExecCount.Load())
@@ -2651,7 +2663,7 @@ func TestExecutorShowVitessMigrations(t *testing.T) {
executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
showQuery := "show vitess_migrations"
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
_, err := executor.Execute(ctx, nil, "", session, showQuery, nil)
require.NoError(t, err)
assert.Contains(t, sbc1.StringQueries(), "show vitess_migrations")
@@ -2662,7 +2674,7 @@ func TestExecutorDescHash(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
showQuery := "desc hash_index"
- session := NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
_, err := executor.Execute(ctx, nil, "", session, showQuery, nil)
require.NoError(t, err)
}
@@ -2670,7 +2682,7 @@ func TestExecutorDescHash(t *testing.T) {
func TestExecutorVExplainQueries(t *testing.T) {
executor, _, _, sbclookup, ctx := createExecutorEnv(t)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
sbclookup.SetResults([]*sqltypes.Result{
sqltypes.MakeTestResult(sqltypes.MakeTestFields("name|user_id", "varchar|int64"), "apa|1", "apa|2"),
@@ -2683,7 +2695,7 @@ func TestExecutorVExplainQueries(t *testing.T) {
// Test the streaming side as well
var results []sqltypes.Row
- session = NewAutocommitSession(&vtgatepb.Session{})
+ session = econtext.NewAutocommitSession(&vtgatepb.Session{})
err = executor.StreamExecute(ctx, nil, "TestExecutorVExplainQueries", session, "vexplain queries select * from user where name = 'apa'", nil, func(result *sqltypes.Result) error {
results = append(results, result.Rows...)
return nil
@@ -2696,7 +2708,7 @@ func TestExecutorVExplainQueries(t *testing.T) {
func TestExecutorStartTxnStmt(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
tcases := []struct {
beginSQL string
@@ -2743,7 +2755,7 @@ func TestExecutorPrepareExecute(t *testing.T) {
executor, _, _, _, _ := createExecutorEnv(t)
executor.normalize = true
- session := NewAutocommitSession(&vtgatepb.Session{})
+ session := econtext.NewAutocommitSession(&vtgatepb.Session{})
// prepare statement.
_, err := executor.Execute(context.Background(), nil, "TestExecutorPrepareExecute", session, "prepare prep_user from 'select * from user where id = ?'", nil)
@@ -2779,6 +2791,77 @@ func TestExecutorPrepareExecute(t *testing.T) {
require.Error(t, err)
}
+// TestExecutorSettingsInTwoPC tests that settings are supported for multi-shard atomic commit.
+func TestExecutorSettingsInTwoPC(t *testing.T) {
+ executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
+ tcases := []struct {
+ sqls []string
+ testRes []*sqltypes.Result
+ expectedQueries [][]string
+ }{
+ {
+ sqls: []string{
+ `set time_zone = "+08:00"`,
+ `insert into user_extra(user_id) values (1)`,
+ `insert into user_extra(user_id) values (2)`,
+ `insert into user_extra(user_id) values (3)`,
+ },
+ testRes: []*sqltypes.Result{
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("id", "varchar"),
+ "+08:00"),
+ },
+ expectedQueries: [][]string{
+ {
+ "select '+08:00' from dual where @@time_zone != '+08:00'",
+ "set @@time_zone = '+08:00'",
+ "set @@time_zone = '+08:00'",
+ "insert into user_extra(user_id) values (1)",
+ "insert into user_extra(user_id) values (2)",
+ },
+ {
+ "set @@time_zone = '+08:00'",
+ "insert into user_extra(user_id) values (3)",
+ },
+ },
+ },
+ }
+
+ for _, tcase := range tcases {
+ t.Run(fmt.Sprintf("%v", tcase.sqls), func(t *testing.T) {
+ sbc1.SetResults(tcase.testRes)
+ sbc2.SetResults(tcase.testRes)
+
+ // create a new session
+ session := econtext.NewSafeSession(&vtgatepb.Session{
+ TargetString: KsTestSharded,
+ TransactionMode: vtgatepb.TransactionMode_TWOPC,
+ EnableSystemSettings: true,
+ })
+
+ // start transaction
+ _, err := executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, "begin", nil)
+ require.NoError(t, err)
+
+ // execute queries
+ for _, sql := range tcase.sqls {
+ _, err = executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, sql, nil)
+ require.NoError(t, err)
+ }
+
+ // commit 2pc
+ _, err = executor.Execute(ctx, nil, "TestExecutorSettingsInTwoPC", session, "commit", nil)
+ require.NoError(t, err)
+
+ queriesRecvd, err := sbc1.GetFinalQueries()
+ require.NoError(t, err)
+ assert.EqualValues(t, tcase.expectedQueries[0], queriesRecvd)
+ queriesRecvd, err = sbc2.GetFinalQueries()
+ require.NoError(t, err)
+ assert.EqualValues(t, tcase.expectedQueries[1], queriesRecvd)
+ })
+ }
+}
+
func TestExecutorTruncateErrors(t *testing.T) {
executor, _, _, _, ctx := createExecutorEnv(t)
@@ -2786,7 +2869,7 @@ func TestExecutorTruncateErrors(t *testing.T) {
truncateErrorLen = 32
defer func() { truncateErrorLen = save }()
- session := NewSafeSession(&vtgatepb.Session{})
+ session := econtext.NewSafeSession(&vtgatepb.Session{})
fn := func(r *sqltypes.Result) error {
return nil
}
@@ -2846,7 +2929,7 @@ func TestExecutorFlushStmt(t *testing.T) {
for _, tc := range tcs {
t.Run(tc.query+tc.targetStr, func(t *testing.T) {
- _, err := executor.Execute(context.Background(), nil, "TestExecutorFlushStmt", NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), tc.query, nil)
+ _, err := executor.Execute(context.Background(), nil, "TestExecutorFlushStmt", econtext.NewSafeSession(&vtgatepb.Session{TargetString: tc.targetStr}), tc.query, nil)
if tc.expectedErr == "" {
require.NoError(t, err)
} else {
@@ -2893,7 +2976,7 @@ func TestExecutorKillStmt(t *testing.T) {
allowKillStmt = !tc.disallow
t.Run("execute:"+tc.query+tc.errStr, func(t *testing.T) {
mysqlCtx := &fakeMysqlConnection{ErrMsg: tc.errStr}
- _, err := executor.Execute(context.Background(), mysqlCtx, "TestExecutorKillStmt", NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil)
+ _, err := executor.Execute(context.Background(), mysqlCtx, "TestExecutorKillStmt", econtext.NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil)
if tc.errStr != "" {
require.ErrorContains(t, err, tc.errStr)
} else {
@@ -2903,7 +2986,7 @@ func TestExecutorKillStmt(t *testing.T) {
})
t.Run("stream:"+tc.query+tc.errStr, func(t *testing.T) {
mysqlCtx := &fakeMysqlConnection{ErrMsg: tc.errStr}
- err := executor.StreamExecute(context.Background(), mysqlCtx, "TestExecutorKillStmt", NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil, func(result *sqltypes.Result) error {
+ err := executor.StreamExecute(context.Background(), mysqlCtx, "TestExecutorKillStmt", econtext.NewAutocommitSession(&vtgatepb.Session{}), tc.query, nil, func(result *sqltypes.Result) error {
return nil
})
if tc.errStr != "" {
@@ -2939,7 +3022,7 @@ func (f *fakeMysqlConnection) KillConnection(ctx context.Context, connID uint32)
var _ vtgateservice.MySQLConnection = (*fakeMysqlConnection)(nil)
-func exec(executor *Executor, session *SafeSession, sql string) (*sqltypes.Result, error) {
+func exec(executor *Executor, session *econtext.SafeSession, sql string) (*sqltypes.Result, error) {
return executor.Execute(context.Background(), nil, "TestExecute", session, sql, nil)
}
diff --git a/go/vt/vtgate/executor_vexplain_test.go b/go/vt/vtgate/executor_vexplain_test.go
new file mode 100644
index 00000000000..a9516492f1b
--- /dev/null
+++ b/go/vt/vtgate/executor_vexplain_test.go
@@ -0,0 +1,172 @@
+/*
+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 vtgate
+
+import (
+ "context"
+ "encoding/json"
+ "fmt"
+ "os"
+ "path/filepath"
+ "testing"
+
+ "github.com/stretchr/testify/assert"
+
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/test/utils"
+ "vitess.io/vitess/go/vt/discovery"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ "vitess.io/vitess/go/vt/vttablet/sandboxconn"
+)
+
+func TestSimpleVexplainTrace(t *testing.T) {
+ ctx := utils.LeakCheckContext(t)
+
+ cell := "aa"
+ hc := discovery.NewFakeHealthCheck(nil)
+ u := createSandbox(KsTestUnsharded)
+ s := createSandbox(KsTestSharded)
+ s.VSchema = executorVSchema
+ u.VSchema = unshardedVSchema
+ serv := newSandboxForCells(ctx, []string{cell})
+ resolver := newTestResolver(ctx, hc, serv, cell)
+ shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"}
+ var conns []*sandboxconn.SandboxConn
+ for i, shard := range shards {
+ sbc := hc.AddTestTablet(cell, shard, 1, "TestExecutor", shard, topodatapb.TabletType_PRIMARY, true, 1, nil)
+ sbc.SetResults([]*sqltypes.Result{{
+ Fields: []*querypb.Field{
+ {Name: "col1", Type: sqltypes.Int32},
+ {Name: "col2", Type: sqltypes.Int32},
+ {Name: "weight_string(col2)"},
+ },
+ InsertID: 0,
+ Rows: [][]sqltypes.Value{
+ {sqltypes.NewInt32(1), sqltypes.NewInt32(int32(i % 4)), sqltypes.NULL},
+ {sqltypes.NewInt32(2), sqltypes.NewInt32(int32(i % 4)), sqltypes.NULL},
+ },
+ }})
+ conns = append(conns, sbc)
+ }
+ executor := createExecutor(ctx, serv, cell, resolver)
+ defer executor.Close()
+
+ query := "vexplain trace select count(*), col2 from music group by col2"
+ session := &vtgatepb.Session{
+ TargetString: "@primary",
+ }
+ gotResult, err := executorExec(ctx, executor, session, query, nil)
+ require.NoError(t, err)
+
+ wantQueries := []*querypb.BoundQuery{{
+ Sql: "select count(*), col2, weight_string(col2) from music group by col2, weight_string(col2) order by col2 asc",
+ BindVariables: map[string]*querypb.BindVariable{},
+ }}
+ for _, conn := range conns {
+ utils.MustMatch(t, wantQueries, conn.Queries)
+ }
+
+ expectedRowString := `{
+ "OperatorType": "Aggregate",
+ "Variant": "Ordered",
+ "NoOfCalls": 1,
+ "AvgNumberOfRows": 4,
+ "MedianNumberOfRows": 4,
+ "Aggregates": "sum_count_star(0) AS count(*)",
+ "GroupBy": "(1|2)",
+ "ResultColumns": 2,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "TestExecutor",
+ "Sharded": true
+ },
+ "NoOfCalls": 1,
+ "AvgNumberOfRows": 16,
+ "MedianNumberOfRows": 16,
+ "ShardsQueried": 8,
+ "FieldQuery": "select count(*), col2, weight_string(col2) from music where 1 != 1 group by col2, weight_string(col2)",
+ "OrderBy": "(1|2) ASC",
+ "Query": "select count(*), col2, weight_string(col2) from music group by col2, weight_string(col2) order by col2 asc",
+ "Table": "music"
+ }
+ ]
+}`
+
+ gotRowString := gotResult.Rows[0][0].ToString()
+ require.Equal(t, expectedRowString, gotRowString)
+}
+
+func TestVExplainKeys(t *testing.T) {
+ type testCase struct {
+ Query string `json:"query"`
+ Expected json.RawMessage `json:"expected"`
+ }
+
+ var tests []testCase
+ data, err := os.ReadFile("testdata/executor_vexplain.json")
+ require.NoError(t, err)
+
+ err = json.Unmarshal(data, &tests)
+ require.NoError(t, err)
+
+ var updatedTests []testCase
+
+ for _, tt := range tests {
+ t.Run(tt.Query, func(t *testing.T) {
+ executor, _, _, _, _ := createExecutorEnv(t)
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary"})
+ gotResult, err := executor.Execute(context.Background(), nil, "Execute", session, "vexplain keys "+tt.Query, nil)
+ require.NoError(t, err)
+
+ gotRowString := gotResult.Rows[0][0].ToString()
+ assert.JSONEq(t, string(tt.Expected), gotRowString)
+
+ updatedTests = append(updatedTests, testCase{
+ Query: tt.Query,
+ Expected: json.RawMessage(gotRowString),
+ })
+
+ if t.Failed() {
+ fmt.Println("Test failed for query:", tt.Query)
+ fmt.Println("Got result:", gotRowString)
+ }
+ })
+ }
+
+ // If anything failed, write the updated test cases to a temp file
+ if t.Failed() {
+ tempFilePath := filepath.Join(os.TempDir(), "updated_vexplain_keys_tests.json")
+ fmt.Println("Writing updated tests to:", tempFilePath)
+
+ updatedTestsData, err := json.MarshalIndent(updatedTests, "", "\t")
+ require.NoError(t, err)
+
+ err = os.WriteFile(tempFilePath, updatedTestsData, 0644)
+ require.NoError(t, err)
+
+ fmt.Println("Updated tests written to:", tempFilePath)
+ }
+}
diff --git a/go/vt/vtgate/executor_vschema_ddl_test.go b/go/vt/vtgate/executor_vschema_ddl_test.go
index 1c912ed0d62..1acc1ba2362 100644
--- a/go/vt/vtgate/executor_vschema_ddl_test.go
+++ b/go/vt/vtgate/executor_vschema_ddl_test.go
@@ -25,6 +25,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/callerid"
@@ -133,12 +135,12 @@ func waitForColVindexes(t *testing.T, ks, table string, names []string, executor
}
func TestPlanExecutorAlterVSchemaKeyspace(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
- session := NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: "@primary", Autocommit: true})
vschemaUpdates := make(chan *vschemapb.SrvVSchema, 2)
executor.serv.WatchSrvVSchema(ctx, "aa", func(vschema *vschemapb.SrvVSchema, err error) bool {
@@ -161,9 +163,9 @@ func TestPlanExecutorAlterVSchemaKeyspace(t *testing.T) {
}
func TestPlanExecutorCreateVindexDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
ks := "TestExecutor"
@@ -180,7 +182,7 @@ func TestPlanExecutorCreateVindexDDL(t *testing.T) {
t.Fatalf("test_vindex should not exist in original vschema")
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema create vindex test_vindex using hash"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.NoError(t, err)
@@ -203,9 +205,9 @@ func TestPlanExecutorCreateVindexDDL(t *testing.T) {
}
func TestPlanExecutorDropVindexDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
ks := "TestExecutor"
@@ -222,7 +224,7 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) {
t.Fatalf("test_vindex should not exist in original vschema")
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema drop vindex test_vindex"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
wantErr := "vindex test_vindex does not exists in keyspace TestExecutor"
@@ -272,9 +274,9 @@ func TestPlanExecutorDropVindexDDL(t *testing.T) {
}
func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
ks := KsTestUnsharded
@@ -296,7 +298,7 @@ func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) {
vschemaTables = append(vschemaTables, t)
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema add table test_table"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.NoError(t, err)
@@ -308,7 +310,7 @@ func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) {
_ = waitForVschemaTables(t, ks, append([]string{"test_table", "test_table2"}, vschemaTables...), executor)
// Should fail adding a table on a sharded keyspace
- session = NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: "TestExecutor"})
stmt = "alter vschema add table test_table"
_, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
wantErr := "add vschema table: unsupported on sharded keyspace TestExecutor"
@@ -329,9 +331,9 @@ func TestPlanExecutorAddDropVschemaTableDDL(t *testing.T) {
}
func TestExecutorAddSequenceDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
ks := KsTestUnsharded
@@ -343,7 +345,7 @@ func TestExecutorAddSequenceDDL(t *testing.T) {
vschemaTables = append(vschemaTables, t)
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema add sequence test_seq"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
require.NoError(t, err)
@@ -357,7 +359,7 @@ func TestExecutorAddSequenceDDL(t *testing.T) {
// Should fail adding a table on a sharded keyspace
ksSharded := "TestExecutor"
- session = NewSafeSession(&vtgatepb.Session{TargetString: ksSharded})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ksSharded})
stmt = "alter vschema add sequence sequence_table"
_, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
@@ -389,9 +391,9 @@ func TestExecutorAddSequenceDDL(t *testing.T) {
}
func TestExecutorDropSequenceDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
ks := KsTestUnsharded
@@ -403,7 +405,7 @@ func TestExecutorDropSequenceDDL(t *testing.T) {
t.Fatalf("test_seq should not exist in original vschema")
}
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
// add test sequence
stmt := "alter vschema add sequence test_seq"
@@ -428,7 +430,7 @@ func TestExecutorDropSequenceDDL(t *testing.T) {
}
// Should fail dropping a non-existing test sequence
- session = NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session = econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt = "alter vschema drop sequence test_seq"
_, err = executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
@@ -440,14 +442,14 @@ func TestExecutorDropSequenceDDL(t *testing.T) {
}
func TestExecutorDropAutoIncDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, _, _, _, ctx := createExecutorEnv(t)
ks := KsTestUnsharded
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
stmt := "alter vschema add table test_table"
_, err := executor.Execute(ctx, nil, "TestExecute", session, stmt, nil)
@@ -482,13 +484,13 @@ func TestExecutorDropAutoIncDDL(t *testing.T) {
}
func TestExecutorAddDropVindexDDL(t *testing.T) {
- vschemaacl.AuthorizedDDLUsers = "%"
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
defer func() {
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}()
executor, sbc1, sbc2, sbclookup, ctx := createExecutorEnv(t)
ks := "TestExecutor"
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
vschemaUpdates := make(chan *vschemapb.SrvVSchema, 4)
executor.serv.WatchSrvVSchema(ctx, "aa", func(vschema *vschemapb.SrvVSchema, err error) bool {
vschemaUpdates <- vschema
@@ -706,7 +708,7 @@ func TestExecutorAddDropVindexDDL(t *testing.T) {
require.EqualError(t, err, "table TestExecutor.nonexistent not defined in vschema")
stmt = "alter vschema on nonexistent drop vindex test_lookup"
- _, err = executor.Execute(ctx, nil, "TestExecute", NewSafeSession(&vtgatepb.Session{TargetString: "InvalidKeyspace"}), stmt, nil)
+ _, err = executor.Execute(ctx, nil, "TestExecute", econtext.NewSafeSession(&vtgatepb.Session{TargetString: "InvalidKeyspace"}), stmt, nil)
require.EqualError(t, err, "VT05003: unknown database 'InvalidKeyspace' in vschema")
stmt = "alter vschema on nowhere.nohow drop vindex test_lookup"
@@ -731,7 +733,7 @@ func TestPlanExecutorVindexDDLACL(t *testing.T) {
// t.Skip("not yet planned")
executor, _, _, _, ctx := createExecutorEnv(t)
ks := "TestExecutor"
- session := NewSafeSession(&vtgatepb.Session{TargetString: ks})
+ session := econtext.NewSafeSession(&vtgatepb.Session{TargetString: ks})
ctxRedUser := callerid.NewContext(ctx, &vtrpcpb.CallerID{}, &querypb.VTGateCallerID{Username: "redUser"})
ctxBlueUser := callerid.NewContext(ctx, &vtrpcpb.CallerID{}, &querypb.VTGateCallerID{Username: "blueUser"})
@@ -745,8 +747,7 @@ func TestPlanExecutorVindexDDLACL(t *testing.T) {
require.EqualError(t, err, `User 'blueUser' is not authorized to perform vschema operations`)
// test when all users are enabled
- vschemaacl.AuthorizedDDLUsers = "%"
- vschemaacl.Init()
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("%"))
_, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil)
if err != nil {
t.Errorf("unexpected error '%v'", err)
@@ -758,8 +759,7 @@ func TestPlanExecutorVindexDDLACL(t *testing.T) {
}
// test when only one user is enabled
- vschemaacl.AuthorizedDDLUsers = "orangeUser, blueUser, greenUser"
- vschemaacl.Init()
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers("orangeUser, blueUser, greenUser"))
_, err = executor.Execute(ctxRedUser, nil, "TestExecute", session, stmt, nil)
require.EqualError(t, err, `User 'redUser' is not authorized to perform vschema operations`)
@@ -770,5 +770,5 @@ func TestPlanExecutorVindexDDLACL(t *testing.T) {
}
// restore the disallowed state
- vschemaacl.AuthorizedDDLUsers = ""
+ vschemaacl.AuthorizedDDLUsers.Set(vschemaacl.NewAuthorizedDDLUsers(""))
}
diff --git a/go/vt/vtgate/executor_vstream_test.go b/go/vt/vtgate/executor_vstream_test.go
index 5466e9e8f3f..22fb7ee1034 100644
--- a/go/vt/vtgate/executor_vstream_test.go
+++ b/go/vt/vtgate/executor_vstream_test.go
@@ -21,6 +21,7 @@ import (
"time"
"vitess.io/vitess/go/vt/vtgate/engine"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -76,7 +77,7 @@ func TestVStreamSQLUnsharded(t *testing.T) {
results := make(chan *sqltypes.Result, 20)
go func() {
- err := executor.StreamExecute(ctx, nil, "TestExecuteStream", NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), sql, nil, func(qr *sqltypes.Result) error {
+ err := executor.StreamExecute(ctx, nil, "TestExecuteStream", econtext.NewAutocommitSession(&vtgatepb.Session{TargetString: KsTestUnsharded}), sql, nil, func(qr *sqltypes.Result) error {
results <- qr
return nil
})
diff --git a/go/vt/vtgate/executorcontext/faketopo.go b/go/vt/vtgate/executorcontext/faketopo.go
new file mode 100644
index 00000000000..f61119dce15
--- /dev/null
+++ b/go/vt/vtgate/executorcontext/faketopo.go
@@ -0,0 +1,68 @@
+/*
+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 executorcontext
+
+import (
+ "context"
+ "encoding/hex"
+
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+ "vitess.io/vitess/go/vt/topo"
+)
+
+type FakeTopoServer struct{}
+
+// GetTopoServer returns the full topo.Server instance.
+func (f *FakeTopoServer) GetTopoServer() (*topo.Server, error) {
+ return nil, nil
+}
+
+// GetSrvKeyspaceNames returns the list of keyspaces served in
+// the provided cell.
+func (f *FakeTopoServer) GetSrvKeyspaceNames(ctx context.Context, cell string, staleOK bool) ([]string, error) {
+ return []string{"ks1"}, nil
+}
+
+// GetSrvKeyspace returns the SrvKeyspace for a cell/keyspace.
+func (f *FakeTopoServer) GetSrvKeyspace(ctx context.Context, cell, keyspace string) (*topodatapb.SrvKeyspace, error) {
+ zeroHexBytes, _ := hex.DecodeString("")
+ eightyHexBytes, _ := hex.DecodeString("80")
+ ks := &topodatapb.SrvKeyspace{
+ Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{
+ {
+ ServedType: topodatapb.TabletType_PRIMARY,
+ ShardReferences: []*topodatapb.ShardReference{
+ {Name: "-80", KeyRange: &topodatapb.KeyRange{Start: zeroHexBytes, End: eightyHexBytes}},
+ {Name: "80-", KeyRange: &topodatapb.KeyRange{Start: eightyHexBytes, End: zeroHexBytes}},
+ },
+ },
+ },
+ }
+ return ks, nil
+}
+
+func (f *FakeTopoServer) WatchSrvKeyspace(ctx context.Context, cell, keyspace string, callback func(*topodatapb.SrvKeyspace, error) bool) {
+ ks, err := f.GetSrvKeyspace(ctx, cell, keyspace)
+ callback(ks, err)
+}
+
+// WatchSrvVSchema starts watching the SrvVSchema object for
+// the provided cell. It will call the callback when
+// a new value or an error occurs.
+func (f *FakeTopoServer) WatchSrvVSchema(ctx context.Context, cell string, callback func(*vschemapb.SrvVSchema, error) bool) {
+}
diff --git a/go/vt/vtgate/safe_session.go b/go/vt/vtgate/executorcontext/safe_session.go
similarity index 77%
rename from go/vt/vtgate/safe_session.go
rename to go/vt/vtgate/executorcontext/safe_session.go
index 45fff46f629..c77bba76ff8 100644
--- a/go/vt/vtgate/safe_session.go
+++ b/go/vt/vtgate/executorcontext/safe_session.go
@@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
limitations under the License.
*/
-package vtgate
+package executorcontext
import (
"fmt"
@@ -26,17 +26,16 @@ import (
"google.golang.org/protobuf/proto"
"vitess.io/vitess/go/mysql/datetime"
-
+ "vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/srvtopo"
"vitess.io/vitess/go/vt/sysvars"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
type (
@@ -56,7 +55,7 @@ type (
rollbackOnPartialExec string
savepointName string
- // this is a signal that found_rows has already been handles by the primitives,
+ // this is a signal that found_rows has already been handled by the primitives,
// and doesn't have to be updated by the executor
foundRowsHandled bool
@@ -64,12 +63,12 @@ type (
// as the query that started a new transaction on the shard belong to a vindex.
queryFromVindex bool
- logging *executeLogger
+ logging *ExecuteLogger
*vtgatepb.Session
}
- executeLogger struct {
+ ExecuteLogger struct {
mu sync.Mutex
entries []engine.ExecuteEntry
lastID int
@@ -125,6 +124,8 @@ const (
savepointRollback
)
+const TxRollback = "Rollback Transaction"
+
// NewSafeSession returns a new SafeSession based on the Session
func NewSafeSession(sessn *vtgatepb.Session) *SafeSession {
if sessn == nil {
@@ -203,6 +204,50 @@ func (session *SafeSession) resetCommonLocked() {
}
}
+// NewAutocommitSession returns a SafeSession based on the original
+// session, but with autocommit enabled.
+func (session *SafeSession) NewAutocommitSession() *SafeSession {
+ ss := NewAutocommitSession(session.Session)
+ ss.logging = session.logging
+ return ss
+}
+
+// IsFoundRowsHandled returns the foundRowsHandled.
+func (session *SafeSession) IsFoundRowsHandled() bool {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ return session.foundRowsHandled
+}
+
+// SetFoundRows set the found rows value.
+func (session *SafeSession) SetFoundRows(value uint64) {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ session.FoundRows = value
+ session.foundRowsHandled = true
+}
+
+// GetRollbackOnPartialExec returns the rollbackOnPartialExec value.
+func (session *SafeSession) GetRollbackOnPartialExec() string {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ return session.rollbackOnPartialExec
+}
+
+// SetQueryFromVindex set the queryFromVindex value.
+func (session *SafeSession) SetQueryFromVindex(value bool) {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ session.queryFromVindex = value
+}
+
+// GetQueryFromVindex returns the queryFromVindex value.
+func (session *SafeSession) GetQueryFromVindex() bool {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ return session.queryFromVindex
+}
+
// SetQueryTimeout sets the query timeout
func (session *SafeSession) SetQueryTimeout(queryTimeout int64) {
session.mu.Lock()
@@ -310,7 +355,7 @@ func (session *SafeSession) SetRollbackCommand() {
if session.savepointState == savepointSet {
session.rollbackOnPartialExec = fmt.Sprintf("rollback to %s", session.savepointName)
} else {
- session.rollbackOnPartialExec = txRollback
+ session.rollbackOnPartialExec = TxRollback
}
session.savepointState = savepointRollbackSet
}
@@ -338,6 +383,18 @@ func (session *SafeSession) SetCommitOrder(co vtgatepb.CommitOrder) {
session.commitOrder = co
}
+// GetCommitOrder returns the commit order.
+func (session *SafeSession) GetCommitOrder() vtgatepb.CommitOrder {
+ session.mu.Lock()
+ defer session.mu.Unlock()
+ return session.commitOrder
+}
+
+// GetLogger returns executor logger.
+func (session *SafeSession) GetLogger() *ExecuteLogger {
+ return session.logging
+}
+
// InTransaction returns true if we are in a transaction
func (session *SafeSession) InTransaction() bool {
session.mu.Lock()
@@ -345,73 +402,88 @@ func (session *SafeSession) InTransaction() bool {
return session.Session.InTransaction
}
-// FindAndChangeSessionIfInSingleTxMode returns the transactionId and tabletAlias, if any, for a session
-// modifies the shard session in a specific case for single mode transaction.
-func (session *SafeSession) FindAndChangeSessionIfInSingleTxMode(keyspace, shard string, tabletType topodatapb.TabletType, txMode vtgatepb.TransactionMode) (int64, int64, *topodatapb.TabletAlias, error) {
+// FindAndChangeSessionIfInSingleTxMode retrieves the ShardSession matching the given keyspace, shard, and tablet type.
+// It performs additional checks and may modify the ShardSession in specific cases for single-mode transactions.
+//
+// Key behavior:
+// 1. Retrieves the appropriate list of sessions (PreSessions, PostSessions, or default ShardSessions) based on the commit order.
+// 2. Identifies a matching session by keyspace, shard, and tablet type.
+// 3. If the session meets specific conditions (e.g., non-vindex-only, single transaction mode), it updates the session state:
+// - Converts a vindex-only session to a standard session if required by the transaction type.
+// - If a multi-shard transaction is detected in Single mode, marks the session for rollback and returns an error.
+//
+// Parameters:
+// - keyspace: The keyspace of the target shard.
+// - shard: The shard name of the target.
+// - tabletType: The type of the tablet for the shard session.
+// - txMode: The transaction mode (e.g., Single, Multi).
+//
+// Returns:
+// - The matching ShardSession, if found and valid for the operation.
+// - An error if a Single-mode transaction attempts to span multiple shards.
+func (session *SafeSession) FindAndChangeSessionIfInSingleTxMode(keyspace, shard string, tabletType topodatapb.TabletType, txMode vtgatepb.TransactionMode) (*vtgatepb.Session_ShardSession, error) {
session.mu.Lock()
defer session.mu.Unlock()
- sessions := session.ShardSessions
+
+ shardSession := session.findSessionLocked(keyspace, shard, tabletType)
+
+ if shardSession == nil {
+ return nil, nil
+ }
+
+ if !shardSession.VindexOnly {
+ return shardSession, nil
+ }
+
+ if err := session.singleModeErrorOnCrossShard(txMode, 0); err != nil {
+ return nil, err
+ }
+
+ // the shard session is now used by non-vindex query as well,
+ // so it is not an exclusive vindex only shard session anymore.
+ shardSession.VindexOnly = false
+ return shardSession, nil
+}
+
+func (session *SafeSession) findSessionLocked(keyspace, shard string, tabletType topodatapb.TabletType) *vtgatepb.Session_ShardSession {
+ // Select the appropriate session list based on the commit order.
+ var sessions []*vtgatepb.Session_ShardSession
switch session.commitOrder {
case vtgatepb.CommitOrder_PRE:
sessions = session.PreSessions
case vtgatepb.CommitOrder_POST:
sessions = session.PostSessions
+ default:
+ sessions = session.ShardSessions
}
+
+ // Find and return the matching shard session.
for _, shardSession := range sessions {
- if keyspace == shardSession.Target.Keyspace && tabletType == shardSession.Target.TabletType && shard == shardSession.Target.Shard {
- if txMode != vtgatepb.TransactionMode_SINGLE || !shardSession.VindexOnly || session.queryFromVindex {
- return shardSession.TransactionId, shardSession.ReservedId, shardSession.TabletAlias, nil
- }
- count := actualNoOfShardSession(session.ShardSessions)
- // If the count of shard session which are non vindex only is greater than 0, then it is a
- if count > 0 {
- session.mustRollback = true
- return 0, 0, nil, vterrors.Errorf(vtrpcpb.Code_ABORTED, "multi-db transaction attempted: %v", session.ShardSessions)
- }
- // the shard session is now used by non-vindex query as well,
- // so it is not an exclusive vindex only shard session anymore.
- shardSession.VindexOnly = false
- return shardSession.TransactionId, shardSession.ReservedId, shardSession.TabletAlias, nil
+ if shardSession.Target.Keyspace == keyspace &&
+ shardSession.Target.Shard == shard &&
+ shardSession.Target.TabletType == tabletType {
+ return shardSession
}
}
- return 0, 0, nil, nil
-}
-
-func addOrUpdate(shardSession *vtgatepb.Session_ShardSession, sessions []*vtgatepb.Session_ShardSession) ([]*vtgatepb.Session_ShardSession, error) {
- appendSession := true
- for i, sess := range sessions {
- targetedAtSameTablet := sess.Target.Keyspace == shardSession.Target.Keyspace &&
- sess.Target.TabletType == shardSession.Target.TabletType &&
- sess.Target.Shard == shardSession.Target.Shard
- if targetedAtSameTablet {
- if !proto.Equal(sess.TabletAlias, shardSession.TabletAlias) {
- errorDetails := fmt.Sprintf("got non-matching aliases (%v vs %v) for the same target (keyspace: %v, tabletType: %v, shard: %v)",
- sess.TabletAlias, shardSession.TabletAlias,
- sess.Target.Keyspace, sess.Target.TabletType, sess.Target.Shard)
- return nil, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, errorDetails)
- }
- // replace the old info with the new one
- sessions[i] = shardSession
- appendSession = false
- break
- }
- }
- if appendSession {
- sessions = append(sessions, shardSession)
- }
+ return nil
+}
- return sessions, nil
+type ShardActionInfo interface {
+ TransactionID() int64
+ ReservedID() int64
+ RowsAffected() bool
+ Alias() *topodatapb.TabletAlias
}
// AppendOrUpdate adds a new ShardSession, or updates an existing one if one already exists for the given shard session
-func (session *SafeSession) AppendOrUpdate(shardSession *vtgatepb.Session_ShardSession, txMode vtgatepb.TransactionMode) error {
+func (session *SafeSession) AppendOrUpdate(target *querypb.Target, info ShardActionInfo, existingSession *vtgatepb.Session_ShardSession, txMode vtgatepb.TransactionMode) error {
session.mu.Lock()
defer session.mu.Unlock()
// additional check of transaction id is required
// as now in autocommit mode there can be session due to reserved connection
// that needs to be stored as shard session.
- if session.autocommitState == autocommitted && shardSession.TransactionId != 0 {
+ if session.autocommitState == autocommitted && info.TransactionID() != 0 {
// Should be unreachable
return vterrors.VT13001("unexpected 'autocommitted' state in transaction")
}
@@ -421,45 +493,62 @@ func (session *SafeSession) AppendOrUpdate(shardSession *vtgatepb.Session_ShardS
}
session.autocommitState = notAutocommittable
- // Always append, in order for rollback to succeed.
- switch session.commitOrder {
- case vtgatepb.CommitOrder_NORMAL:
- if session.queryFromVindex {
- shardSession.VindexOnly = true
+ if existingSession != nil {
+ existingSession.TransactionId = info.TransactionID()
+ existingSession.ReservedId = info.ReservedID()
+ if !existingSession.RowsAffected {
+ existingSession.RowsAffected = info.RowsAffected()
}
- newSessions, err := addOrUpdate(shardSession, session.ShardSessions)
- if err != nil {
+ if existingSession.VindexOnly {
+ existingSession.VindexOnly = session.queryFromVindex
+ }
+ if err := session.singleModeErrorOnCrossShard(txMode, 1); err != nil {
return err
}
- session.ShardSessions = newSessions
+ return nil
+ }
+ newSession := &vtgatepb.Session_ShardSession{
+ Target: target,
+ TabletAlias: info.Alias(),
+ TransactionId: info.TransactionID(),
+ ReservedId: info.ReservedID(),
+ RowsAffected: info.RowsAffected(),
+ VindexOnly: session.queryFromVindex,
+ }
- if session.queryFromVindex {
- break
- }
- // isSingle is enforced only for normal commit order operations.
- if session.isSingleDB(txMode) && len(session.ShardSessions) > 1 {
- count := actualNoOfShardSession(session.ShardSessions)
- if count <= 1 {
- break
- }
- session.mustRollback = true
- return vterrors.Errorf(vtrpcpb.Code_ABORTED, "multi-db transaction attempted: %v", session.ShardSessions)
- }
- case vtgatepb.CommitOrder_PRE:
- newSessions, err := addOrUpdate(shardSession, session.PreSessions)
- if err != nil {
+ // Always append, in order for rollback to succeed.
+ switch session.commitOrder {
+ case vtgatepb.CommitOrder_NORMAL:
+ session.ShardSessions = append(session.ShardSessions, newSession)
+ if err := session.singleModeErrorOnCrossShard(txMode, 1); err != nil {
return err
}
- session.PreSessions = newSessions
+ case vtgatepb.CommitOrder_PRE:
+ session.PreSessions = append(session.PreSessions, newSession)
case vtgatepb.CommitOrder_POST:
- newSessions, err := addOrUpdate(shardSession, session.PostSessions)
- if err != nil {
- return err
- }
- session.PostSessions = newSessions
+ session.PostSessions = append(session.PostSessions, newSession)
default:
// Should be unreachable
- return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] SafeSession.AppendOrUpdate: unexpected commitOrder")
+ return vterrors.VT13001(fmt.Sprintf("unexpected commitOrder to append shard session: %v", session.commitOrder))
+ }
+
+ return nil
+}
+
+// singleModeErrorOnCrossShard checks if a transaction violates the Single mode constraint by spanning multiple shards.
+func (session *SafeSession) singleModeErrorOnCrossShard(txMode vtgatepb.TransactionMode, exceedsCrossShard int) error {
+ // Skip the check if:
+ // 1. The query comes from a lookup vindex.
+ // 2. The transaction mode is not Single.
+ // 3. The transaction is not in the normal shard session.
+ if session.queryFromVindex || session.commitOrder != vtgatepb.CommitOrder_NORMAL || !session.isSingleDB(txMode) {
+ return nil
+ }
+
+ // If the transaction spans multiple shards, abort it.
+ if actualNoOfShardSession(session.ShardSessions) > exceedsCrossShard {
+ session.mustRollback = true // Mark the session for rollback.
+ return vterrors.Errorf(vtrpcpb.Code_ABORTED, "multi-db transaction attempted: %v", session.ShardSessions)
}
return nil
@@ -563,13 +652,21 @@ func (session *SafeSession) HasSystemVariables() (found bool) {
func (session *SafeSession) TimeZone() *time.Location {
session.mu.Lock()
- tz, ok := session.SystemVariables["time_zone"]
+ zoneSQL, ok := session.SystemVariables["time_zone"]
session.mu.Unlock()
if !ok {
return nil
}
+
+ tz, err := sqltypes.DecodeStringSQL(zoneSQL)
+ if err != nil {
+ return nil
+ }
+
loc, _ := datetime.ParseTimeZone(tz)
+ // it's safe to ignore the error - if we get an error, loc will be nil,
+ // and this is exactly the behaviour we want anyway
return loc
}
@@ -668,12 +765,11 @@ func (session *SafeSession) UpdateLockHeartbeat() {
session.LastLockHeartbeat = time.Now().Unix()
}
-// TriggerLockHeartBeat returns if it time to trigger next lock heartbeat
-func (session *SafeSession) TriggerLockHeartBeat() bool {
+// GetLockHeartbeat returns last time the lock heartbeat was sent.
+func (session *SafeSession) GetLockHeartbeat() int64 {
session.mu.Lock()
defer session.mu.Unlock()
- now := time.Now().Unix()
- return now-session.LastLockHeartbeat >= int64(lockHeartbeatTime.Seconds())
+ return session.LastLockHeartbeat
}
// InLockSession returns whether locking is used on this session.
@@ -826,9 +922,7 @@ func (session *SafeSession) GetOrCreateOptions() *querypb.ExecuteOptions {
return session.Session.Options
}
-var _ iQueryOption = (*SafeSession)(nil)
-
-func (session *SafeSession) cachePlan() bool {
+func (session *SafeSession) CachePlan() bool {
if session == nil || session.Options == nil {
return true
}
@@ -839,7 +933,7 @@ func (session *SafeSession) cachePlan() bool {
return !(session.Options.SkipQueryPlanCache || session.Options.HasCreatedTempTables)
}
-func (session *SafeSession) getSelectLimit() int {
+func (session *SafeSession) GetSelectLimit() int {
if session == nil || session.Options == nil {
return -1
}
@@ -850,16 +944,16 @@ func (session *SafeSession) getSelectLimit() int {
return int(session.Options.SqlSelectLimit)
}
-// isTxOpen returns true if there is open connection to any of the shard.
-func (session *SafeSession) isTxOpen() bool {
+// IsTxOpen returns true if there is open connection to any of the shard.
+func (session *SafeSession) IsTxOpen() bool {
session.mu.Lock()
defer session.mu.Unlock()
return len(session.ShardSessions) > 0 || len(session.PreSessions) > 0 || len(session.PostSessions) > 0
}
-// getSessions returns the shard session for the current commit order.
-func (session *SafeSession) getSessions() []*vtgatepb.Session_ShardSession {
+// GetSessions returns the shard session for the current commit order.
+func (session *SafeSession) GetSessions() []*vtgatepb.Session_ShardSession {
session.mu.Lock()
defer session.mu.Unlock()
@@ -946,7 +1040,7 @@ func (session *SafeSession) EnableLogging(parser *sqlparser.Parser) {
session.mu.Lock()
defer session.mu.Unlock()
- session.logging = &executeLogger{
+ session.logging = &ExecuteLogger{
parser: parser,
}
}
@@ -984,7 +1078,15 @@ func (session *SafeSession) GetPrepareData(name string) *vtgatepb.PrepareData {
return session.PrepareStatement[name]
}
-func (l *executeLogger) log(primitive engine.Primitive, target *querypb.Target, gateway srvtopo.Gateway, query string, begin bool, bv map[string]*querypb.BindVariable) {
+func (session *SafeSession) Log(primitive engine.Primitive, target *querypb.Target, gateway srvtopo.Gateway, query string, begin bool, bv map[string]*querypb.BindVariable) {
+ session.logging.Log(primitive, target, gateway, query, begin, bv)
+}
+
+func (session *SafeSession) GetLogs() []engine.ExecuteEntry {
+ return session.logging.GetLogs()
+}
+
+func (l *ExecuteLogger) Log(primitive engine.Primitive, target *querypb.Target, gateway srvtopo.Gateway, query string, begin bool, bv map[string]*querypb.BindVariable) {
if l == nil {
return
}
@@ -1023,7 +1125,10 @@ func (l *executeLogger) log(primitive engine.Primitive, target *querypb.Target,
})
}
-func (l *executeLogger) GetLogs() []engine.ExecuteEntry {
+func (l *ExecuteLogger) GetLogs() []engine.ExecuteEntry {
+ if l == nil {
+ return nil
+ }
l.mu.Lock()
defer l.mu.Unlock()
result := make([]engine.ExecuteEntry, len(l.entries))
diff --git a/go/vt/vtgate/executorcontext/safe_session_test.go b/go/vt/vtgate/executorcontext/safe_session_test.go
new file mode 100644
index 00000000000..14ea2ad9dac
--- /dev/null
+++ b/go/vt/vtgate/executorcontext/safe_session_test.go
@@ -0,0 +1,201 @@
+/*
+Copyright 2020 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 executorcontext
+
+import (
+ "reflect"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+)
+
+type fakeInfo struct {
+ transactionID int64
+ alias *topodatapb.TabletAlias
+}
+
+func (s *fakeInfo) TransactionID() int64 {
+ return s.transactionID
+}
+
+func (s *fakeInfo) ReservedID() int64 {
+ return 0
+}
+
+func (s *fakeInfo) RowsAffected() bool {
+ return false
+}
+
+func (s *fakeInfo) Alias() *topodatapb.TabletAlias {
+ return s.alias
+}
+
+func info(txId, uid int) ShardActionInfo {
+ return &fakeInfo{transactionID: int64(txId), alias: &topodatapb.TabletAlias{Cell: "cell", Uid: uint32(uid)}}
+}
+
+// TestFailToMultiShardWhenSetToSingleDb tests that single db transactions fails on going multi shard.
+func TestFailToMultiShardWhenSetToSingleDb(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{
+ InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE,
+ })
+
+ err := session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "0"},
+ info(1, 0),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+ err = session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "1"},
+ info(1, 1),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.Error(t, err)
+}
+
+// TestSingleDbUpdateToMultiShard tests that a single db transaction cannot be updated to multi shard.
+func TestSingleDbUpdateToMultiShard(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{
+ InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE,
+ })
+
+ // shard session s0 due to a vindex query
+ session.queryFromVindex = true
+ err := session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "0"},
+ info(1, 0),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+ session.queryFromVindex = false
+
+ // shard session s1
+ err = session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "1"},
+ info(1, 1),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+
+ // shard session s0 with normal query
+ err = session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "0"},
+ info(1, 1),
+ session.ShardSessions[0],
+ vtgatepb.TransactionMode_SINGLE)
+ require.Error(t, err)
+}
+
+// TestSingleDbPreFailOnFind tests that finding a shard session fails
+// if already shard session exists on another shard and the query is not from vindex.
+func TestSingleDbPreFailOnFind(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{
+ InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE,
+ })
+
+ // shard session s0 due to a vindex query
+ session.queryFromVindex = true
+ err := session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "0"},
+ info(1, 0),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+ session.queryFromVindex = false
+
+ // shard session s1
+ err = session.AppendOrUpdate(
+ &querypb.Target{Keyspace: "keyspace", Shard: "1"},
+ info(1, 1),
+ nil,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+
+ // shard session s1 for normal query again - should not fail as already part of the session.
+ ss, err := session.FindAndChangeSessionIfInSingleTxMode(
+ "keyspace",
+ "1",
+ topodatapb.TabletType_UNKNOWN,
+ vtgatepb.TransactionMode_SINGLE)
+ require.NoError(t, err)
+ require.NotNil(t, ss)
+ require.False(t, ss.VindexOnly)
+ require.EqualValues(t, 1, ss.TabletAlias.Uid)
+
+ // shard session s0 for normal query
+ _, err = session.FindAndChangeSessionIfInSingleTxMode(
+ "keyspace",
+ "0",
+ topodatapb.TabletType_UNKNOWN,
+ vtgatepb.TransactionMode_SINGLE)
+ require.Error(t, err)
+}
+
+func TestPrequeries(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{
+ SystemVariables: map[string]string{
+ "s1": "'apa'",
+ "s2": "42",
+ },
+ })
+
+ want := []string{"set s1 = 'apa', s2 = 42"}
+ preQueries := session.SetPreQueries()
+
+ if !reflect.DeepEqual(want, preQueries) {
+ t.Errorf("got %v but wanted %v", preQueries, want)
+ }
+}
+
+func TestTimeZone(t *testing.T) {
+ testCases := []struct {
+ tz string
+ want string
+ }{
+ {
+ tz: "'Europe/Amsterdam'",
+ want: "Europe/Amsterdam",
+ },
+ {
+ tz: "'+02:00'",
+ want: "UTC+02:00",
+ },
+ {
+ tz: "foo",
+ want: (*time.Location)(nil).String(),
+ },
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.tz, func(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{
+ SystemVariables: map[string]string{
+ "time_zone": tc.tz,
+ },
+ })
+
+ assert.Equal(t, tc.want, session.TimeZone().String())
+ })
+ }
+}
diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/executorcontext/vcursor_impl.go
similarity index 53%
rename from go/vt/vtgate/vcursor_impl.go
rename to go/vt/vtgate/executorcontext/vcursor_impl.go
index ee000abed8f..c1f341b38cf 100644
--- a/go/vt/vtgate/vcursor_impl.go
+++ b/go/vt/vtgate/executorcontext/vcursor_impl.go
@@ -14,7 +14,7 @@ See the License for the specific language governing permissions and
limitations under the License.
*/
-package vtgate
+package executorcontext
import (
"context"
@@ -26,10 +26,12 @@ import (
"time"
"github.com/google/uuid"
+ "golang.org/x/exp/maps"
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/mysql/config"
"vitess.io/vitess/go/mysql/sqlerror"
+ "vitess.io/vitess/go/protoutil"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/discovery"
@@ -58,92 +60,124 @@ import (
"vitess.io/vitess/go/vt/vtgate/vtgateservice"
)
-var _ engine.VCursor = (*vcursorImpl)(nil)
-var _ plancontext.VSchema = (*vcursorImpl)(nil)
-var _ iExecute = (*Executor)(nil)
-var _ vindexes.VCursor = (*vcursorImpl)(nil)
-
-// vcursor_impl needs these facilities to be able to be able to execute queries for vindexes
-type iExecute interface {
- Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable) (*sqltypes.Result, error)
- ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *SafeSession, autocommit bool, ignoreMaxMemoryRows bool) (qr *sqltypes.Result, errs []error)
- StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error) []error
- ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error)
- Commit(ctx context.Context, safeSession *SafeSession) error
- ExecuteMessageStream(ctx context.Context, rss []*srvtopo.ResolvedShard, name string, callback func(*sqltypes.Result) error) error
- ExecuteVStream(ctx context.Context, rss []*srvtopo.ResolvedShard, filter *binlogdatapb.Filter, gtid string, callback func(evs []*binlogdatapb.VEvent) error) error
- ReleaseLock(ctx context.Context, session *SafeSession) error
-
- showVitessReplicationStatus(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
- showShards(ctx context.Context, filter *sqlparser.ShowFilter, destTabletType topodatapb.TabletType) (*sqltypes.Result, error)
- showTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
- showVitessMetadata(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
- setVitessMetadata(ctx context.Context, name, value string) error
-
- // TODO: remove when resolver is gone
- ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error)
- VSchema() *vindexes.VSchema
- planPrepareStmt(ctx context.Context, vcursor *vcursorImpl, query string) (*engine.Plan, sqlparser.Statement, error)
-
- environment() *vtenv.Environment
- ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error)
-}
-
-// VSchemaOperator is an interface to Vschema Operations
-type VSchemaOperator interface {
- GetCurrentSrvVschema() *vschemapb.SrvVSchema
- UpdateVSchema(ctx context.Context, ksName string, vschema *vschemapb.SrvVSchema) error
-}
-
-// vcursorImpl implements the VCursor functionality used by dependent
-// packages to call back into VTGate.
-type vcursorImpl struct {
- safeSession *SafeSession
- keyspace string
- tabletType topodatapb.TabletType
- destination key.Destination
- marginComments sqlparser.MarginComments
- executor iExecute
- resolver *srvtopo.Resolver
- topoServer *topo.Server
- logStats *logstats.LogStats
- collation collations.ID
-
- // fkChecksState stores the state of foreign key checks variable.
- // This state is meant to be the final fk checks state after consulting the
- // session state, and the given query's comments for `SET_VAR` optimizer hints.
- // A nil value represents that no foreign_key_checks value was provided.
- fkChecksState *bool
- ignoreMaxMemoryRows bool
- vschema *vindexes.VSchema
- vm VSchemaOperator
- semTable *semantics.SemTable
- warnShardedOnly bool // when using sharded only features, a warning will be warnings field
-
- warnings []*querypb.QueryWarning // any warnings that are accumulated during the planning phase are stored here
- pv plancontext.PlannerVersion
-
- warmingReadsPercent int
- warmingReadsChannel chan bool
-}
-
-// newVcursorImpl creates a vcursorImpl. Before creating this object, you have to separate out any marginComments that came with
+var (
+ _ engine.VCursor = (*VCursorImpl)(nil)
+ _ plancontext.VSchema = (*VCursorImpl)(nil)
+ _ vindexes.VCursor = (*VCursorImpl)(nil)
+)
+
+var ErrNoKeyspace = vterrors.VT09005()
+
+type (
+ ResultsObserver interface {
+ Observe(*sqltypes.Result)
+ }
+
+ VCursorConfig struct {
+ Collation collations.ID
+
+ MaxMemoryRows int
+ EnableShardRouting bool
+ DefaultTabletType topodatapb.TabletType
+ QueryTimeout int
+ DBDDLPlugin string
+ ForeignKeyMode vschemapb.Keyspace_ForeignKeyMode
+ SetVarEnabled bool
+ EnableViews bool
+ WarnShardedOnly bool
+ PlannerVersion plancontext.PlannerVersion
+
+ WarmingReadsPercent int
+ WarmingReadsTimeout time.Duration
+ WarmingReadsChannel chan bool
+ }
+
+ // vcursor_impl needs these facilities to be able to be able to execute queries for vindexes
+ iExecute interface {
+ Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable) (*sqltypes.Result, error)
+ ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *SafeSession, autocommit bool, ignoreMaxMemoryRows bool, resultsObserver ResultsObserver) (qr *sqltypes.Result, errs []error)
+ StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error, observer ResultsObserver) []error
+ ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error)
+ Commit(ctx context.Context, safeSession *SafeSession) error
+ ExecuteMessageStream(ctx context.Context, rss []*srvtopo.ResolvedShard, name string, callback func(*sqltypes.Result) error) error
+ ExecuteVStream(ctx context.Context, rss []*srvtopo.ResolvedShard, filter *binlogdatapb.Filter, gtid string, callback func(evs []*binlogdatapb.VEvent) error) error
+ ReleaseLock(ctx context.Context, session *SafeSession) error
+
+ ShowVitessReplicationStatus(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
+ ShowShards(ctx context.Context, filter *sqlparser.ShowFilter, destTabletType topodatapb.TabletType) (*sqltypes.Result, error)
+ ShowTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
+ ShowVitessMetadata(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error)
+ SetVitessMetadata(ctx context.Context, name, value string) error
+
+ // TODO: remove when resolver is gone
+ VSchema() *vindexes.VSchema
+ PlanPrepareStmt(ctx context.Context, vcursor *VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error)
+
+ Environment() *vtenv.Environment
+ ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error)
+ UnresolvedTransactions(ctx context.Context, targets []*querypb.Target) ([]*querypb.TransactionMetadata, error)
+ AddWarningCount(name string, value int64)
+ }
+
+ // VSchemaOperator is an interface to Vschema Operations
+ VSchemaOperator interface {
+ GetCurrentSrvVschema() *vschemapb.SrvVSchema
+ UpdateVSchema(ctx context.Context, ksName string, vschema *vschemapb.SrvVSchema) error
+ }
+
+ // VCursorImpl implements the VCursor functionality used by dependent
+ // packages to call back into VTGate.
+ VCursorImpl struct {
+ config VCursorConfig
+ SafeSession *SafeSession
+ keyspace string
+ tabletType topodatapb.TabletType
+ destination key.Destination
+ marginComments sqlparser.MarginComments
+ executor iExecute
+ resolver *srvtopo.Resolver
+ topoServer *topo.Server
+ logStats *logstats.LogStats
+
+ // fkChecksState stores the state of foreign key checks variable.
+ // This state is meant to be the final fk checks state after consulting the
+ // session state, and the given query's comments for `SET_VAR` optimizer hints.
+ // A nil value represents that no foreign_key_checks value was provided.
+ fkChecksState *bool
+ ignoreMaxMemoryRows bool
+ vschema *vindexes.VSchema
+ vm VSchemaOperator
+ semTable *semantics.SemTable
+ queryTimeout time.Duration
+
+ warnings []*querypb.QueryWarning // any warnings that are accumulated during the planning phase are stored here
+
+ observer ResultsObserver
+
+ // this is a map of the number of rows that every primitive has returned
+ // if this field is nil, it means that we are not logging operator traffic
+ interOpStats map[engine.Primitive]engine.RowsReceived
+ shardsStats map[engine.Primitive]engine.ShardsQueried
+ }
+)
+
+// NewVCursorImpl creates a VCursorImpl. Before creating this object, you have to separate out any marginComments that came with
// the query and supply it here. Trailing comments are typically sent by the application for various reasons,
// including as identifying markers. So, they have to be added back to all queries that are executed
// on behalf of the original query.
-func newVCursorImpl(
+func NewVCursorImpl(
safeSession *SafeSession,
marginComments sqlparser.MarginComments,
- executor *Executor,
+ executor iExecute,
logStats *logstats.LogStats,
vm VSchemaOperator,
vschema *vindexes.VSchema,
resolver *srvtopo.Resolver,
serv srvtopo.Server,
- warnShardedOnly bool,
- pv plancontext.PlannerVersion,
-) (*vcursorImpl, error) {
- keyspace, tabletType, destination, err := parseDestinationTarget(safeSession.TargetString, vschema)
+ observer ResultsObserver,
+ cfg VCursorConfig,
+) (*VCursorImpl, error) {
+ keyspace, tabletType, destination, err := ParseDestinationTarget(safeSession.TargetString, cfg.DefaultTabletType, vschema)
if err != nil {
return nil, err
}
@@ -158,107 +192,204 @@ func newVCursorImpl(
}
}
- // we only support collations for the new TabletGateway implementation
- var connCollation collations.ID
- if executor != nil {
- if gw, isTabletGw := executor.resolver.resolver.GetGateway().(*TabletGateway); isTabletGw {
- connCollation = gw.DefaultConnCollation()
- }
- }
- if connCollation == collations.Unknown {
- connCollation = executor.env.CollationEnv().DefaultConnectionCharset()
- }
-
- warmingReadsPct := 0
- var warmingReadsChan chan bool
- if executor != nil {
- warmingReadsPct = executor.warmingReadsPercent
- warmingReadsChan = executor.warmingReadsChannel
- }
- return &vcursorImpl{
- safeSession: safeSession,
- keyspace: keyspace,
- tabletType: tabletType,
- destination: destination,
- marginComments: marginComments,
- executor: executor,
- logStats: logStats,
- collation: connCollation,
- resolver: resolver,
- vschema: vschema,
- vm: vm,
- topoServer: ts,
- warnShardedOnly: warnShardedOnly,
- pv: pv,
- warmingReadsPercent: warmingReadsPct,
- warmingReadsChannel: warmingReadsChan,
+ return &VCursorImpl{
+ config: cfg,
+ SafeSession: safeSession,
+ keyspace: keyspace,
+ tabletType: tabletType,
+ destination: destination,
+ marginComments: marginComments,
+ executor: executor,
+ logStats: logStats,
+ resolver: resolver,
+ vschema: vschema,
+ vm: vm,
+ topoServer: ts,
+
+ observer: observer,
}, nil
}
+func (vc *VCursorImpl) CloneForMirroring(ctx context.Context) engine.VCursor {
+ callerId := callerid.EffectiveCallerIDFromContext(ctx)
+ immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx)
+
+ clonedCtx := callerid.NewContext(ctx, callerId, immediateCallerId)
+
+ v := &VCursorImpl{
+ config: vc.config,
+ SafeSession: NewAutocommitSession(vc.SafeSession.Session),
+ keyspace: vc.keyspace,
+ tabletType: vc.tabletType,
+ destination: vc.destination,
+ marginComments: vc.marginComments,
+ executor: vc.executor,
+ resolver: vc.resolver,
+ topoServer: vc.topoServer,
+ logStats: &logstats.LogStats{Ctx: clonedCtx},
+ ignoreMaxMemoryRows: vc.ignoreMaxMemoryRows,
+ vschema: vc.vschema,
+ vm: vc.vm,
+ semTable: vc.semTable,
+ warnings: vc.warnings,
+ observer: vc.observer,
+ }
+
+ v.marginComments.Trailing += "/* mirror query */"
+
+ return v
+}
+
+func (vc *VCursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor {
+ callerId := callerid.EffectiveCallerIDFromContext(ctx)
+ immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx)
+
+ timedCtx, _ := context.WithTimeout(context.Background(), vc.config.WarmingReadsTimeout) // nolint
+ clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId)
+
+ v := &VCursorImpl{
+ config: vc.config,
+ SafeSession: NewAutocommitSession(vc.SafeSession.Session),
+ keyspace: vc.keyspace,
+ tabletType: topodatapb.TabletType_REPLICA,
+ destination: vc.destination,
+ marginComments: vc.marginComments,
+ executor: vc.executor,
+ resolver: vc.resolver,
+ topoServer: vc.topoServer,
+ logStats: &logstats.LogStats{Ctx: clonedCtx},
+
+ ignoreMaxMemoryRows: vc.ignoreMaxMemoryRows,
+ vschema: vc.vschema,
+ vm: vc.vm,
+ semTable: vc.semTable,
+ warnings: vc.warnings,
+ observer: vc.observer,
+ }
+
+ v.marginComments.Trailing += "/* warming read */"
+
+ return v
+}
+
+func (vc *VCursorImpl) cloneWithAutocommitSession() *VCursorImpl {
+ safeSession := vc.SafeSession.NewAutocommitSession()
+ return &VCursorImpl{
+ config: vc.config,
+ SafeSession: safeSession,
+ keyspace: vc.keyspace,
+ tabletType: vc.tabletType,
+ destination: vc.destination,
+ marginComments: vc.marginComments,
+ executor: vc.executor,
+ logStats: vc.logStats,
+ resolver: vc.resolver,
+ vschema: vc.vschema,
+ vm: vc.vm,
+ topoServer: vc.topoServer,
+ observer: vc.observer,
+ }
+}
+
// HasSystemVariables returns whether the session has set system variables or not
-func (vc *vcursorImpl) HasSystemVariables() bool {
- return vc.safeSession.HasSystemVariables()
+func (vc *VCursorImpl) HasSystemVariables() bool {
+ return vc.SafeSession.HasSystemVariables()
}
// GetSystemVariables takes a visitor function that will save each system variables of the session
-func (vc *vcursorImpl) GetSystemVariables(f func(k string, v string)) {
- vc.safeSession.GetSystemVariables(f)
+func (vc *VCursorImpl) GetSystemVariables(f func(k string, v string)) {
+ vc.SafeSession.GetSystemVariables(f)
+}
+
+// GetSystemVariablesCopy returns a copy of the system variables of the session. Changes to the original map will not affect the session.
+func (vc *VCursorImpl) GetSystemVariablesCopy() map[string]string {
+ vc.SafeSession.mu.Lock()
+ defer vc.SafeSession.mu.Unlock()
+ return maps.Clone(vc.SafeSession.SystemVariables)
}
// ConnCollation returns the collation of this session
-func (vc *vcursorImpl) ConnCollation() collations.ID {
- return vc.collation
+func (vc *VCursorImpl) ConnCollation() collations.ID {
+ return vc.config.Collation
}
// Environment returns the vtenv associated with this session
-func (vc *vcursorImpl) Environment() *vtenv.Environment {
- return vc.executor.environment()
+func (vc *VCursorImpl) Environment() *vtenv.Environment {
+ return vc.executor.Environment()
}
-func (vc *vcursorImpl) TimeZone() *time.Location {
- return vc.safeSession.TimeZone()
+func (vc *VCursorImpl) TimeZone() *time.Location {
+ return vc.SafeSession.TimeZone()
}
-func (vc *vcursorImpl) SQLMode() string {
+func (vc *VCursorImpl) SQLMode() string {
// TODO: Implement return the current sql_mode.
// This is currently hardcoded to the default in MySQL 8.0.
return config.DefaultSQLMode
}
// MaxMemoryRows returns the maxMemoryRows flag value.
-func (vc *vcursorImpl) MaxMemoryRows() int {
- return maxMemoryRows
+func (vc *VCursorImpl) MaxMemoryRows() int {
+ return vc.config.MaxMemoryRows
}
// ExceedsMaxMemoryRows returns a boolean indicating whether the maxMemoryRows value has been exceeded.
// Returns false if the max memory rows override directive is set to true.
-func (vc *vcursorImpl) ExceedsMaxMemoryRows(numRows int) bool {
- return !vc.ignoreMaxMemoryRows && numRows > maxMemoryRows
+func (vc *VCursorImpl) ExceedsMaxMemoryRows(numRows int) bool {
+ return !vc.ignoreMaxMemoryRows && numRows > vc.config.MaxMemoryRows
}
// SetIgnoreMaxMemoryRows sets the ignoreMaxMemoryRows value.
-func (vc *vcursorImpl) SetIgnoreMaxMemoryRows(ignoreMaxMemoryRows bool) {
+func (vc *VCursorImpl) SetIgnoreMaxMemoryRows(ignoreMaxMemoryRows bool) {
vc.ignoreMaxMemoryRows = ignoreMaxMemoryRows
}
// RecordWarning stores the given warning in the current session
-func (vc *vcursorImpl) RecordWarning(warning *querypb.QueryWarning) {
- vc.safeSession.RecordWarning(warning)
+func (vc *VCursorImpl) RecordWarning(warning *querypb.QueryWarning) {
+ vc.SafeSession.RecordWarning(warning)
}
// IsShardRoutingEnabled implements the VCursor interface.
-func (vc *vcursorImpl) IsShardRoutingEnabled() bool {
- return enableShardRouting
+func (vc *VCursorImpl) IsShardRoutingEnabled() bool {
+ return vc.config.EnableShardRouting
}
-func (vc *vcursorImpl) ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error) {
+func (vc *VCursorImpl) ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error) {
return vc.executor.ReadTransaction(ctx, transactionID)
}
+// UnresolvedTransactions gets the unresolved transactions for the given keyspace. If the keyspace is not given,
+// then we use the default keyspace.
+func (vc *VCursorImpl) UnresolvedTransactions(ctx context.Context, keyspace string) ([]*querypb.TransactionMetadata, error) {
+ if keyspace == "" {
+ keyspace = vc.GetKeyspace()
+ }
+ rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.Destination{key.DestinationAllShards{}})
+ if err != nil {
+ return nil, err
+ }
+ var targets []*querypb.Target
+ for _, rs := range rss {
+ targets = append(targets, rs.Target)
+ }
+ return vc.executor.UnresolvedTransactions(ctx, targets)
+}
+
+func (vc *VCursorImpl) StartPrimitiveTrace() func() engine.Stats {
+ vc.interOpStats = make(map[engine.Primitive]engine.RowsReceived)
+ vc.shardsStats = make(map[engine.Primitive]engine.ShardsQueried)
+ return func() engine.Stats {
+ return engine.Stats{
+ InterOpStats: vc.interOpStats,
+ ShardsStats: vc.shardsStats,
+ }
+ }
+}
+
// FindTable finds the specified table. If the keyspace what specified in the input, it gets used as qualifier.
// Otherwise, the keyspace from the request is used, if one was provided.
-func (vc *vcursorImpl) FindTable(name sqlparser.TableName) (*vindexes.Table, string, topodatapb.TabletType, key.Destination, error) {
- destKeyspace, destTabletType, dest, err := vc.executor.ParseDestinationTarget(name.Qualifier.String())
+func (vc *VCursorImpl) FindTable(name sqlparser.TableName) (*vindexes.Table, string, topodatapb.TabletType, key.Destination, error) {
+ destKeyspace, destTabletType, dest, err := vc.ParseDestinationTarget(name.Qualifier.String())
if err != nil {
return nil, "", destTabletType, nil, err
}
@@ -272,8 +403,8 @@ func (vc *vcursorImpl) FindTable(name sqlparser.TableName) (*vindexes.Table, str
return table, destKeyspace, destTabletType, dest, err
}
-func (vc *vcursorImpl) FindView(name sqlparser.TableName) sqlparser.SelectStatement {
- ks, _, _, err := vc.executor.ParseDestinationTarget(name.Qualifier.String())
+func (vc *VCursorImpl) FindView(name sqlparser.TableName) sqlparser.SelectStatement {
+ ks, _, _, err := vc.ParseDestinationTarget(name.Qualifier.String())
if err != nil {
return nil
}
@@ -283,8 +414,8 @@ func (vc *vcursorImpl) FindView(name sqlparser.TableName) sqlparser.SelectStatem
return vc.vschema.FindView(ks, name.Name.String())
}
-func (vc *vcursorImpl) FindRoutedTable(name sqlparser.TableName) (*vindexes.Table, error) {
- destKeyspace, destTabletType, _, err := vc.executor.ParseDestinationTarget(name.Qualifier.String())
+func (vc *VCursorImpl) FindRoutedTable(name sqlparser.TableName) (*vindexes.Table, error) {
+ destKeyspace, destTabletType, _, err := vc.ParseDestinationTarget(name.Qualifier.String())
if err != nil {
return nil, err
}
@@ -301,14 +432,14 @@ func (vc *vcursorImpl) FindRoutedTable(name sqlparser.TableName) (*vindexes.Tabl
}
// FindTableOrVindex finds the specified table or vindex.
-func (vc *vcursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) {
+func (vc *VCursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) {
if name.Qualifier.IsEmpty() && name.Name.String() == "dual" {
// The magical MySQL dual table should only be resolved
// when it is not qualified by a database name.
return vc.getDualTable()
}
- destKeyspace, destTabletType, dest, err := vc.executor.ParseDestinationTarget(name.Qualifier.String())
+ destKeyspace, destTabletType, dest, err := ParseDestinationTarget(name.Qualifier.String(), vc.tabletType, vc.vschema)
if err != nil {
return nil, nil, "", destTabletType, nil, err
}
@@ -322,7 +453,23 @@ func (vc *vcursorImpl) FindTableOrVindex(name sqlparser.TableName) (*vindexes.Ta
return table, vindex, destKeyspace, destTabletType, dest, nil
}
-func (vc *vcursorImpl) getDualTable() (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) {
+func (vc *VCursorImpl) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) {
+ return ParseDestinationTarget(targetString, vc.tabletType, vc.vschema)
+}
+
+// ParseDestinationTarget parses destination target string and provides a keyspace if possible.
+func ParseDestinationTarget(targetString string, tablet topodatapb.TabletType, vschema *vindexes.VSchema) (string, topodatapb.TabletType, key.Destination, error) {
+ destKeyspace, destTabletType, dest, err := topoprotopb.ParseDestination(targetString, tablet)
+ // If the keyspace is not specified, and there is only one keyspace in the VSchema, use that.
+ if destKeyspace == "" && len(vschema.Keyspaces) == 1 {
+ for k := range vschema.Keyspaces {
+ destKeyspace = k
+ }
+ }
+ return destKeyspace, destTabletType, dest, err
+}
+
+func (vc *VCursorImpl) getDualTable() (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error) {
ksName := vc.getActualKeyspace()
var ks *vindexes.Keyspace
if ksName == "" {
@@ -339,7 +486,7 @@ func (vc *vcursorImpl) getDualTable() (*vindexes.Table, vindexes.Vindex, string,
return tbl, nil, ksName, topodatapb.TabletType_PRIMARY, nil, nil
}
-func (vc *vcursorImpl) getActualKeyspace() string {
+func (vc *VCursorImpl) getActualKeyspace() string {
if !sqlparser.SystemSchema(vc.keyspace) {
return vc.keyspace
}
@@ -350,12 +497,12 @@ func (vc *vcursorImpl) getActualKeyspace() string {
return ks.Name
}
-// DefaultKeyspace returns the default keyspace of the current request
+// SelectedKeyspace returns the selected keyspace of the current request
// if there is one. If the keyspace specified in the target cannot be
// identified, it returns an error.
-func (vc *vcursorImpl) DefaultKeyspace() (*vindexes.Keyspace, error) {
+func (vc *VCursorImpl) SelectedKeyspace() (*vindexes.Keyspace, error) {
if ignoreKeyspace(vc.keyspace) {
- return nil, errNoKeyspace
+ return nil, ErrNoKeyspace
}
ks, ok := vc.vschema.Keyspaces[vc.keyspace]
if !ok {
@@ -366,12 +513,12 @@ func (vc *vcursorImpl) DefaultKeyspace() (*vindexes.Keyspace, error) {
var errNoDbAvailable = vterrors.NewErrorf(vtrpcpb.Code_FAILED_PRECONDITION, vterrors.NoDB, "no database available")
-func (vc *vcursorImpl) AnyKeyspace() (*vindexes.Keyspace, error) {
- keyspace, err := vc.DefaultKeyspace()
+func (vc *VCursorImpl) AnyKeyspace() (*vindexes.Keyspace, error) {
+ keyspace, err := vc.SelectedKeyspace()
if err == nil {
return keyspace, nil
}
- if err != errNoKeyspace {
+ if err != ErrNoKeyspace {
return nil, err
}
@@ -392,7 +539,7 @@ func (vc *vcursorImpl) AnyKeyspace() (*vindexes.Keyspace, error) {
}
// getSortedServingKeyspaces gets the sorted serving keyspaces
-func (vc *vcursorImpl) getSortedServingKeyspaces() []*vindexes.Keyspace {
+func (vc *VCursorImpl) getSortedServingKeyspaces() []*vindexes.Keyspace {
var keyspaces []*vindexes.Keyspace
if vc.resolver != nil && vc.resolver.GetGateway() != nil {
@@ -416,7 +563,7 @@ func (vc *vcursorImpl) getSortedServingKeyspaces() []*vindexes.Keyspace {
return keyspaces
}
-func (vc *vcursorImpl) FirstSortedKeyspace() (*vindexes.Keyspace, error) {
+func (vc *VCursorImpl) FirstSortedKeyspace() (*vindexes.Keyspace, error) {
if len(vc.vschema.Keyspaces) == 0 {
return nil, errNoDbAvailable
}
@@ -426,17 +573,17 @@ func (vc *vcursorImpl) FirstSortedKeyspace() (*vindexes.Keyspace, error) {
}
// SysVarSetEnabled implements the ContextVSchema interface
-func (vc *vcursorImpl) SysVarSetEnabled() bool {
+func (vc *VCursorImpl) SysVarSetEnabled() bool {
return vc.GetSessionEnableSystemSettings()
}
// KeyspaceExists provides whether the keyspace exists or not.
-func (vc *vcursorImpl) KeyspaceExists(ks string) bool {
+func (vc *VCursorImpl) KeyspaceExists(ks string) bool {
return vc.vschema.Keyspaces[ks] != nil
}
// AllKeyspace implements the ContextVSchema interface
-func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) {
+func (vc *VCursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) {
if len(vc.vschema.Keyspaces) == 0 {
return nil, errNoDbAvailable
}
@@ -448,7 +595,7 @@ func (vc *vcursorImpl) AllKeyspace() ([]*vindexes.Keyspace, error) {
}
// FindKeyspace implements the VSchema interface
-func (vc *vcursorImpl) FindKeyspace(keyspace string) (*vindexes.Keyspace, error) {
+func (vc *VCursorImpl) FindKeyspace(keyspace string) (*vindexes.Keyspace, error) {
if len(vc.vschema.Keyspaces) == 0 {
return nil, errNoDbAvailable
}
@@ -461,52 +608,85 @@ func (vc *vcursorImpl) FindKeyspace(keyspace string) (*vindexes.Keyspace, error)
}
// Planner implements the ContextVSchema interface
-func (vc *vcursorImpl) Planner() plancontext.PlannerVersion {
- if vc.safeSession.Options != nil &&
- vc.safeSession.Options.PlannerVersion != querypb.ExecuteOptions_DEFAULT_PLANNER {
- return vc.safeSession.Options.PlannerVersion
+func (vc *VCursorImpl) Planner() plancontext.PlannerVersion {
+ if vc.SafeSession.Options != nil &&
+ vc.SafeSession.Options.PlannerVersion != querypb.ExecuteOptions_DEFAULT_PLANNER {
+ return vc.SafeSession.Options.PlannerVersion
}
- return vc.pv
+ return vc.config.PlannerVersion
}
// GetSemTable implements the ContextVSchema interface
-func (vc *vcursorImpl) GetSemTable() *semantics.SemTable {
+func (vc *VCursorImpl) GetSemTable() *semantics.SemTable {
return vc.semTable
}
// TargetString returns the current TargetString of the session.
-func (vc *vcursorImpl) TargetString() string {
- return vc.safeSession.TargetString
+func (vc *VCursorImpl) TargetString() string {
+ return vc.SafeSession.TargetString
}
// MaxBufferingRetries is to represent max retries on buffering.
const MaxBufferingRetries = 3
-func (vc *vcursorImpl) ExecutePrimitive(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
+func (vc *VCursorImpl) ExecutePrimitive(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
for try := 0; try < MaxBufferingRetries; try++ {
res, err := primitive.TryExecute(ctx, vc, bindVars, wantfields)
if err != nil && vterrors.RootCause(err) == buffer.ShardMissingError {
continue
}
+ vc.logOpTraffic(primitive, res)
return res, err
}
return nil, vterrors.New(vtrpcpb.Code_UNAVAILABLE, "upstream shards are not available")
}
-func (vc *vcursorImpl) ExecutePrimitiveStandalone(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
- // clone the vcursorImpl with a new session.
+func (vc *VCursorImpl) logOpTraffic(primitive engine.Primitive, res *sqltypes.Result) {
+ if vc.interOpStats != nil {
+ rows := vc.interOpStats[primitive]
+ if res == nil {
+ rows = append(rows, 0)
+ } else {
+ rows = append(rows, len(res.Rows))
+ }
+ vc.interOpStats[primitive] = rows
+ }
+}
+
+func (vc *VCursorImpl) logShardsQueried(primitive engine.Primitive, shardsNb int) {
+ if vc.shardsStats != nil {
+ vc.shardsStats[primitive] += engine.ShardsQueried(shardsNb)
+ }
+}
+
+func (vc *VCursorImpl) ExecutePrimitiveStandalone(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool) (*sqltypes.Result, error) {
+ // clone the VCursorImpl with a new session.
newVC := vc.cloneWithAutocommitSession()
for try := 0; try < MaxBufferingRetries; try++ {
res, err := primitive.TryExecute(ctx, newVC, bindVars, wantfields)
if err != nil && vterrors.RootCause(err) == buffer.ShardMissingError {
continue
}
+ vc.logOpTraffic(primitive, res)
return res, err
}
return nil, vterrors.New(vtrpcpb.Code_UNAVAILABLE, "upstream shards are not available")
}
-func (vc *vcursorImpl) StreamExecutePrimitive(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
+func (vc *VCursorImpl) wrapCallback(callback func(*sqltypes.Result) error, primitive engine.Primitive) func(*sqltypes.Result) error {
+ if vc.interOpStats == nil {
+ return callback
+ }
+
+ return func(result *sqltypes.Result) error {
+ vc.logOpTraffic(primitive, result)
+ return callback(result)
+ }
+}
+
+func (vc *VCursorImpl) StreamExecutePrimitive(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(*sqltypes.Result) error) error {
+ callback = vc.wrapCallback(callback, primitive)
+
for try := 0; try < MaxBufferingRetries; try++ {
err := primitive.TryStreamExecute(ctx, vc, bindVars, wantfields, callback)
if err != nil && vterrors.RootCause(err) == buffer.ShardMissingError {
@@ -517,8 +697,10 @@ func (vc *vcursorImpl) StreamExecutePrimitive(ctx context.Context, primitive eng
return vterrors.New(vtrpcpb.Code_UNAVAILABLE, "upstream shards are not available")
}
-func (vc *vcursorImpl) StreamExecutePrimitiveStandalone(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(result *sqltypes.Result) error) error {
- // clone the vcursorImpl with a new session.
+func (vc *VCursorImpl) StreamExecutePrimitiveStandalone(ctx context.Context, primitive engine.Primitive, bindVars map[string]*querypb.BindVariable, wantfields bool, callback func(result *sqltypes.Result) error) error {
+ callback = vc.wrapCallback(callback, primitive)
+
+ // clone the VCursorImpl with a new session.
newVC := vc.cloneWithAutocommitSession()
for try := 0; try < MaxBufferingRetries; try++ {
err := primitive.TryStreamExecute(ctx, newVC, bindVars, wantfields, callback)
@@ -531,12 +713,11 @@ func (vc *vcursorImpl) StreamExecutePrimitiveStandalone(ctx context.Context, pri
}
// Execute is part of the engine.VCursor interface.
-func (vc *vcursorImpl) Execute(ctx context.Context, method string, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError bool, co vtgatepb.CommitOrder) (*sqltypes.Result, error) {
- session := vc.safeSession
+func (vc *VCursorImpl) Execute(ctx context.Context, method string, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError bool, co vtgatepb.CommitOrder) (*sqltypes.Result, error) {
+ session := vc.SafeSession
if co == vtgatepb.CommitOrder_AUTOCOMMIT {
// For autocommit, we have to create an independent session.
- session = NewAutocommitSession(vc.safeSession.Session)
- session.logging = vc.safeSession.logging
+ session = vc.SafeSession.NewAutocommitSession()
rollbackOnError = false
} else {
session.SetCommitOrder(co)
@@ -557,24 +738,22 @@ func (vc *vcursorImpl) Execute(ctx context.Context, method string, query string,
// markSavepoint opens an internal savepoint before executing the original query.
// This happens only when rollback is allowed and no other savepoint was executed
// and the query is executed in an explicit transaction (i.e. started by the client).
-func (vc *vcursorImpl) markSavepoint(ctx context.Context, needsRollbackOnParialExec bool, bindVars map[string]*querypb.BindVariable) error {
- if !needsRollbackOnParialExec || !vc.safeSession.CanAddSavepoint() {
+func (vc *VCursorImpl) markSavepoint(ctx context.Context, needsRollbackOnParialExec bool, bindVars map[string]*querypb.BindVariable) error {
+ if !needsRollbackOnParialExec || !vc.SafeSession.CanAddSavepoint() {
return nil
}
uID := fmt.Sprintf("_vt%s", strings.ReplaceAll(uuid.NewString(), "-", "_"))
spQuery := fmt.Sprintf("%ssavepoint %s%s", vc.marginComments.Leading, uID, vc.marginComments.Trailing)
- _, err := vc.executor.Execute(ctx, nil, "MarkSavepoint", vc.safeSession, spQuery, bindVars)
+ _, err := vc.executor.Execute(ctx, nil, "MarkSavepoint", vc.SafeSession, spQuery, bindVars)
if err != nil {
return err
}
- vc.safeSession.SetSavepoint(uID)
+ vc.SafeSession.SetSavepoint(uID)
return nil
}
-const txRollback = "Rollback Transaction"
-
// ExecuteMultiShard is part of the engine.VCursor interface.
-func (vc *vcursorImpl) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, rollbackOnError, canAutocommit bool) (*sqltypes.Result, []error) {
+func (vc *VCursorImpl) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, rollbackOnError, canAutocommit bool) (*sqltypes.Result, []error) {
noOfShards := len(rss)
atomic.AddUint64(&vc.logStats.ShardQueries, uint64(noOfShards))
err := vc.markSavepoint(ctx, rollbackOnError && (noOfShards > 1), map[string]*querypb.BindVariable{})
@@ -582,14 +761,16 @@ func (vc *vcursorImpl) ExecuteMultiShard(ctx context.Context, primitive engine.P
return nil, []error{err}
}
- qr, errs := vc.executor.ExecuteMultiShard(ctx, primitive, rss, commentedShardQueries(queries, vc.marginComments), vc.safeSession, canAutocommit, vc.ignoreMaxMemoryRows)
+ qr, errs := vc.executor.ExecuteMultiShard(ctx, primitive, rss, commentedShardQueries(queries, vc.marginComments), vc.SafeSession, canAutocommit, vc.ignoreMaxMemoryRows, vc.observer)
vc.setRollbackOnPartialExecIfRequired(len(errs) != len(rss), rollbackOnError)
-
+ vc.logShardsQueried(primitive, len(rss))
return qr, errs
}
// StreamExecuteMulti is the streaming version of ExecuteMultiShard.
-func (vc *vcursorImpl) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, bindVars []map[string]*querypb.BindVariable, rollbackOnError bool, autocommit bool, callback func(reply *sqltypes.Result) error) []error {
+func (vc *VCursorImpl) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, bindVars []map[string]*querypb.BindVariable, rollbackOnError bool, autocommit bool, callback func(reply *sqltypes.Result) error) []error {
+ callback = vc.wrapCallback(callback, primitive)
+
noOfShards := len(rss)
atomic.AddUint64(&vc.logStats.ShardQueries, uint64(noOfShards))
err := vc.markSavepoint(ctx, rollbackOnError && (noOfShards > 1), map[string]*querypb.BindVariable{})
@@ -597,20 +778,20 @@ func (vc *vcursorImpl) StreamExecuteMulti(ctx context.Context, primitive engine.
return []error{err}
}
- errs := vc.executor.StreamExecuteMulti(ctx, primitive, vc.marginComments.Leading+query+vc.marginComments.Trailing, rss, bindVars, vc.safeSession, autocommit, callback)
+ errs := vc.executor.StreamExecuteMulti(ctx, primitive, vc.marginComments.Leading+query+vc.marginComments.Trailing, rss, bindVars, vc.SafeSession, autocommit, callback, vc.observer)
vc.setRollbackOnPartialExecIfRequired(len(errs) != len(rss), rollbackOnError)
return errs
}
// ExecuteLock is for executing advisory lock statements.
-func (vc *vcursorImpl) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
+func (vc *VCursorImpl) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
query.Sql = vc.marginComments.Leading + query.Sql + vc.marginComments.Trailing
- return vc.executor.ExecuteLock(ctx, rs, query, vc.safeSession, lockFuncType)
+ return vc.executor.ExecuteLock(ctx, rs, query, vc.SafeSession, lockFuncType)
}
// ExecuteStandalone is part of the engine.VCursor interface.
-func (vc *vcursorImpl) ExecuteStandalone(ctx context.Context, primitive engine.Primitive, query string, bindVars map[string]*querypb.BindVariable, rs *srvtopo.ResolvedShard) (*sqltypes.Result, error) {
+func (vc *VCursorImpl) ExecuteStandalone(ctx context.Context, primitive engine.Primitive, query string, bindVars map[string]*querypb.BindVariable, rs *srvtopo.ResolvedShard) (*sqltypes.Result, error) {
rss := []*srvtopo.ResolvedShard{rs}
bqs := []*querypb.BoundQuery{
{
@@ -620,12 +801,13 @@ func (vc *vcursorImpl) ExecuteStandalone(ctx context.Context, primitive engine.P
}
// The autocommit flag is always set to false because we currently don't
// execute DMLs through ExecuteStandalone.
- qr, errs := vc.executor.ExecuteMultiShard(ctx, primitive, rss, bqs, NewAutocommitSession(vc.safeSession.Session), false /* autocommit */, vc.ignoreMaxMemoryRows)
+ qr, errs := vc.executor.ExecuteMultiShard(ctx, primitive, rss, bqs, NewAutocommitSession(vc.SafeSession.Session), false /* autocommit */, vc.ignoreMaxMemoryRows, vc.observer)
+ vc.logShardsQueried(primitive, len(rss))
return qr, vterrors.Aggregate(errs)
}
// ExecuteKeyspaceID is part of the engine.VCursor interface.
-func (vc *vcursorImpl) ExecuteKeyspaceID(ctx context.Context, keyspace string, ksid []byte, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError, autocommit bool) (*sqltypes.Result, error) {
+func (vc *VCursorImpl) ExecuteKeyspaceID(ctx context.Context, keyspace string, ksid []byte, query string, bindVars map[string]*querypb.BindVariable, rollbackOnError, autocommit bool) (*sqltypes.Result, error) {
atomic.AddUint64(&vc.logStats.ShardQueries, 1)
rss, _, err := vc.ResolveDestinations(ctx, keyspace, nil, []key.Destination{key.DestinationKeyspaceID(ksid)})
if err != nil {
@@ -642,17 +824,17 @@ func (vc *vcursorImpl) ExecuteKeyspaceID(ctx context.Context, keyspace string, k
// This creates a transaction but that transaction is for locking purpose only and should not cause multi-db transaction error.
// This fields helps in to ignore multi-db transaction error when it states `queryFromVindex`.
if !rollbackOnError {
- vc.safeSession.queryFromVindex = true
+ vc.SafeSession.SetQueryFromVindex(true)
defer func() {
- vc.safeSession.queryFromVindex = false
+ vc.SafeSession.SetQueryFromVindex(false)
}()
}
qr, errs := vc.ExecuteMultiShard(ctx, nil, rss, queries, rollbackOnError, autocommit)
return qr, vterrors.Aggregate(errs)
}
-func (vc *vcursorImpl) InTransactionAndIsDML() bool {
- if !vc.safeSession.InTransaction() {
+func (vc *VCursorImpl) InTransactionAndIsDML() bool {
+ if !vc.SafeSession.InTransaction() {
return false
}
switch vc.logStats.StmtType {
@@ -662,7 +844,7 @@ func (vc *vcursorImpl) InTransactionAndIsDML() bool {
return false
}
-func (vc *vcursorImpl) LookupRowLockShardSession() vtgatepb.CommitOrder {
+func (vc *VCursorImpl) LookupRowLockShardSession() vtgatepb.CommitOrder {
switch vc.logStats.StmtType {
case "DELETE", "UPDATE":
return vtgatepb.CommitOrder_POST
@@ -671,23 +853,23 @@ func (vc *vcursorImpl) LookupRowLockShardSession() vtgatepb.CommitOrder {
}
// AutocommitApproval is part of the engine.VCursor interface.
-func (vc *vcursorImpl) AutocommitApproval() bool {
- return vc.safeSession.AutocommitApproval()
+func (vc *VCursorImpl) AutocommitApproval() bool {
+ return vc.SafeSession.AutocommitApproval()
}
// setRollbackOnPartialExecIfRequired sets the value on SafeSession.rollbackOnPartialExec
// when the query gets successfully executed on at least one shard,
// there does not exist any old savepoint for which rollback is already set
// and rollback on error is allowed.
-func (vc *vcursorImpl) setRollbackOnPartialExecIfRequired(atleastOneSuccess bool, rollbackOnError bool) {
- if atleastOneSuccess && rollbackOnError && !vc.safeSession.IsRollbackSet() {
- vc.safeSession.SetRollbackCommand()
+func (vc *VCursorImpl) setRollbackOnPartialExecIfRequired(atleastOneSuccess bool, rollbackOnError bool) {
+ if atleastOneSuccess && rollbackOnError && !vc.SafeSession.IsRollbackSet() {
+ vc.SafeSession.SetRollbackCommand()
}
}
// fixupPartiallyMovedShards checks if any of the shards in the route has a ShardRoutingRule (true when a keyspace
// is in the middle of being moved to another keyspace using MoveTables moving a subset of shards at a time
-func (vc *vcursorImpl) fixupPartiallyMovedShards(rss []*srvtopo.ResolvedShard) ([]*srvtopo.ResolvedShard, error) {
+func (vc *VCursorImpl) fixupPartiallyMovedShards(rss []*srvtopo.ResolvedShard) ([]*srvtopo.ResolvedShard, error) {
if vc.vschema.ShardRoutingRules == nil {
return rss, nil
}
@@ -704,12 +886,12 @@ func (vc *vcursorImpl) fixupPartiallyMovedShards(rss []*srvtopo.ResolvedShard) (
return rss, nil
}
-func (vc *vcursorImpl) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) {
+func (vc *VCursorImpl) ResolveDestinations(ctx context.Context, keyspace string, ids []*querypb.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][]*querypb.Value, error) {
rss, values, err := vc.resolver.ResolveDestinations(ctx, keyspace, vc.tabletType, ids, destinations)
if err != nil {
return nil, nil, err
}
- if enableShardRouting {
+ if vc.config.EnableShardRouting {
rss, err = vc.fixupPartiallyMovedShards(rss)
if err != nil {
return nil, nil, err
@@ -718,12 +900,12 @@ func (vc *vcursorImpl) ResolveDestinations(ctx context.Context, keyspace string,
return rss, values, err
}
-func (vc *vcursorImpl) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) {
+func (vc *VCursorImpl) ResolveDestinationsMultiCol(ctx context.Context, keyspace string, ids [][]sqltypes.Value, destinations []key.Destination) ([]*srvtopo.ResolvedShard, [][][]sqltypes.Value, error) {
rss, values, err := vc.resolver.ResolveDestinationsMultiCol(ctx, keyspace, vc.tabletType, ids, destinations)
if err != nil {
return nil, nil, err
}
- if enableShardRouting {
+ if vc.config.EnableShardRouting {
rss, err = vc.fixupPartiallyMovedShards(rss)
if err != nil {
return nil, nil, err
@@ -732,12 +914,12 @@ func (vc *vcursorImpl) ResolveDestinationsMultiCol(ctx context.Context, keyspace
return rss, values, err
}
-func (vc *vcursorImpl) Session() engine.SessionActions {
+func (vc *VCursorImpl) Session() engine.SessionActions {
return vc
}
-func (vc *vcursorImpl) SetTarget(target string) error {
- keyspace, tabletType, _, err := topoprotopb.ParseDestination(target, defaultTabletType)
+func (vc *VCursorImpl) SetTarget(target string) error {
+ keyspace, tabletType, _, err := topoprotopb.ParseDestination(target, vc.config.DefaultTabletType)
if err != nil {
return err
}
@@ -745,10 +927,12 @@ func (vc *vcursorImpl) SetTarget(target string) error {
return vterrors.VT05003(keyspace)
}
- if vc.safeSession.InTransaction() && tabletType != topodatapb.TabletType_PRIMARY {
+ if vc.SafeSession.InTransaction() && tabletType != topodatapb.TabletType_PRIMARY {
return vterrors.NewErrorf(vtrpcpb.Code_INVALID_ARGUMENT, vterrors.LockOrActiveTransaction, "can't execute the given command because you have an active transaction")
}
- vc.safeSession.SetTargetString(target)
+ vc.SafeSession.SetTargetString(target)
+ vc.keyspace = keyspace
+ vc.tabletType = tabletType
return nil
}
@@ -756,30 +940,30 @@ func ignoreKeyspace(keyspace string) bool {
return keyspace == "" || sqlparser.SystemSchema(keyspace)
}
-func (vc *vcursorImpl) SetUDV(key string, value any) error {
+func (vc *VCursorImpl) SetUDV(key string, value any) error {
bindValue, err := sqltypes.BuildBindVariable(value)
if err != nil {
return err
}
- vc.safeSession.SetUserDefinedVariable(key, bindValue)
+ vc.SafeSession.SetUserDefinedVariable(key, bindValue)
return nil
}
-func (vc *vcursorImpl) SetSysVar(name string, expr string) {
- vc.safeSession.SetSystemVariable(name, expr)
+func (vc *VCursorImpl) SetSysVar(name string, expr string) {
+ vc.SafeSession.SetSystemVariable(name, expr)
}
// NeedsReservedConn implements the SessionActions interface
-func (vc *vcursorImpl) NeedsReservedConn() {
- vc.safeSession.SetReservedConn(true)
+func (vc *VCursorImpl) NeedsReservedConn() {
+ vc.SafeSession.SetReservedConn(true)
}
-func (vc *vcursorImpl) InReservedConn() bool {
- return vc.safeSession.InReservedConn()
+func (vc *VCursorImpl) InReservedConn() bool {
+ return vc.SafeSession.InReservedConn()
}
-func (vc *vcursorImpl) ShardSession() []*srvtopo.ResolvedShard {
- ss := vc.safeSession.GetShardSessions()
+func (vc *VCursorImpl) ShardSession() []*srvtopo.ResolvedShard {
+ ss := vc.SafeSession.GetShardSessions()
if len(ss) == 0 {
return nil
}
@@ -794,12 +978,12 @@ func (vc *vcursorImpl) ShardSession() []*srvtopo.ResolvedShard {
}
// Destination implements the ContextVSchema interface
-func (vc *vcursorImpl) Destination() key.Destination {
+func (vc *VCursorImpl) Destination() key.Destination {
return vc.destination
}
// TabletType implements the ContextVSchema interface
-func (vc *vcursorImpl) TabletType() topodatapb.TabletType {
+func (vc *VCursorImpl) TabletType() topodatapb.TabletType {
return vc.tabletType
}
@@ -818,13 +1002,13 @@ func commentedShardQueries(shardQueries []*querypb.BoundQuery, marginComments sq
}
// TargetDestination implements the ContextVSchema interface
-func (vc *vcursorImpl) TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) {
+func (vc *VCursorImpl) TargetDestination(qualifier string) (key.Destination, *vindexes.Keyspace, topodatapb.TabletType, error) {
keyspaceName := vc.getActualKeyspace()
if vc.destination == nil && qualifier != "" {
keyspaceName = qualifier
}
if keyspaceName == "" {
- return nil, nil, 0, errNoKeyspace
+ return nil, nil, 0, ErrNoKeyspace
}
keyspace := vc.vschema.Keyspaces[keyspaceName]
if keyspace == nil {
@@ -834,202 +1018,219 @@ func (vc *vcursorImpl) TargetDestination(qualifier string) (key.Destination, *vi
}
// SetAutocommit implements the SessionActions interface
-func (vc *vcursorImpl) SetAutocommit(ctx context.Context, autocommit bool) error {
- if autocommit && vc.safeSession.InTransaction() {
- if err := vc.executor.Commit(ctx, vc.safeSession); err != nil {
+func (vc *VCursorImpl) SetAutocommit(ctx context.Context, autocommit bool) error {
+ if autocommit && vc.SafeSession.InTransaction() {
+ if err := vc.executor.Commit(ctx, vc.SafeSession); err != nil {
return err
}
}
- vc.safeSession.Autocommit = autocommit
+ vc.SafeSession.Autocommit = autocommit
return nil
}
// SetQueryTimeout implements the SessionActions interface
-func (vc *vcursorImpl) SetQueryTimeout(maxExecutionTime int64) {
- vc.safeSession.QueryTimeout = maxExecutionTime
-}
-
-// GetQueryTimeout implements the SessionActions interface
-// The priority of adding query timeouts -
-// 1. Query timeout comment directive.
-// 2. If the comment directive is unspecified, then we use the session setting.
-// 3. If the comment directive and session settings is unspecified, then we use the global default specified by a flag.
-func (vc *vcursorImpl) GetQueryTimeout(queryTimeoutFromComments int) int {
- if queryTimeoutFromComments != 0 {
- return queryTimeoutFromComments
- }
- sessionQueryTimeout := int(vc.safeSession.GetQueryTimeout())
- if sessionQueryTimeout != 0 {
- return sessionQueryTimeout
- }
- return queryTimeout
+func (vc *VCursorImpl) SetQueryTimeout(maxExecutionTime int64) {
+ vc.SafeSession.QueryTimeout = maxExecutionTime
}
// SetClientFoundRows implements the SessionActions interface
-func (vc *vcursorImpl) SetClientFoundRows(_ context.Context, clientFoundRows bool) error {
- vc.safeSession.GetOrCreateOptions().ClientFoundRows = clientFoundRows
+func (vc *VCursorImpl) SetClientFoundRows(_ context.Context, clientFoundRows bool) error {
+ vc.SafeSession.GetOrCreateOptions().ClientFoundRows = clientFoundRows
return nil
}
// SetSkipQueryPlanCache implements the SessionActions interface
-func (vc *vcursorImpl) SetSkipQueryPlanCache(_ context.Context, skipQueryPlanCache bool) error {
- vc.safeSession.GetOrCreateOptions().SkipQueryPlanCache = skipQueryPlanCache
+func (vc *VCursorImpl) SetSkipQueryPlanCache(_ context.Context, skipQueryPlanCache bool) error {
+ vc.SafeSession.GetOrCreateOptions().SkipQueryPlanCache = skipQueryPlanCache
return nil
}
// SetSQLSelectLimit implements the SessionActions interface
-func (vc *vcursorImpl) SetSQLSelectLimit(limit int64) error {
- vc.safeSession.GetOrCreateOptions().SqlSelectLimit = limit
+func (vc *VCursorImpl) SetSQLSelectLimit(limit int64) error {
+ vc.SafeSession.GetOrCreateOptions().SqlSelectLimit = limit
return nil
}
// SetTransactionMode implements the SessionActions interface
-func (vc *vcursorImpl) SetTransactionMode(mode vtgatepb.TransactionMode) {
- vc.safeSession.TransactionMode = mode
+func (vc *VCursorImpl) SetTransactionMode(mode vtgatepb.TransactionMode) {
+ vc.SafeSession.TransactionMode = mode
}
// SetWorkload implements the SessionActions interface
-func (vc *vcursorImpl) SetWorkload(workload querypb.ExecuteOptions_Workload) {
- vc.safeSession.GetOrCreateOptions().Workload = workload
+func (vc *VCursorImpl) SetWorkload(workload querypb.ExecuteOptions_Workload) {
+ vc.SafeSession.GetOrCreateOptions().Workload = workload
}
// SetPlannerVersion implements the SessionActions interface
-func (vc *vcursorImpl) SetPlannerVersion(v plancontext.PlannerVersion) {
- vc.safeSession.GetOrCreateOptions().PlannerVersion = v
+func (vc *VCursorImpl) SetPlannerVersion(v plancontext.PlannerVersion) {
+ vc.SafeSession.GetOrCreateOptions().PlannerVersion = v
}
-func (vc *vcursorImpl) SetPriority(priority string) {
+func (vc *VCursorImpl) SetPriority(priority string) {
if priority != "" {
- vc.safeSession.GetOrCreateOptions().Priority = priority
- } else if vc.safeSession.Options != nil && vc.safeSession.Options.Priority != "" {
- vc.safeSession.Options.Priority = ""
+ vc.SafeSession.GetOrCreateOptions().Priority = priority
+ } else if vc.SafeSession.Options != nil && vc.SafeSession.Options.Priority != "" {
+ vc.SafeSession.Options.Priority = ""
+ }
+}
+
+func (vc *VCursorImpl) SetExecQueryTimeout(timeout *int) {
+ // Determine the effective timeout: use passed timeout if non-nil, otherwise use session's query timeout if available
+ var execTimeout *int
+ if timeout != nil {
+ execTimeout = timeout
+ } else if sessionTimeout := vc.getQueryTimeout(); sessionTimeout > 0 {
+ execTimeout = &sessionTimeout
}
+ // If no effective timeout and no session options, return early
+ if execTimeout == nil {
+ if vc.SafeSession.GetOptions() == nil {
+ return
+ }
+ vc.SafeSession.GetOrCreateOptions().Timeout = nil
+ return
+ }
+
+ vc.queryTimeout = time.Duration(*execTimeout) * time.Millisecond
+ // Set the authoritative timeout using the determined execTimeout
+ vc.SafeSession.GetOrCreateOptions().Timeout = &querypb.ExecuteOptions_AuthoritativeTimeout{
+ AuthoritativeTimeout: int64(*execTimeout),
+ }
+}
+
+// getQueryTimeout returns timeout based on the priority
+// session setting > global default specified by a flag.
+func (vc *VCursorImpl) getQueryTimeout() int {
+ sessionQueryTimeout := int(vc.SafeSession.GetQueryTimeout())
+ if sessionQueryTimeout != 0 {
+ return sessionQueryTimeout
+ }
+ return vc.config.QueryTimeout
}
// SetConsolidator implements the SessionActions interface
-func (vc *vcursorImpl) SetConsolidator(consolidator querypb.ExecuteOptions_Consolidator) {
+func (vc *VCursorImpl) SetConsolidator(consolidator querypb.ExecuteOptions_Consolidator) {
// Avoid creating session Options when they do not yet exist and the
// consolidator is unspecified.
- if consolidator == querypb.ExecuteOptions_CONSOLIDATOR_UNSPECIFIED && vc.safeSession.GetOptions() == nil {
+ if consolidator == querypb.ExecuteOptions_CONSOLIDATOR_UNSPECIFIED && vc.SafeSession.GetOptions() == nil {
return
}
- vc.safeSession.GetOrCreateOptions().Consolidator = consolidator
+ vc.SafeSession.GetOrCreateOptions().Consolidator = consolidator
}
-func (vc *vcursorImpl) SetWorkloadName(workloadName string) {
+func (vc *VCursorImpl) SetWorkloadName(workloadName string) {
if workloadName != "" {
- vc.safeSession.GetOrCreateOptions().WorkloadName = workloadName
+ vc.SafeSession.GetOrCreateOptions().WorkloadName = workloadName
}
}
// SetFoundRows implements the SessionActions interface
-func (vc *vcursorImpl) SetFoundRows(foundRows uint64) {
- vc.safeSession.FoundRows = foundRows
- vc.safeSession.foundRowsHandled = true
+func (vc *VCursorImpl) SetFoundRows(foundRows uint64) {
+ vc.SafeSession.SetFoundRows(foundRows)
}
// SetDDLStrategy implements the SessionActions interface
-func (vc *vcursorImpl) SetDDLStrategy(strategy string) {
- vc.safeSession.SetDDLStrategy(strategy)
+func (vc *VCursorImpl) SetDDLStrategy(strategy string) {
+ vc.SafeSession.SetDDLStrategy(strategy)
}
// GetDDLStrategy implements the SessionActions interface
-func (vc *vcursorImpl) GetDDLStrategy() string {
- return vc.safeSession.GetDDLStrategy()
+func (vc *VCursorImpl) GetDDLStrategy() string {
+ return vc.SafeSession.GetDDLStrategy()
}
// SetMigrationContext implements the SessionActions interface
-func (vc *vcursorImpl) SetMigrationContext(migrationContext string) {
- vc.safeSession.SetMigrationContext(migrationContext)
+func (vc *VCursorImpl) SetMigrationContext(migrationContext string) {
+ vc.SafeSession.SetMigrationContext(migrationContext)
}
// GetMigrationContext implements the SessionActions interface
-func (vc *vcursorImpl) GetMigrationContext() string {
- return vc.safeSession.GetMigrationContext()
+func (vc *VCursorImpl) GetMigrationContext() string {
+ return vc.SafeSession.GetMigrationContext()
}
// GetSessionUUID implements the SessionActions interface
-func (vc *vcursorImpl) GetSessionUUID() string {
- return vc.safeSession.GetSessionUUID()
+func (vc *VCursorImpl) GetSessionUUID() string {
+ return vc.SafeSession.GetSessionUUID()
}
// SetSessionEnableSystemSettings implements the SessionActions interface
-func (vc *vcursorImpl) SetSessionEnableSystemSettings(_ context.Context, allow bool) error {
- vc.safeSession.SetSessionEnableSystemSettings(allow)
+func (vc *VCursorImpl) SetSessionEnableSystemSettings(_ context.Context, allow bool) error {
+ vc.SafeSession.SetSessionEnableSystemSettings(allow)
return nil
}
// GetSessionEnableSystemSettings implements the SessionActions interface
-func (vc *vcursorImpl) GetSessionEnableSystemSettings() bool {
- return vc.safeSession.GetSessionEnableSystemSettings()
+func (vc *VCursorImpl) GetSessionEnableSystemSettings() bool {
+ return vc.SafeSession.GetSessionEnableSystemSettings()
}
// SetReadAfterWriteGTID implements the SessionActions interface
-func (vc *vcursorImpl) SetReadAfterWriteGTID(vtgtid string) {
- vc.safeSession.SetReadAfterWriteGTID(vtgtid)
+func (vc *VCursorImpl) SetReadAfterWriteGTID(vtgtid string) {
+ vc.SafeSession.SetReadAfterWriteGTID(vtgtid)
}
// SetReadAfterWriteTimeout implements the SessionActions interface
-func (vc *vcursorImpl) SetReadAfterWriteTimeout(timeout float64) {
- vc.safeSession.SetReadAfterWriteTimeout(timeout)
+func (vc *VCursorImpl) SetReadAfterWriteTimeout(timeout float64) {
+ vc.SafeSession.SetReadAfterWriteTimeout(timeout)
}
// SetSessionTrackGTIDs implements the SessionActions interface
-func (vc *vcursorImpl) SetSessionTrackGTIDs(enable bool) {
- vc.safeSession.SetSessionTrackGtids(enable)
+func (vc *VCursorImpl) SetSessionTrackGTIDs(enable bool) {
+ vc.SafeSession.SetSessionTrackGtids(enable)
}
// HasCreatedTempTable implements the SessionActions interface
-func (vc *vcursorImpl) HasCreatedTempTable() {
- vc.safeSession.GetOrCreateOptions().HasCreatedTempTables = true
+func (vc *VCursorImpl) HasCreatedTempTable() {
+ vc.SafeSession.GetOrCreateOptions().HasCreatedTempTables = true
}
// GetWarnings implements the SessionActions interface
-func (vc *vcursorImpl) GetWarnings() []*querypb.QueryWarning {
- return vc.safeSession.GetWarnings()
+func (vc *VCursorImpl) GetWarnings() []*querypb.QueryWarning {
+ return vc.SafeSession.GetWarnings()
}
// AnyAdvisoryLockTaken implements the SessionActions interface
-func (vc *vcursorImpl) AnyAdvisoryLockTaken() bool {
- return vc.safeSession.HasAdvisoryLock()
+func (vc *VCursorImpl) AnyAdvisoryLockTaken() bool {
+ return vc.SafeSession.HasAdvisoryLock()
}
// AddAdvisoryLock implements the SessionActions interface
-func (vc *vcursorImpl) AddAdvisoryLock(name string) {
- vc.safeSession.AddAdvisoryLock(name)
+func (vc *VCursorImpl) AddAdvisoryLock(name string) {
+ vc.SafeSession.AddAdvisoryLock(name)
}
// RemoveAdvisoryLock implements the SessionActions interface
-func (vc *vcursorImpl) RemoveAdvisoryLock(name string) {
- vc.safeSession.RemoveAdvisoryLock(name)
+func (vc *VCursorImpl) RemoveAdvisoryLock(name string) {
+ vc.SafeSession.RemoveAdvisoryLock(name)
}
-func (vc *vcursorImpl) SetCommitOrder(co vtgatepb.CommitOrder) {
- vc.safeSession.SetCommitOrder(co)
+func (vc *VCursorImpl) SetCommitOrder(co vtgatepb.CommitOrder) {
+ vc.SafeSession.SetCommitOrder(co)
}
-func (vc *vcursorImpl) InTransaction() bool {
- return vc.safeSession.InTransaction()
+func (vc *VCursorImpl) InTransaction() bool {
+ return vc.SafeSession.InTransaction()
}
-func (vc *vcursorImpl) Commit(ctx context.Context) error {
- return vc.executor.Commit(ctx, vc.safeSession)
+func (vc *VCursorImpl) Commit(ctx context.Context) error {
+ return vc.executor.Commit(ctx, vc.SafeSession)
}
// GetDBDDLPluginName implements the VCursor interface
-func (vc *vcursorImpl) GetDBDDLPluginName() string {
- return dbDDLPlugin
+func (vc *VCursorImpl) GetDBDDLPluginName() string {
+ return vc.config.DBDDLPlugin
}
// KeyspaceAvailable implements the VCursor interface
-func (vc *vcursorImpl) KeyspaceAvailable(ks string) bool {
+func (vc *VCursorImpl) KeyspaceAvailable(ks string) bool {
_, exists := vc.executor.VSchema().Keyspaces[ks]
return exists
}
// ErrorIfShardedF implements the VCursor interface
-func (vc *vcursorImpl) ErrorIfShardedF(ks *vindexes.Keyspace, warn, errFormat string, params ...any) error {
+func (vc *VCursorImpl) ErrorIfShardedF(ks *vindexes.Keyspace, warn, errFormat string, params ...any) error {
if ks.Sharded {
return vterrors.Errorf(vtrpcpb.Code_UNIMPLEMENTED, errFormat, params...)
}
@@ -1038,19 +1239,25 @@ func (vc *vcursorImpl) ErrorIfShardedF(ks *vindexes.Keyspace, warn, errFormat st
return nil
}
+func (vc *VCursorImpl) GetAndEmptyWarnings() []*querypb.QueryWarning {
+ w := vc.warnings
+ vc.warnings = nil
+ return w
+}
+
// WarnUnshardedOnly implements the VCursor interface
-func (vc *vcursorImpl) WarnUnshardedOnly(format string, params ...any) {
- if vc.warnShardedOnly {
+func (vc *VCursorImpl) WarnUnshardedOnly(format string, params ...any) {
+ if vc.config.WarnShardedOnly {
vc.warnings = append(vc.warnings, &querypb.QueryWarning{
Code: uint32(sqlerror.ERNotSupportedYet),
Message: fmt.Sprintf(format, params...),
})
- warnings.Add("WarnUnshardedOnly", 1)
+ vc.executor.AddWarningCount("WarnUnshardedOnly", 1)
}
}
// PlannerWarning implements the VCursor interface
-func (vc *vcursorImpl) PlannerWarning(message string) {
+func (vc *VCursorImpl) PlannerWarning(message string) {
if message == "" {
return
}
@@ -1061,8 +1268,8 @@ func (vc *vcursorImpl) PlannerWarning(message string) {
}
// ForeignKeyMode implements the VCursor interface
-func (vc *vcursorImpl) ForeignKeyMode(keyspace string) (vschemapb.Keyspace_ForeignKeyMode, error) {
- if strings.ToLower(foreignKeyMode) == "disallow" {
+func (vc *VCursorImpl) ForeignKeyMode(keyspace string) (vschemapb.Keyspace_ForeignKeyMode, error) {
+ if vc.config.ForeignKeyMode == vschemapb.Keyspace_disallow {
return vschemapb.Keyspace_disallow, nil
}
ks := vc.vschema.Keyspaces[keyspace]
@@ -1072,7 +1279,7 @@ func (vc *vcursorImpl) ForeignKeyMode(keyspace string) (vschemapb.Keyspace_Forei
return ks.ForeignKeyMode, nil
}
-func (vc *vcursorImpl) KeyspaceError(keyspace string) error {
+func (vc *VCursorImpl) KeyspaceError(keyspace string) error {
ks := vc.vschema.Keyspaces[keyspace]
if ks == nil {
return vterrors.VT14004(keyspace)
@@ -1080,27 +1287,32 @@ func (vc *vcursorImpl) KeyspaceError(keyspace string) error {
return ks.Error
}
-func (vc *vcursorImpl) GetAggregateUDFs() []string {
+func (vc *VCursorImpl) GetAggregateUDFs() []string {
return vc.vschema.GetAggregateUDFs()
}
-// ParseDestinationTarget parses destination target string and sets default keyspace if possible.
-func parseDestinationTarget(targetString string, vschema *vindexes.VSchema) (string, topodatapb.TabletType, key.Destination, error) {
- destKeyspace, destTabletType, dest, err := topoprotopb.ParseDestination(targetString, defaultTabletType)
- // Set default keyspace
- if destKeyspace == "" && len(vschema.Keyspaces) == 1 {
- for k := range vschema.Keyspaces {
- destKeyspace = k
- }
+// FindMirrorRule finds the mirror rule for the requested table name and
+// VSchema tablet type.
+func (vc *VCursorImpl) FindMirrorRule(name sqlparser.TableName) (*vindexes.MirrorRule, error) {
+ destKeyspace, destTabletType, _, err := vc.ParseDestinationTarget(name.Qualifier.String())
+ if err != nil {
+ return nil, err
}
- return destKeyspace, destTabletType, dest, err
+ if destKeyspace == "" {
+ destKeyspace = vc.keyspace
+ }
+ mirrorRule, err := vc.vschema.FindMirrorRule(destKeyspace, name.Name.String(), destTabletType)
+ if err != nil {
+ return nil, err
+ }
+ return mirrorRule, err
}
-func (vc *vcursorImpl) keyForPlan(ctx context.Context, query string, buf io.StringWriter) {
+func (vc *VCursorImpl) KeyForPlan(ctx context.Context, query string, buf io.StringWriter) {
_, _ = buf.WriteString(vc.keyspace)
_, _ = buf.WriteString(vindexes.TabletTypeSuffix[vc.tabletType])
_, _ = buf.WriteString("+Collate:")
- _, _ = buf.WriteString(vc.Environment().CollationEnv().LookupName(vc.collation))
+ _, _ = buf.WriteString(vc.Environment().CollationEnv().LookupName(vc.config.Collation))
if vc.destination != nil {
switch vc.destination.(type) {
@@ -1130,11 +1342,11 @@ func (vc *vcursorImpl) keyForPlan(ctx context.Context, query string, buf io.Stri
_, _ = buf.WriteString(query)
}
-func (vc *vcursorImpl) GetKeyspace() string {
+func (vc *VCursorImpl) GetKeyspace() string {
return vc.keyspace
}
-func (vc *vcursorImpl) ExecuteVSchema(ctx context.Context, keyspace string, vschemaDDL *sqlparser.AlterVschema) error {
+func (vc *VCursorImpl) ExecuteVSchema(ctx context.Context, keyspace string, vschemaDDL *sqlparser.AlterVschema) error {
srvVschema := vc.vm.GetCurrentSrvVschema()
if srvVschema == nil {
return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "vschema not loaded")
@@ -1144,7 +1356,6 @@ func (vc *vcursorImpl) ExecuteVSchema(ctx context.Context, keyspace string, vsch
allowed := vschemaacl.Authorized(user)
if !allowed {
return vterrors.NewErrorf(vtrpcpb.Code_PERMISSION_DENIED, vterrors.AccessDeniedError, "User '%s' is not authorized to perform vschema operations", user.GetUsername())
-
}
// Resolve the keyspace either from the table qualifier or the target keyspace
@@ -1156,12 +1367,11 @@ func (vc *vcursorImpl) ExecuteVSchema(ctx context.Context, keyspace string, vsch
ksName = keyspace
}
if ksName == "" {
- return errNoKeyspace
+ return ErrNoKeyspace
}
ks := srvVschema.Keyspaces[ksName]
ks, err := topotools.ApplyVSchemaDDL(ksName, ks, vschemaDDL)
-
if err != nil {
return err
}
@@ -1169,46 +1379,45 @@ func (vc *vcursorImpl) ExecuteVSchema(ctx context.Context, keyspace string, vsch
srvVschema.Keyspaces[ksName] = ks
return vc.vm.UpdateVSchema(ctx, ksName, srvVschema)
-
}
-func (vc *vcursorImpl) MessageStream(ctx context.Context, rss []*srvtopo.ResolvedShard, tableName string, callback func(*sqltypes.Result) error) error {
+func (vc *VCursorImpl) MessageStream(ctx context.Context, rss []*srvtopo.ResolvedShard, tableName string, callback func(*sqltypes.Result) error) error {
atomic.AddUint64(&vc.logStats.ShardQueries, uint64(len(rss)))
return vc.executor.ExecuteMessageStream(ctx, rss, tableName, callback)
}
-func (vc *vcursorImpl) VStream(ctx context.Context, rss []*srvtopo.ResolvedShard, filter *binlogdatapb.Filter, gtid string, callback func(evs []*binlogdatapb.VEvent) error) error {
+func (vc *VCursorImpl) VStream(ctx context.Context, rss []*srvtopo.ResolvedShard, filter *binlogdatapb.Filter, gtid string, callback func(evs []*binlogdatapb.VEvent) error) error {
return vc.executor.ExecuteVStream(ctx, rss, filter, gtid, callback)
}
-func (vc *vcursorImpl) ShowExec(ctx context.Context, command sqlparser.ShowCommandType, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+func (vc *VCursorImpl) ShowExec(ctx context.Context, command sqlparser.ShowCommandType, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
switch command {
case sqlparser.VitessReplicationStatus:
- return vc.executor.showVitessReplicationStatus(ctx, filter)
+ return vc.executor.ShowVitessReplicationStatus(ctx, filter)
case sqlparser.VitessShards:
- return vc.executor.showShards(ctx, filter, vc.tabletType)
+ return vc.executor.ShowShards(ctx, filter, vc.tabletType)
case sqlparser.VitessTablets:
- return vc.executor.showTablets(filter)
+ return vc.executor.ShowTablets(filter)
case sqlparser.VitessVariables:
- return vc.executor.showVitessMetadata(ctx, filter)
+ return vc.executor.ShowVitessMetadata(ctx, filter)
default:
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "bug: unexpected show command: %v", command)
}
}
-func (vc *vcursorImpl) GetVSchema() *vindexes.VSchema {
+func (vc *VCursorImpl) GetVSchema() *vindexes.VSchema {
return vc.vschema
}
-func (vc *vcursorImpl) GetSrvVschema() *vschemapb.SrvVSchema {
+func (vc *VCursorImpl) GetSrvVschema() *vschemapb.SrvVSchema {
return vc.vm.GetCurrentSrvVschema()
}
-func (vc *vcursorImpl) SetExec(ctx context.Context, name string, value string) error {
- return vc.executor.setVitessMetadata(ctx, name, value)
+func (vc *VCursorImpl) SetExec(ctx context.Context, name string, value string) error {
+ return vc.executor.SetVitessMetadata(ctx, name, value)
}
-func (vc *vcursorImpl) ThrottleApp(ctx context.Context, throttledAppRule *topodatapb.ThrottledAppRule) (err error) {
+func (vc *VCursorImpl) ThrottleApp(ctx context.Context, throttledAppRule *topodatapb.ThrottledAppRule) (err error) {
if throttledAppRule == nil {
return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ThrottleApp: nil rule")
}
@@ -1231,14 +1440,12 @@ func (vc *vcursorImpl) ThrottleApp(ctx context.Context, throttledAppRule *topoda
throttlerConfig.ThrottledApps = make(map[string]*topodatapb.ThrottledAppRule)
}
if req.ThrottledApp != nil && req.ThrottledApp.Name != "" {
- // TODO(shlomi) in v22: replace the following line with the commented out block
- throttlerConfig.ThrottledApps[req.ThrottledApp.Name] = req.ThrottledApp
- // timeNow := time.Now()
- // if protoutil.TimeFromProto(req.ThrottledApp.ExpiresAt).After(timeNow) {
- // throttlerConfig.ThrottledApps[req.ThrottledApp.Name] = req.ThrottledApp
- // } else {
- // delete(throttlerConfig.ThrottledApps, req.ThrottledApp.Name)
- // }
+ timeNow := time.Now()
+ if protoutil.TimeFromProto(req.ThrottledApp.ExpiresAt).After(timeNow) {
+ throttlerConfig.ThrottledApps[req.ThrottledApp.Name] = req.ThrottledApp
+ } else {
+ delete(throttlerConfig.ThrottledApps, req.ThrottledApp.Name)
+ }
}
return throttlerConfig
}
@@ -1266,112 +1473,60 @@ func (vc *vcursorImpl) ThrottleApp(ctx context.Context, throttledAppRule *topoda
return err
}
-func (vc *vcursorImpl) CanUseSetVar() bool {
- return vc.Environment().Parser().IsMySQL80AndAbove() && setVarEnabled
+func (vc *VCursorImpl) CanUseSetVar() bool {
+ return vc.Environment().Parser().IsMySQL80AndAbove() && vc.config.SetVarEnabled
}
-func (vc *vcursorImpl) ReleaseLock(ctx context.Context) error {
- return vc.executor.ReleaseLock(ctx, vc.safeSession)
+func (vc *VCursorImpl) ReleaseLock(ctx context.Context) error {
+ return vc.executor.ReleaseLock(ctx, vc.SafeSession)
}
-func (vc *vcursorImpl) cloneWithAutocommitSession() *vcursorImpl {
- safeSession := NewAutocommitSession(vc.safeSession.Session)
- safeSession.logging = vc.safeSession.logging
- return &vcursorImpl{
- safeSession: safeSession,
- keyspace: vc.keyspace,
- tabletType: vc.tabletType,
- destination: vc.destination,
- marginComments: vc.marginComments,
- executor: vc.executor,
- logStats: vc.logStats,
- collation: vc.collation,
- resolver: vc.resolver,
- vschema: vc.vschema,
- vm: vc.vm,
- topoServer: vc.topoServer,
- warnShardedOnly: vc.warnShardedOnly,
- pv: vc.pv,
- }
+func (vc *VCursorImpl) VExplainLogging() {
+ vc.SafeSession.EnableLogging(vc.Environment().Parser())
}
-func (vc *vcursorImpl) VExplainLogging() {
- vc.safeSession.EnableLogging(vc.Environment().Parser())
+func (vc *VCursorImpl) GetVExplainLogs() []engine.ExecuteEntry {
+ return vc.SafeSession.GetLogs()
}
-func (vc *vcursorImpl) GetVExplainLogs() []engine.ExecuteEntry {
- return vc.safeSession.logging.GetLogs()
-}
-func (vc *vcursorImpl) FindRoutedShard(keyspace, shard string) (keyspaceName string, err error) {
+func (vc *VCursorImpl) FindRoutedShard(keyspace, shard string) (keyspaceName string, err error) {
return vc.vschema.FindRoutedShard(keyspace, shard)
}
-func (vc *vcursorImpl) IsViewsEnabled() bool {
- return enableViews
+func (vc *VCursorImpl) IsViewsEnabled() bool {
+ return vc.config.EnableViews
}
-func (vc *vcursorImpl) GetUDV(name string) *querypb.BindVariable {
- return vc.safeSession.GetUDV(name)
+func (vc *VCursorImpl) GetUDV(name string) *querypb.BindVariable {
+ return vc.SafeSession.GetUDV(name)
}
-func (vc *vcursorImpl) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) {
- return vc.executor.planPrepareStmt(ctx, vc, query)
+func (vc *VCursorImpl) PlanPrepareStatement(ctx context.Context, query string) (*engine.Plan, sqlparser.Statement, error) {
+ return vc.executor.PlanPrepareStmt(ctx, vc, query)
}
-func (vc *vcursorImpl) ClearPrepareData(name string) {
- delete(vc.safeSession.PrepareStatement, name)
+func (vc *VCursorImpl) ClearPrepareData(name string) {
+ delete(vc.SafeSession.PrepareStatement, name)
}
-func (vc *vcursorImpl) StorePrepareData(stmtName string, prepareData *vtgatepb.PrepareData) {
- vc.safeSession.StorePrepareData(stmtName, prepareData)
+func (vc *VCursorImpl) StorePrepareData(stmtName string, prepareData *vtgatepb.PrepareData) {
+ vc.SafeSession.StorePrepareData(stmtName, prepareData)
}
-func (vc *vcursorImpl) GetPrepareData(stmtName string) *vtgatepb.PrepareData {
- return vc.safeSession.GetPrepareData(stmtName)
+func (vc *VCursorImpl) GetPrepareData(stmtName string) *vtgatepb.PrepareData {
+ return vc.SafeSession.GetPrepareData(stmtName)
}
-func (vc *vcursorImpl) GetWarmingReadsPercent() int {
- return vc.warmingReadsPercent
+func (vc *VCursorImpl) GetWarmingReadsPercent() int {
+ return vc.config.WarmingReadsPercent
}
-func (vc *vcursorImpl) GetWarmingReadsChannel() chan bool {
- return vc.warmingReadsChannel
-}
-
-func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor {
- callerId := callerid.EffectiveCallerIDFromContext(ctx)
- immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx)
-
- timedCtx, _ := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) // nolint
- clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId)
-
- v := &vcursorImpl{
- safeSession: NewAutocommitSession(vc.safeSession.Session),
- keyspace: vc.keyspace,
- tabletType: topodatapb.TabletType_REPLICA,
- destination: vc.destination,
- marginComments: vc.marginComments,
- executor: vc.executor,
- resolver: vc.resolver,
- topoServer: vc.topoServer,
- logStats: &logstats.LogStats{Ctx: clonedCtx},
- collation: vc.collation,
- ignoreMaxMemoryRows: vc.ignoreMaxMemoryRows,
- vschema: vc.vschema,
- vm: vc.vm,
- semTable: vc.semTable,
- warnShardedOnly: vc.warnShardedOnly,
- warnings: vc.warnings,
- pv: vc.pv,
- }
-
- v.marginComments.Trailing += "/* warming read */"
-
- return v
+func (vc *VCursorImpl) GetWarmingReadsChannel() chan bool {
+ return vc.config.WarmingReadsChannel
}
// UpdateForeignKeyChecksState updates the foreign key checks state of the vcursor.
-func (vc *vcursorImpl) UpdateForeignKeyChecksState(fkStateFromQuery *bool) {
+func (vc *VCursorImpl) UpdateForeignKeyChecksState(fkStateFromQuery *bool) {
// Initialize the state to unspecified.
vc.fkChecksState = nil
// If the query has a SET_VAR optimizer hint that explicitly sets the foreign key checks state,
@@ -1381,10 +1536,36 @@ func (vc *vcursorImpl) UpdateForeignKeyChecksState(fkStateFromQuery *bool) {
return
}
// If the query doesn't have anything, then we consult the session state.
- vc.fkChecksState = vc.safeSession.ForeignKeyChecks()
+ vc.fkChecksState = vc.SafeSession.ForeignKeyChecks()
}
// GetForeignKeyChecksState gets the stored foreign key checks state in the vcursor.
-func (vc *vcursorImpl) GetForeignKeyChecksState() *bool {
+func (vc *VCursorImpl) GetForeignKeyChecksState() *bool {
return vc.fkChecksState
}
+
+// RecordMirrorStats is used to record stats about a mirror query.
+func (vc *VCursorImpl) RecordMirrorStats(sourceExecTime, targetExecTime time.Duration, targetErr error) {
+ vc.logStats.MirrorSourceExecuteTime = sourceExecTime
+ vc.logStats.MirrorTargetExecuteTime = targetExecTime
+ vc.logStats.MirrorTargetError = targetErr
+}
+
+func (vc *VCursorImpl) GetMarginComments() sqlparser.MarginComments {
+ return vc.marginComments
+}
+
+func (vc *VCursorImpl) CachePlan() bool {
+ return vc.SafeSession.CachePlan()
+}
+
+func (vc *VCursorImpl) GetContextWithTimeOut(ctx context.Context) (context.Context, context.CancelFunc) {
+ if vc.queryTimeout == 0 {
+ return ctx, func() {}
+ }
+ return context.WithTimeout(ctx, vc.queryTimeout)
+}
+
+func (vc *VCursorImpl) IgnoreMaxMemoryRows() bool {
+ return vc.ignoreMaxMemoryRows
+}
diff --git a/go/vt/vtgate/executorcontext/vcursor_impl_test.go b/go/vt/vtgate/executorcontext/vcursor_impl_test.go
new file mode 100644
index 00000000000..16d2c03bf1c
--- /dev/null
+++ b/go/vt/vtgate/executorcontext/vcursor_impl_test.go
@@ -0,0 +1,493 @@
+/*
+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 executorcontext
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "strconv"
+ "strings"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/mysql/collations"
+ "vitess.io/vitess/go/sqltypes"
+ binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vtgate/engine"
+ "vitess.io/vitess/go/vt/vtgate/vtgateservice"
+
+ "vitess.io/vitess/go/vt/key"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/srvtopo"
+ "vitess.io/vitess/go/vt/vtgate/logstats"
+ "vitess.io/vitess/go/vt/vtgate/vindexes"
+
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+)
+
+var _ VSchemaOperator = (*fakeVSchemaOperator)(nil)
+
+type fakeVSchemaOperator struct {
+ vschema *vindexes.VSchema
+}
+
+func (f fakeVSchemaOperator) GetCurrentSrvVschema() *vschemapb.SrvVSchema {
+ panic("implement me")
+}
+
+func (f fakeVSchemaOperator) UpdateVSchema(ctx context.Context, ksName string, vschema *vschemapb.SrvVSchema) error {
+ panic("implement me")
+}
+
+func TestDestinationKeyspace(t *testing.T) {
+ ks1 := &vindexes.Keyspace{
+ Name: "ks1",
+ Sharded: false,
+ }
+ ks1Schema := &vindexes.KeyspaceSchema{
+ Keyspace: ks1,
+ Tables: nil,
+ Vindexes: nil,
+ Error: nil,
+ }
+ ks2 := &vindexes.Keyspace{
+ Name: "ks2",
+ Sharded: false,
+ }
+ ks2Schema := &vindexes.KeyspaceSchema{
+ Keyspace: ks2,
+ Tables: nil,
+ Vindexes: nil,
+ Error: nil,
+ }
+ vschemaWith2KS := &vindexes.VSchema{
+ Keyspaces: map[string]*vindexes.KeyspaceSchema{
+ ks1.Name: ks1Schema,
+ ks2.Name: ks2Schema,
+ },
+ }
+
+ vschemaWith1KS := &vindexes.VSchema{
+ Keyspaces: map[string]*vindexes.KeyspaceSchema{
+ ks1.Name: ks1Schema,
+ },
+ }
+
+ type testCase struct {
+ vschema *vindexes.VSchema
+ targetString, qualifier string
+ expectedError string
+ expectedKeyspace string
+ expectedDest key.Destination
+ expectedTabletType topodatapb.TabletType
+ }
+
+ tests := []testCase{{
+ vschema: vschemaWith1KS,
+ targetString: "",
+ qualifier: "",
+ expectedKeyspace: ks1.Name,
+ expectedDest: nil,
+ expectedTabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1",
+ qualifier: "",
+ expectedKeyspace: ks1.Name,
+ expectedDest: nil,
+ expectedTabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1:-80",
+ qualifier: "",
+ expectedKeyspace: ks1.Name,
+ expectedDest: key.DestinationShard("-80"),
+ expectedTabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1@replica",
+ qualifier: "",
+ expectedKeyspace: ks1.Name,
+ expectedDest: nil,
+ expectedTabletType: topodatapb.TabletType_REPLICA,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1:-80@replica",
+ qualifier: "",
+ expectedKeyspace: ks1.Name,
+ expectedDest: key.DestinationShard("-80"),
+ expectedTabletType: topodatapb.TabletType_REPLICA,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "",
+ qualifier: "ks1",
+ expectedKeyspace: ks1.Name,
+ expectedDest: nil,
+ expectedTabletType: topodatapb.TabletType_PRIMARY,
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks2",
+ qualifier: "",
+ expectedError: "VT05003: unknown database 'ks2' in vschema",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks2:-80",
+ qualifier: "",
+ expectedError: "VT05003: unknown database 'ks2' in vschema",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "",
+ qualifier: "ks2",
+ expectedError: "VT05003: unknown database 'ks2' in vschema",
+ }, {
+ vschema: vschemaWith2KS,
+ targetString: "",
+ expectedError: ErrNoKeyspace.Error(),
+ }}
+
+ for i, tc := range tests {
+ t.Run(strconv.Itoa(i)+tc.targetString, func(t *testing.T) {
+ session := NewSafeSession(&vtgatepb.Session{TargetString: tc.targetString})
+ impl, _ := NewVCursorImpl(session, sqlparser.MarginComments{}, nil, nil,
+ &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, nil, nil,
+ fakeObserver{}, VCursorConfig{
+ DefaultTabletType: topodatapb.TabletType_PRIMARY,
+ })
+ impl.vschema = tc.vschema
+ dest, keyspace, tabletType, err := impl.TargetDestination(tc.qualifier)
+ if tc.expectedError == "" {
+ require.NoError(t, err)
+ require.Equal(t, tc.expectedDest, dest)
+ require.Equal(t, tc.expectedKeyspace, keyspace.Name)
+ require.Equal(t, tc.expectedTabletType, tabletType)
+ } else {
+ require.EqualError(t, err, tc.expectedError)
+ }
+ })
+ }
+}
+
+var (
+ ks1 = &vindexes.Keyspace{Name: "ks1"}
+ ks1Schema = &vindexes.KeyspaceSchema{Keyspace: ks1}
+ ks2 = &vindexes.Keyspace{Name: "ks2"}
+ ks2Schema = &vindexes.KeyspaceSchema{Keyspace: ks2}
+ vschemaWith1KS = &vindexes.VSchema{
+ Keyspaces: map[string]*vindexes.KeyspaceSchema{
+ ks1.Name: ks1Schema,
+ },
+ }
+)
+
+var vschemaWith2KS = &vindexes.VSchema{
+ Keyspaces: map[string]*vindexes.KeyspaceSchema{
+ ks1.Name: ks1Schema,
+ ks2.Name: ks2Schema,
+ },
+}
+
+func TestSetTarget(t *testing.T) {
+ type testCase struct {
+ vschema *vindexes.VSchema
+ targetString string
+ expectedError string
+ }
+
+ tests := []testCase{{
+ vschema: vschemaWith2KS,
+ targetString: "",
+ }, {
+ vschema: vschemaWith2KS,
+ targetString: "ks1",
+ }, {
+ vschema: vschemaWith2KS,
+ targetString: "ks2",
+ }, {
+ vschema: vschemaWith2KS,
+ targetString: "ks3",
+ expectedError: "VT05003: unknown database 'ks3' in vschema",
+ }, {
+ vschema: vschemaWith2KS,
+ targetString: "ks2@replica",
+ expectedError: "can't execute the given command because you have an active transaction",
+ }}
+
+ for i, tc := range tests {
+ t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) {
+ cfg := VCursorConfig{DefaultTabletType: topodatapb.TabletType_PRIMARY}
+ vc, _ := NewVCursorImpl(NewSafeSession(&vtgatepb.Session{InTransaction: true}), sqlparser.MarginComments{}, nil, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, nil, nil, fakeObserver{}, cfg)
+ vc.vschema = tc.vschema
+ err := vc.SetTarget(tc.targetString)
+ if tc.expectedError == "" {
+ require.NoError(t, err)
+ require.Equal(t, vc.SafeSession.TargetString, tc.targetString)
+ } else {
+ require.EqualError(t, err, tc.expectedError)
+ }
+ })
+ }
+}
+
+func TestKeyForPlan(t *testing.T) {
+ type testCase struct {
+ vschema *vindexes.VSchema
+ targetString string
+ expectedPlanPrefixKey string
+ }
+
+ tests := []testCase{{
+ vschema: vschemaWith1KS,
+ targetString: "",
+ expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1@replica",
+ expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1:-80",
+ expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+DestinationShard(-80)+Query:SELECT 1",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1[deadbeef]",
+ expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+KsIDsResolved:80-+Query:SELECT 1",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "",
+ expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
+ }, {
+ vschema: vschemaWith1KS,
+ targetString: "ks1@replica",
+ expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
+ }}
+
+ for i, tc := range tests {
+ t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) {
+ ss := NewSafeSession(&vtgatepb.Session{InTransaction: false})
+ ss.SetTargetString(tc.targetString)
+ cfg := VCursorConfig{
+ Collation: collations.CollationUtf8mb4ID,
+ DefaultTabletType: topodatapb.TabletType_PRIMARY,
+ }
+ vc, err := NewVCursorImpl(ss, sqlparser.MarginComments{}, &fakeExecutor{}, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, srvtopo.NewResolver(&FakeTopoServer{}, nil, ""), nil, fakeObserver{}, cfg)
+ require.NoError(t, err)
+ vc.vschema = tc.vschema
+
+ var buf strings.Builder
+ vc.KeyForPlan(context.Background(), "SELECT 1", &buf)
+ require.Equal(t, tc.expectedPlanPrefixKey, buf.String())
+ })
+ }
+}
+
+func TestFirstSortedKeyspace(t *testing.T) {
+ ks1Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "xks1"}}
+ ks2Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "aks2"}}
+ ks3Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "aks1"}}
+ vschemaWith2KS := &vindexes.VSchema{
+ Keyspaces: map[string]*vindexes.KeyspaceSchema{
+ ks1Schema.Keyspace.Name: ks1Schema,
+ ks2Schema.Keyspace.Name: ks2Schema,
+ ks3Schema.Keyspace.Name: ks3Schema,
+ },
+ }
+
+ vc, err := NewVCursorImpl(NewSafeSession(nil), sqlparser.MarginComments{}, nil, nil, &fakeVSchemaOperator{vschema: vschemaWith2KS}, vschemaWith2KS, srvtopo.NewResolver(&FakeTopoServer{}, nil, ""), nil, fakeObserver{}, VCursorConfig{})
+ require.NoError(t, err)
+ ks, err := vc.FirstSortedKeyspace()
+ require.NoError(t, err)
+ require.Equal(t, ks3Schema.Keyspace, ks)
+}
+
+// TestSetExecQueryTimeout tests the SetExecQueryTimeout method.
+// Validates the timeout value is set based on override rule.
+func TestSetExecQueryTimeout(t *testing.T) {
+ safeSession := NewSafeSession(nil)
+ vc, err := NewVCursorImpl(safeSession, sqlparser.MarginComments{}, nil, nil, nil, &vindexes.VSchema{}, nil, nil, fakeObserver{}, VCursorConfig{
+ // flag timeout
+ QueryTimeout: 20,
+ })
+ require.NoError(t, err)
+
+ vc.SetExecQueryTimeout(nil)
+ require.Equal(t, 20*time.Millisecond, vc.queryTimeout)
+ require.NotNil(t, safeSession.Options.Timeout)
+ require.EqualValues(t, 20, safeSession.Options.GetAuthoritativeTimeout())
+
+ // session timeout
+ safeSession.SetQueryTimeout(40)
+ vc.SetExecQueryTimeout(nil)
+ require.Equal(t, 40*time.Millisecond, vc.queryTimeout)
+ require.NotNil(t, safeSession.Options.Timeout)
+ require.EqualValues(t, 40, safeSession.Options.GetAuthoritativeTimeout())
+
+ // query hint timeout
+ timeoutQueryHint := 60
+ vc.SetExecQueryTimeout(&timeoutQueryHint)
+ require.Equal(t, 60*time.Millisecond, vc.queryTimeout)
+ require.NotNil(t, safeSession.Options.Timeout)
+ require.EqualValues(t, 60, safeSession.Options.GetAuthoritativeTimeout())
+
+ // query hint timeout - infinite
+ timeoutQueryHint = 0
+ vc.SetExecQueryTimeout(&timeoutQueryHint)
+ require.Equal(t, 0*time.Millisecond, vc.queryTimeout)
+ require.NotNil(t, safeSession.Options.Timeout)
+ require.EqualValues(t, 0, safeSession.Options.GetAuthoritativeTimeout())
+
+ // reset flag timeout
+ vc.config.QueryTimeout = 0
+ safeSession.SetQueryTimeout(0)
+ vc.SetExecQueryTimeout(nil)
+ require.Equal(t, 0*time.Millisecond, vc.queryTimeout)
+ // this should be reset.
+ require.Nil(t, safeSession.Options.Timeout)
+}
+
+func TestRecordMirrorStats(t *testing.T) {
+ safeSession := NewSafeSession(nil)
+ logStats := logstats.NewLogStats(context.Background(), t.Name(), "select 1", "", nil)
+ vc, err := NewVCursorImpl(safeSession, sqlparser.MarginComments{}, nil, logStats, nil, &vindexes.VSchema{}, nil, nil, fakeObserver{}, VCursorConfig{})
+ require.NoError(t, err)
+
+ require.Zero(t, logStats.MirrorSourceExecuteTime)
+ require.Zero(t, logStats.MirrorTargetExecuteTime)
+ require.Nil(t, logStats.MirrorTargetError)
+
+ vc.RecordMirrorStats(10*time.Millisecond, 20*time.Millisecond, errors.New("test error"))
+
+ require.Equal(t, 10*time.Millisecond, logStats.MirrorSourceExecuteTime)
+ require.Equal(t, 20*time.Millisecond, logStats.MirrorTargetExecuteTime)
+ require.ErrorContains(t, logStats.MirrorTargetError, "test error")
+}
+
+type fakeExecutor struct{}
+
+func (f fakeExecutor) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConnection, method string, session *SafeSession, s string, vars map[string]*querypb.BindVariable) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ExecuteMultiShard(ctx context.Context, primitive engine.Primitive, rss []*srvtopo.ResolvedShard, queries []*querypb.BoundQuery, session *SafeSession, autocommit bool, ignoreMaxMemoryRows bool, resultsObserver ResultsObserver) (qr *sqltypes.Result, errs []error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) StreamExecuteMulti(ctx context.Context, primitive engine.Primitive, query string, rss []*srvtopo.ResolvedShard, vars []map[string]*querypb.BindVariable, session *SafeSession, autocommit bool, callback func(reply *sqltypes.Result) error, observer ResultsObserver) []error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) Commit(ctx context.Context, safeSession *SafeSession) error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ExecuteMessageStream(ctx context.Context, rss []*srvtopo.ResolvedShard, name string, callback func(*sqltypes.Result) error) error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ExecuteVStream(ctx context.Context, rss []*srvtopo.ResolvedShard, filter *binlogdatapb.Filter, gtid string, callback func(evs []*binlogdatapb.VEvent) error) error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ReleaseLock(ctx context.Context, session *SafeSession) error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ShowVitessReplicationStatus(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ShowShards(ctx context.Context, filter *sqlparser.ShowFilter, destTabletType topodatapb.TabletType) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ShowTablets(filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ShowVitessMetadata(ctx context.Context, filter *sqlparser.ShowFilter) (*sqltypes.Result, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) SetVitessMetadata(ctx context.Context, name, value string) error {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) ParseDestinationTarget(targetString string) (string, topodatapb.TabletType, key.Destination, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) VSchema() *vindexes.VSchema {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) PlanPrepareStmt(ctx context.Context, vcursor *VCursorImpl, query string) (*engine.Plan, sqlparser.Statement, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) Environment() *vtenv.Environment {
+ return vtenv.NewTestEnv()
+}
+
+func (f fakeExecutor) ReadTransaction(ctx context.Context, transactionID string) (*querypb.TransactionMetadata, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) UnresolvedTransactions(ctx context.Context, targets []*querypb.Target) ([]*querypb.TransactionMetadata, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
+func (f fakeExecutor) AddWarningCount(name string, value int64) {
+ // TODO implement me
+ panic("implement me")
+}
+
+var _ iExecute = (*fakeExecutor)(nil)
+
+type fakeObserver struct{}
+
+func (f fakeObserver) Observe(*sqltypes.Result) {
+}
+
+var _ ResultsObserver = (*fakeObserver)(nil)
diff --git a/go/vt/vtgate/legacy_scatter_conn_test.go b/go/vt/vtgate/legacy_scatter_conn_test.go
index 7ada1c3ac31..0d49e7b7bd9 100644
--- a/go/vt/vtgate/legacy_scatter_conn_test.go
+++ b/go/vt/vtgate/legacy_scatter_conn_test.go
@@ -26,6 +26,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
@@ -99,7 +101,7 @@ func TestLegacyExecuteFailOnAutocommit(t *testing.T) {
},
Autocommit: false,
}
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, NewSafeSession(session), true /*autocommit*/, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, econtext.NewSafeSession(session), true /*autocommit*/, false, nullResultsObserver{})
err := vterrors.Aggregate(errs)
require.Error(t, err)
require.Contains(t, err.Error(), "in autocommit mode, transactionID should be zero but was: 123")
@@ -123,7 +125,7 @@ func TestScatterConnExecuteMulti(t *testing.T) {
}
}
- qr, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, NewSafeSession(nil), false /*autocommit*/, false)
+ qr, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, econtext.NewSafeSession(nil), false /*autocommit*/, false, nullResultsObserver{})
return qr, vterrors.Aggregate(errs)
})
}
@@ -138,12 +140,12 @@ func TestScatterConnStreamExecuteMulti(t *testing.T) {
bvs := make([]map[string]*querypb.BindVariable, len(rss))
qr := new(sqltypes.Result)
var mu sync.Mutex
- errors := sc.StreamExecuteMulti(ctx, nil, "query", rss, bvs, NewSafeSession(&vtgatepb.Session{InTransaction: true}), true /* autocommit */, func(r *sqltypes.Result) error {
+ errors := sc.StreamExecuteMulti(ctx, nil, "query", rss, bvs, econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true}), true /* autocommit */, func(r *sqltypes.Result) error {
mu.Lock()
defer mu.Unlock()
qr.AppendResult(r)
return nil
- })
+ }, nullResultsObserver{})
return qr, vterrors.Aggregate(errors)
})
}
@@ -280,7 +282,7 @@ func TestMaxMemoryRows(t *testing.T) {
[]key.Destination{key.DestinationShard("0"), key.DestinationShard("1")})
require.NoError(t, err)
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
queries := []*querypb.BoundQuery{{
Sql: "query1",
BindVariables: map[string]*querypb.BindVariable{},
@@ -310,7 +312,7 @@ func TestMaxMemoryRows(t *testing.T) {
sbc0.SetResults([]*sqltypes.Result{tworows, tworows})
sbc1.SetResults([]*sqltypes.Result{tworows, tworows})
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, test.ignoreMaxMemoryRows)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, test.ignoreMaxMemoryRows, nullResultsObserver{})
if test.ignoreMaxMemoryRows {
require.NoError(t, err)
} else {
@@ -328,7 +330,7 @@ func TestLegaceHealthCheckFailsOnReservedConnections(t *testing.T) {
res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
destinations := []key.Destination{key.DestinationShard("0")}
rss, _, err := res.ResolveDestinations(ctx, keyspace, topodatapb.TabletType_REPLICA, nil, destinations)
require.NoError(t, err)
@@ -342,16 +344,16 @@ func TestLegaceHealthCheckFailsOnReservedConnections(t *testing.T) {
})
}
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false, nullResultsObserver{})
require.Error(t, vterrors.Aggregate(errs))
}
-func executeOnShards(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *SafeSession, destinations []key.Destination) {
+func executeOnShards(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.Destination) {
t.Helper()
require.Empty(t, executeOnShardsReturnsErr(t, ctx, res, keyspace, sc, session, destinations))
}
-func executeOnShardsReturnsErr(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *SafeSession, destinations []key.Destination) error {
+func executeOnShardsReturnsErr(t *testing.T, ctx context.Context, res *srvtopo.Resolver, keyspace string, sc *ScatterConn, session *econtext.SafeSession, destinations []key.Destination) error {
t.Helper()
rss, _, err := res.ResolveDestinations(ctx, keyspace, topodatapb.TabletType_REPLICA, nil, destinations)
require.NoError(t, err)
@@ -365,10 +367,21 @@ func executeOnShardsReturnsErr(t *testing.T, ctx context.Context, res *srvtopo.R
})
}
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false, nullResultsObserver{})
return vterrors.Aggregate(errs)
}
+type recordingResultsObserver struct {
+ mu sync.Mutex
+ recorded []*sqltypes.Result
+}
+
+func (o *recordingResultsObserver) Observe(result *sqltypes.Result) {
+ mu.Lock()
+ o.recorded = append(o.recorded, result)
+ mu.Unlock()
+}
+
func TestMultiExecs(t *testing.T) {
ctx := utils.LeakCheckContext(t)
createSandbox("TestMultiExecs")
@@ -409,9 +422,17 @@ func TestMultiExecs(t *testing.T) {
},
},
}
+ results := []*sqltypes.Result{
+ {Info: "r0"},
+ {Info: "r1"},
+ }
+ sbc0.SetResults(results[0:1])
+ sbc1.SetResults(results[1:2])
+
+ observer := recordingResultsObserver{}
- session := NewSafeSession(&vtgatepb.Session{})
- _, err := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{})
+ _, err := sc.ExecuteMultiShard(ctx, nil, rss, queries, session, false, false, &observer)
require.NoError(t, vterrors.Aggregate(err))
if len(sbc0.Queries) == 0 || len(sbc1.Queries) == 0 {
t.Fatalf("didn't get expected query")
@@ -428,8 +449,12 @@ func TestMultiExecs(t *testing.T) {
if !reflect.DeepEqual(sbc1.Queries[0].BindVariables, wantVars1) {
t.Errorf("got %+v, want %+v", sbc0.Queries[0].BindVariables, wantVars1)
}
+ assert.ElementsMatch(t, results, observer.recorded)
+
sbc0.Queries = nil
sbc1.Queries = nil
+ sbc0.SetResults(results[0:1])
+ sbc1.SetResults(results[1:2])
rss = []*srvtopo.ResolvedShard{
{
@@ -455,15 +480,18 @@ func TestMultiExecs(t *testing.T) {
"bv1": sqltypes.Int64BindVariable(1),
},
}
+
+ observer = recordingResultsObserver{}
_ = sc.StreamExecuteMulti(ctx, nil, "query", rss, bvs, session, false /* autocommit */, func(*sqltypes.Result) error {
return nil
- })
+ }, &observer)
if !reflect.DeepEqual(sbc0.Queries[0].BindVariables, wantVars0) {
t.Errorf("got %+v, want %+v", sbc0.Queries[0].BindVariables, wantVars0)
}
if !reflect.DeepEqual(sbc1.Queries[0].BindVariables, wantVars1) {
t.Errorf("got %+v, want %+v", sbc0.Queries[0].BindVariables, wantVars1)
}
+ assert.ElementsMatch(t, results, observer.recorded)
}
func TestScatterConnSingleDB(t *testing.T) {
@@ -485,29 +513,29 @@ func TestScatterConnSingleDB(t *testing.T) {
want := "multi-db transaction attempted"
// TransactionMode_SINGLE in session
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE})
queries := []*querypb.BoundQuery{{Sql: "query1"}}
- _, errors := sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ _, errors := sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errors)
- _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
require.Error(t, errors[0])
assert.Contains(t, errors[0].Error(), want)
// TransactionMode_SINGLE in txconn
sc.txConn.mode = vtgatepb.TransactionMode_SINGLE
- session = NewSafeSession(&vtgatepb.Session{InTransaction: true})
- _, errors = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session = econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ _, errors = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errors)
- _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
require.Error(t, errors[0])
assert.Contains(t, errors[0].Error(), want)
// TransactionMode_MULTI in txconn. Should not fail.
sc.txConn.mode = vtgatepb.TransactionMode_MULTI
- session = NewSafeSession(&vtgatepb.Session{InTransaction: true})
- _, errors = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session = econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ _, errors = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errors)
- _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ _, errors = sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errors)
}
@@ -575,7 +603,7 @@ func TestReservePrequeries(t *testing.T) {
res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa")
- session := NewSafeSession(&vtgatepb.Session{
+ session := econtext.NewSafeSession(&vtgatepb.Session{
InTransaction: false,
InReservedConn: true,
SystemVariables: map[string]string{
@@ -594,6 +622,6 @@ func newTestScatterConn(ctx context.Context, hc discovery.HealthCheck, serv srvt
// in '-cells_to_watch' command line parameter, which is
// empty by default. So it's unused in this test, set to nil.
gw := NewTabletGateway(ctx, hc, serv, cell)
- tc := NewTxConn(gw, vtgatepb.TransactionMode_TWOPC)
+ tc := NewTxConn(gw, vtgatepb.TransactionMode_MULTI)
return NewScatterConn("", tc, gw)
}
diff --git a/go/vt/vtgate/logstats/logstats.go b/go/vt/vtgate/logstats/logstats.go
index 8f8ba41e3cd..fdc0e69c8db 100644
--- a/go/vt/vtgate/logstats/logstats.go
+++ b/go/vt/vtgate/logstats/logstats.go
@@ -33,25 +33,28 @@ import (
// LogStats records the stats for a single vtgate query
type LogStats struct {
- Ctx context.Context
- Method string
- TabletType string
- StmtType string
- SQL string
- BindVariables map[string]*querypb.BindVariable
- StartTime time.Time
- EndTime time.Time
- ShardQueries uint64
- RowsAffected uint64
- RowsReturned uint64
- PlanTime time.Duration
- ExecuteTime time.Duration
- CommitTime time.Duration
- Error error
- TablesUsed []string
- SessionUUID string
- CachedPlan bool
- ActiveKeyspace string // ActiveKeyspace is the selected keyspace `use ks`
+ Ctx context.Context
+ Method string
+ TabletType string
+ StmtType string
+ SQL string
+ BindVariables map[string]*querypb.BindVariable
+ StartTime time.Time
+ EndTime time.Time
+ ShardQueries uint64
+ RowsAffected uint64
+ RowsReturned uint64
+ PlanTime time.Duration
+ ExecuteTime time.Duration
+ CommitTime time.Duration
+ Error error
+ TablesUsed []string
+ SessionUUID string
+ CachedPlan bool
+ ActiveKeyspace string // ActiveKeyspace is the selected keyspace `use ks`
+ MirrorSourceExecuteTime time.Duration
+ MirrorTargetExecuteTime time.Duration
+ MirrorTargetError error
}
// NewLogStats constructs a new LogStats with supplied Method and ctx
@@ -116,6 +119,14 @@ func (stats *LogStats) RemoteAddrUsername() (string, string) {
return ci.RemoteAddr(), ci.Username()
}
+// MirorTargetErrorStr returns the mirror target error string or ""
+func (stats *LogStats) MirrorTargetErrorStr() string {
+ if stats.MirrorTargetError != nil {
+ return stats.MirrorTargetError.Error()
+ }
+ return ""
+}
+
// Logf formats the log record to the given writer, either as
// tab-separated list of logged fields or as JSON.
func (stats *LogStats) Logf(w io.Writer, params url.Values) error {
@@ -177,6 +188,12 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error {
log.Strings(stats.TablesUsed)
log.Key("ActiveKeyspace")
log.String(stats.ActiveKeyspace)
+ log.Key("MirrorSourceExecuteTime")
+ log.Duration(stats.MirrorSourceExecuteTime)
+ log.Key("MirrorTargetExecuteTime")
+ log.Duration(stats.MirrorTargetExecuteTime)
+ log.Key("MirrorTargetError")
+ log.String(stats.MirrorTargetErrorStr())
return log.Flush(w)
}
diff --git a/go/vt/vtgate/logstats/logstats_test.go b/go/vt/vtgate/logstats/logstats_test.go
index ae3c01e0f0b..872b34c6964 100644
--- a/go/vt/vtgate/logstats/logstats_test.go
+++ b/go/vt/vtgate/logstats/logstats_test.go
@@ -79,42 +79,42 @@ func TestLogStatsFormat(t *testing.T) {
{ // 0
redact: false,
format: "text",
- expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\n",
+ expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\t0.000000\t0.000000\t\"\"\n",
bindVars: intBindVar,
}, { // 1
redact: true,
format: "text",
- expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t\"[REDACTED]\"\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\n",
+ expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t\"[REDACTED]\"\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\t0.000000\t0.000000\t\"\"\n",
bindVars: intBindVar,
}, { // 2
redact: false,
format: "json",
- expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":{\"intVal\":{\"type\":\"INT64\",\"value\":1}},\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
+ expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":{\"intVal\":{\"type\":\"INT64\",\"value\":1}},\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"MirrorSourceExecuteTime\":0,\"MirrorTargetError\":\"\",\"MirrorTargetExecuteTime\":0,\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
bindVars: intBindVar,
}, { // 3
redact: true,
format: "json",
- expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":\"[REDACTED]\",\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
+ expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":\"[REDACTED]\",\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"MirrorSourceExecuteTime\":0,\"MirrorTargetError\":\"\",\"MirrorTargetExecuteTime\":0,\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
bindVars: intBindVar,
}, { // 4
redact: false,
format: "text",
- expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t{\"strVal\": {\"type\": \"VARCHAR\", \"value\": \"abc\"}}\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\n",
+ expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t{\"strVal\": {\"type\": \"VARCHAR\", \"value\": \"abc\"}}\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\t0.000000\t0.000000\t\"\"\n",
bindVars: stringBindVar,
}, { // 5
redact: true,
format: "text",
- expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t\"[REDACTED]\"\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\n",
+ expected: "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1\"\t\"[REDACTED]\"\t0\t0\t\"\"\t\"PRIMARY\"\t\"suuid\"\tfalse\t[\"ks1.tbl1\",\"ks2.tbl2\"]\t\"db\"\t0.000000\t0.000000\t\"\"\n",
bindVars: stringBindVar,
}, { // 6
redact: false,
format: "json",
- expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":{\"strVal\":{\"type\":\"VARCHAR\",\"value\":\"abc\"}},\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
+ expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":{\"strVal\":{\"type\":\"VARCHAR\",\"value\":\"abc\"}},\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"MirrorSourceExecuteTime\":0,\"MirrorTargetError\":\"\",\"MirrorTargetExecuteTime\":0,\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
bindVars: stringBindVar,
}, { // 7
redact: true,
format: "json",
- expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":\"[REDACTED]\",\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
+ expected: "{\"ActiveKeyspace\":\"db\",\"BindVars\":\"[REDACTED]\",\"Cached Plan\":false,\"CommitTime\":0,\"Effective Caller\":\"\",\"End\":\"2017-01-01 01:02:04.000001\",\"Error\":\"\",\"ExecuteTime\":0,\"ImmediateCaller\":\"\",\"Method\":\"test\",\"MirrorSourceExecuteTime\":0,\"MirrorTargetError\":\"\",\"MirrorTargetExecuteTime\":0,\"PlanTime\":0,\"RemoteAddr\":\"\",\"RowsAffected\":0,\"SQL\":\"sql1\",\"SessionUUID\":\"suuid\",\"ShardQueries\":0,\"Start\":\"2017-01-01 01:02:03.000000\",\"StmtType\":\"\",\"TablesUsed\":[\"ks1.tbl1\",\"ks2.tbl2\"],\"TabletType\":\"PRIMARY\",\"TotalTime\":1.000001,\"Username\":\"\"}",
bindVars: stringBindVar,
},
}
@@ -156,12 +156,12 @@ func TestLogStatsFilter(t *testing.T) {
params := map[string][]string{"full": {}}
got := testFormat(t, logStats, params)
- want := "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\n"
+ want := "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\t0.000000\t0.000000\t\"\"\n"
assert.Equal(t, want, got)
streamlog.SetQueryLogFilterTag("LOG_THIS_QUERY")
got = testFormat(t, logStats, params)
- want = "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\n"
+ want = "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\t0.000000\t0.000000\t\"\"\n"
assert.Equal(t, want, got)
streamlog.SetQueryLogFilterTag("NOT_THIS_QUERY")
@@ -179,12 +179,12 @@ func TestLogStatsRowThreshold(t *testing.T) {
params := map[string][]string{"full": {}}
got := testFormat(t, logStats, params)
- want := "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\n"
+ want := "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\t0.000000\t0.000000\t\"\"\n"
assert.Equal(t, want, got)
streamlog.SetQueryLogRowThreshold(0)
got = testFormat(t, logStats, params)
- want = "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\n"
+ want = "test\t\t\t''\t''\t2017-01-01 01:02:03.000000\t2017-01-01 01:02:04.000001\t1.000001\t0.000000\t0.000000\t0.000000\t\t\"sql1 /* LOG_THIS_QUERY */\"\t{\"intVal\": {\"type\": \"INT64\", \"value\": 1}}\t0\t0\t\"\"\t\"\"\t\"\"\tfalse\t[]\t\"\"\t0.000000\t0.000000\t\"\"\n"
assert.Equal(t, want, got)
streamlog.SetQueryLogRowThreshold(1)
got = testFormat(t, logStats, params)
@@ -215,6 +215,18 @@ func TestLogStatsErrorStr(t *testing.T) {
}
}
+func TestLogStatsMirrorTargetErrorStr(t *testing.T) {
+ logStats := NewLogStats(context.Background(), "test", "sql1", "", map[string]*querypb.BindVariable{})
+ if logStats.MirrorTargetErrorStr() != "" {
+ t.Fatalf("should not get error in stats, but got: %s", logStats.ErrorStr())
+ }
+ errStr := "unknown error"
+ logStats.MirrorTargetError = errors.New(errStr)
+ if !strings.Contains(logStats.MirrorTargetErrorStr(), errStr) {
+ t.Fatalf("expect string '%s' in error message, but got: %s", errStr, logStats.ErrorStr())
+ }
+}
+
func TestLogStatsRemoteAddrUsername(t *testing.T) {
logStats := NewLogStats(context.Background(), "test", "sql1", "", map[string]*querypb.BindVariable{})
addr, user := logStats.RemoteAddrUsername()
diff --git a/go/vt/vtgate/plan_execute.go b/go/vt/vtgate/plan_execute.go
index 199892842ee..db7923c09f0 100644
--- a/go/vt/vtgate/plan_execute.go
+++ b/go/vt/vtgate/plan_execute.go
@@ -24,19 +24,21 @@ import (
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/log"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vtgate/logstats"
"vitess.io/vitess/go/vt/vtgate/vtgateservice"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
-type planExec func(ctx context.Context, plan *engine.Plan, vc *vcursorImpl, bindVars map[string]*querypb.BindVariable, startTime time.Time) error
+type planExec func(ctx context.Context, plan *engine.Plan, vc *econtext.VCursorImpl, bindVars map[string]*querypb.BindVariable, startTime time.Time) error
type txResult func(sqlparser.StatementType, *sqltypes.Result) error
+var vschemaWaitTimeout = 30 * time.Second
+
func waitForNewerVSchema(ctx context.Context, e *Executor, lastVSchemaCreated time.Time, timeout time.Duration) bool {
pollingInterval := 10 * time.Millisecond
waitCtx, cancel := context.WithTimeout(ctx, timeout)
@@ -55,10 +57,12 @@ func waitForNewerVSchema(ctx context.Context, e *Executor, lastVSchemaCreated ti
}
}
+const MaxBufferingRetries = 3
+
func (e *Executor) newExecute(
ctx context.Context,
mysqlCtx vtgateservice.MySQLConnection,
- safeSession *SafeSession,
+ safeSession *econtext.SafeSession,
sql string,
bindVars map[string]*querypb.BindVariable,
logStats *logstats.LogStats,
@@ -90,6 +94,7 @@ func (e *Executor) newExecute(
lastVSchemaCreated = vs.GetCreated()
result *sqltypes.Result
plan *engine.Plan
+ cancel context.CancelFunc
)
for try := 0; try < MaxBufferingRetries; try++ {
@@ -104,14 +109,17 @@ func (e *Executor) newExecute(
// based on the buffering configuration. This way we should be able to perform the max retries
// within the given window of time for most queries and we should not end up waiting too long
// after the traffic switch fails or the buffer window has ended, retrying old queries.
- timeout := e.resolver.scatterConn.gateway.buffer.GetConfig().MaxFailoverDuration / (MaxBufferingRetries - 1)
+ timeout := vschemaWaitTimeout
+ if e.resolver.scatterConn.gateway.buffer != nil && e.resolver.scatterConn.gateway.buffer.GetConfig() != nil {
+ timeout = e.resolver.scatterConn.gateway.buffer.GetConfig().MaxFailoverDuration / (MaxBufferingRetries - 1)
+ }
if waitForNewerVSchema(ctx, e, lastVSchemaCreated, timeout) {
vs = e.VSchema()
lastVSchemaCreated = vs.GetCreated()
}
}
- vcursor, err := newVCursorImpl(safeSession, comments, e, logStats, e.vm, vs, e.resolver.resolver, e.serv, e.warnShardedOnly, e.pv)
+ vcursor, err := econtext.NewVCursorImpl(safeSession, comments, e, logStats, e.vm, vs, e.resolver.resolver, e.serv, nullResultsObserver{}, e.vConfig)
if err != nil {
return err
}
@@ -140,6 +148,10 @@ func (e *Executor) newExecute(
safeSession.RecordWarning(warning)
}
+ // set the overall query timeout if it is not already set
+ ctx, cancel = vcursor.GetContextWithTimeOut(ctx)
+ defer cancel()
+
result, err = e.handleTransactions(ctx, mysqlCtx, safeSession, plan, logStats, vcursor, stmt)
if err != nil {
return err
@@ -214,10 +226,10 @@ func (e *Executor) newExecute(
func (e *Executor) handleTransactions(
ctx context.Context,
mysqlCtx vtgateservice.MySQLConnection,
- safeSession *SafeSession,
+ safeSession *econtext.SafeSession,
plan *engine.Plan,
logStats *logstats.LogStats,
- vcursor *vcursorImpl,
+ vcursor *econtext.VCursorImpl,
stmt sqlparser.Statement,
) (*sqltypes.Result, error) {
// We need to explicitly handle errors, and begin/commit/rollback, since these control transactions. Everything else
@@ -236,19 +248,19 @@ func (e *Executor) handleTransactions(
qr, err := e.handleSavepoint(ctx, safeSession, plan.Original, "Savepoint", logStats, func(_ string) (*sqltypes.Result, error) {
// Safely to ignore as there is no transaction.
return &sqltypes.Result{}, nil
- }, vcursor.ignoreMaxMemoryRows)
+ }, vcursor.IgnoreMaxMemoryRows())
return qr, err
case sqlparser.StmtSRollback:
qr, err := e.handleSavepoint(ctx, safeSession, plan.Original, "Rollback Savepoint", logStats, func(query string) (*sqltypes.Result, error) {
// Error as there is no transaction, so there is no savepoint that exists.
return nil, vterrors.NewErrorf(vtrpcpb.Code_NOT_FOUND, vterrors.SPDoesNotExist, "SAVEPOINT does not exist: %s", query)
- }, vcursor.ignoreMaxMemoryRows)
+ }, vcursor.IgnoreMaxMemoryRows())
return qr, err
case sqlparser.StmtRelease:
qr, err := e.handleSavepoint(ctx, safeSession, plan.Original, "Release Savepoint", logStats, func(query string) (*sqltypes.Result, error) {
// Error as there is no transaction, so there is no savepoint that exists.
return nil, vterrors.NewErrorf(vtrpcpb.Code_NOT_FOUND, vterrors.SPDoesNotExist, "SAVEPOINT does not exist: %s", query)
- }, vcursor.ignoreMaxMemoryRows)
+ }, vcursor.IgnoreMaxMemoryRows())
return qr, err
case sqlparser.StmtKill:
return e.handleKill(ctx, mysqlCtx, stmt, logStats)
@@ -256,7 +268,7 @@ func (e *Executor) handleTransactions(
return nil, nil
}
-func (e *Executor) startTxIfNecessary(ctx context.Context, safeSession *SafeSession) error {
+func (e *Executor) startTxIfNecessary(ctx context.Context, safeSession *econtext.SafeSession) error {
if !safeSession.Autocommit && !safeSession.InTransaction() {
if err := e.txConn.Begin(ctx, safeSession, nil); err != nil {
return err
@@ -265,7 +277,7 @@ func (e *Executor) startTxIfNecessary(ctx context.Context, safeSession *SafeSess
return nil
}
-func (e *Executor) insideTransaction(ctx context.Context, safeSession *SafeSession, logStats *logstats.LogStats, execPlan func() error) error {
+func (e *Executor) insideTransaction(ctx context.Context, safeSession *econtext.SafeSession, logStats *logstats.LogStats, execPlan func() error) error {
mustCommit := false
if safeSession.Autocommit && !safeSession.InTransaction() {
mustCommit = true
@@ -309,9 +321,9 @@ func (e *Executor) insideTransaction(ctx context.Context, safeSession *SafeSessi
func (e *Executor) executePlan(
ctx context.Context,
- safeSession *SafeSession,
+ safeSession *econtext.SafeSession,
plan *engine.Plan,
- vcursor *vcursorImpl,
+ vcursor *econtext.VCursorImpl,
bindVars map[string]*querypb.BindVariable,
logStats *logstats.LogStats,
execStart time.Time,
@@ -331,7 +343,7 @@ func (e *Executor) executePlan(
}
// rollbackExecIfNeeded rollbacks the partial execution if earlier it was detected that it needs partial query execution to be rolled back.
-func (e *Executor) rollbackExecIfNeeded(ctx context.Context, safeSession *SafeSession, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats, err error) error {
+func (e *Executor) rollbackExecIfNeeded(ctx context.Context, safeSession *econtext.SafeSession, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats, err error) error {
if safeSession.InTransaction() && safeSession.IsRollbackSet() {
rErr := e.rollbackPartialExec(ctx, safeSession, bindVars, logStats)
return vterrors.Wrap(err, rErr.Error())
@@ -342,7 +354,7 @@ func (e *Executor) rollbackExecIfNeeded(ctx context.Context, safeSession *SafeSe
// rollbackPartialExec rollbacks to the savepoint or rollbacks transaction based on the value set on SafeSession.rollbackOnPartialExec.
// Once, it is used the variable is reset.
// If it fails to rollback to the previous savepoint then, the transaction is forced to be rolled back.
-func (e *Executor) rollbackPartialExec(ctx context.Context, safeSession *SafeSession, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) error {
+func (e *Executor) rollbackPartialExec(ctx context.Context, safeSession *econtext.SafeSession, bindVars map[string]*querypb.BindVariable, logStats *logstats.LogStats) error {
var err error
var errMsg strings.Builder
@@ -356,8 +368,8 @@ func (e *Executor) rollbackPartialExec(ctx context.Context, safeSession *SafeSes
}
// needs to rollback only once.
- rQuery := safeSession.rollbackOnPartialExec
- if rQuery != txRollback {
+ rQuery := safeSession.GetRollbackOnPartialExec()
+ if rQuery != econtext.TxRollback {
safeSession.SavepointRollback()
_, _, err = e.execute(ctx, nil, safeSession, rQuery, bindVars, logStats)
// If no error, the revert is successful with the savepoint. Notify the reason as error to the client.
@@ -377,9 +389,9 @@ func (e *Executor) rollbackPartialExec(ctx context.Context, safeSession *SafeSes
return vterrors.New(vtrpcpb.Code_ABORTED, errMsg.String())
}
-func (e *Executor) setLogStats(logStats *logstats.LogStats, plan *engine.Plan, vcursor *vcursorImpl, execStart time.Time, err error, qr *sqltypes.Result) {
+func (e *Executor) setLogStats(logStats *logstats.LogStats, plan *engine.Plan, vcursor *econtext.VCursorImpl, execStart time.Time, err error, qr *sqltypes.Result) {
logStats.StmtType = plan.Type.String()
- logStats.ActiveKeyspace = vcursor.keyspace
+ logStats.ActiveKeyspace = vcursor.GetKeyspace()
logStats.TablesUsed = plan.TablesUsed
logStats.TabletType = vcursor.TabletType().String()
errCount := e.logExecutionEnd(logStats, execStart, plan, err, qr)
diff --git a/go/vt/vtgate/planbuilder/builder.go b/go/vt/vtgate/planbuilder/builder.go
index 5d1d4ecd622..ca4ccb7ac5a 100644
--- a/go/vt/vtgate/planbuilder/builder.go
+++ b/go/vt/vtgate/planbuilder/builder.go
@@ -28,6 +28,7 @@ import (
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/dynamicconfig"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
"vitess.io/vitess/go/vt/vtgate/vindexes"
@@ -63,6 +64,16 @@ func singleTable(ks, tbl string) string {
return fmt.Sprintf("%s.%s", ks, tbl)
}
+type staticConfig struct{}
+
+func (staticConfig) OnlineEnabled() bool {
+ return true
+}
+
+func (staticConfig) DirectEnabled() bool {
+ return true
+}
+
// TestBuilder builds a plan for a query based on the specified vschema.
// This method is only used from tests
func TestBuilder(query string, vschema plancontext.VSchema, keyspace string) (*engine.Plan, error) {
@@ -73,11 +84,14 @@ func TestBuilder(query string, vschema plancontext.VSchema, keyspace string) (*e
// Store the foreign key mode like we do for vcursor.
vw, isVw := vschema.(*vschemawrapper.VSchemaWrapper)
if isVw {
- fkState := sqlparser.ForeignKeyChecksState(stmt)
- if fkState != nil {
+ qh, err := sqlparser.BuildQueryHints(stmt)
+ if err != nil {
+ return nil, err
+ }
+ if qh.ForeignKeyChecks != nil {
// Restore the old volue of ForeignKeyChecksState to not interfere with the next test cases.
oldVal := vw.ForeignKeyChecksState
- vw.ForeignKeyChecksState = fkState
+ vw.ForeignKeyChecksState = qh.ForeignKeyChecks
defer func() {
vw.ForeignKeyChecksState = oldVal
}()
@@ -89,12 +103,12 @@ func TestBuilder(query string, vschema plancontext.VSchema, keyspace string) (*e
}
reservedVars := sqlparser.NewReservedVars("vtg", reserved)
- return BuildFromStmt(context.Background(), query, result.AST, reservedVars, vschema, result.BindVarNeeds, true, true)
+ return BuildFromStmt(context.Background(), query, result.AST, reservedVars, vschema, result.BindVarNeeds, staticConfig{})
}
// BuildFromStmt builds a plan based on the AST provided.
-func BuildFromStmt(ctx context.Context, query string, stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, bindVarNeeds *sqlparser.BindVarNeeds, enableOnlineDDL, enableDirectDDL bool) (*engine.Plan, error) {
- planResult, err := createInstructionFor(ctx, query, stmt, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+func BuildFromStmt(ctx context.Context, query string, stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, bindVarNeeds *sqlparser.BindVarNeeds, cfg dynamicconfig.DDL) (*engine.Plan, error) {
+ planResult, err := createInstructionFor(ctx, query, stmt, reservedVars, vschema, cfg)
if err != nil {
return nil, err
}
@@ -151,7 +165,7 @@ func buildRoutePlan(stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVa
return f(stmt, reservedVars, vschema)
}
-func createInstructionFor(ctx context.Context, query string, stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
+func createInstructionFor(ctx context.Context, query string, stmt sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
switch stmt := stmt.(type) {
case *sqlparser.Select, *sqlparser.Insert, *sqlparser.Update, *sqlparser.Delete:
configuredPlanner, err := getConfiguredPlanner(vschema, stmt, query)
@@ -166,13 +180,13 @@ func createInstructionFor(ctx context.Context, query string, stmt sqlparser.Stat
}
return buildRoutePlan(stmt, reservedVars, vschema, configuredPlanner)
case sqlparser.DDLStatement:
- return buildGeneralDDLPlan(ctx, query, stmt, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ return buildGeneralDDLPlan(ctx, query, stmt, reservedVars, vschema, cfg)
case *sqlparser.AlterMigration:
- return buildAlterMigrationPlan(query, stmt, vschema, enableOnlineDDL)
+ return buildAlterMigrationPlan(query, stmt, vschema, cfg)
case *sqlparser.RevertMigration:
- return buildRevertMigrationPlan(query, stmt, vschema, enableOnlineDDL)
+ return buildRevertMigrationPlan(query, stmt, vschema, cfg)
case *sqlparser.ShowMigrationLogs:
- return buildShowMigrationLogsPlan(query, vschema, enableOnlineDDL)
+ return buildShowMigrationLogsPlan(query, vschema, cfg)
case *sqlparser.ShowThrottledApps:
return buildShowThrottledAppsPlan(query, vschema)
case *sqlparser.ShowThrottlerStatus:
@@ -186,7 +200,7 @@ func createInstructionFor(ctx context.Context, query string, stmt sqlparser.Stat
case *sqlparser.ExplainStmt:
return buildRoutePlan(stmt, reservedVars, vschema, buildExplainStmtPlan)
case *sqlparser.VExplainStmt:
- return buildVExplainPlan(ctx, stmt, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ return buildVExplainPlan(ctx, stmt, reservedVars, vschema, cfg)
case *sqlparser.OtherAdmin:
return buildOtherReadAndAdmin(query, vschema)
case *sqlparser.Analyze:
@@ -272,7 +286,7 @@ func buildDBDDLPlan(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vschema
dbDDLstmt := stmt.(sqlparser.DBDDLStatement)
ksName := dbDDLstmt.GetDatabaseName()
if ksName == "" {
- ks, err := vschema.DefaultKeyspace()
+ ks, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
@@ -307,7 +321,7 @@ func buildDBDDLPlan(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vschema
}
func buildLoadPlan(query string, vschema plancontext.VSchema) (*planResult, error) {
- keyspace, err := vschema.DefaultKeyspace()
+ keyspace, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
@@ -352,7 +366,7 @@ func buildFlushOptions(stmt *sqlparser.Flush, vschema plancontext.VSchema) (*pla
return nil, vterrors.VT09012("FLUSH", vschema.TabletType().String())
}
- keyspace, err := vschema.DefaultKeyspace()
+ keyspace, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
diff --git a/go/vt/vtgate/planbuilder/bypass.go b/go/vt/vtgate/planbuilder/bypass.go
index 62cae9655b1..d3384d509c1 100644
--- a/go/vt/vtgate/planbuilder/bypass.go
+++ b/go/vt/vtgate/planbuilder/bypass.go
@@ -26,7 +26,7 @@ import (
)
func buildPlanForBypass(stmt sqlparser.Statement, _ *sqlparser.ReservedVars, vschema plancontext.VSchema) (*planResult, error) {
- keyspace, err := vschema.DefaultKeyspace()
+ keyspace, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
diff --git a/go/vt/vtgate/planbuilder/collations_test.go b/go/vt/vtgate/planbuilder/collations_test.go
index b393e186679..0595039e673 100644
--- a/go/vt/vtgate/planbuilder/collations_test.go
+++ b/go/vt/vtgate/planbuilder/collations_test.go
@@ -41,15 +41,13 @@ type collationTestCase struct {
}
func (tc *collationTestCase) run(t *testing.T) {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(t, "vschemas/schema.json", false),
- SysVarEnabled: true,
- Version: Gen4,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(t, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(t, err)
- tc.addCollationsToSchema(vschemaWrapper)
- plan, err := TestBuilder(tc.query, vschemaWrapper, vschemaWrapper.CurrentDb())
+ tc.addCollationsToSchema(vw)
+ plan, err := TestBuilder(tc.query, vw, vw.CurrentDb())
require.NoError(t, err)
tc.check(t, tc.collations, plan.Instructions)
}
diff --git a/go/vt/vtgate/planbuilder/ddl.go b/go/vt/vtgate/planbuilder/ddl.go
index 4c4b3791c20..a0045cec060 100644
--- a/go/vt/vtgate/planbuilder/ddl.go
+++ b/go/vt/vtgate/planbuilder/ddl.go
@@ -9,6 +9,7 @@ import (
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/dynamicconfig"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
"vitess.io/vitess/go/vt/vtgate/vindexes"
@@ -43,11 +44,11 @@ func (fk *fkContraint) FkWalk(node sqlparser.SQLNode) (kontinue bool, err error)
// a session context. It's only when we Execute() the primitive that we have that context.
// This is why we return a compound primitive (DDL) which contains fully populated primitives (Send & OnlineDDL),
// and which chooses which of the two to invoke at runtime.
-func buildGeneralDDLPlan(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
+func buildGeneralDDLPlan(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
if vschema.Destination() != nil {
- return buildByPassPlan(sql, vschema)
+ return buildByPassPlan(sql, vschema, true)
}
- normalDDLPlan, onlineDDLPlan, err := buildDDLPlans(ctx, sql, ddlStatement, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ normalDDLPlan, onlineDDLPlan, err := buildDDLPlans(ctx, sql, ddlStatement, reservedVars, vschema, cfg)
if err != nil {
return nil, err
}
@@ -61,15 +62,12 @@ func buildGeneralDDLPlan(ctx context.Context, sql string, ddlStatement sqlparser
}
eddl := &engine.DDL{
- Keyspace: normalDDLPlan.Keyspace,
- SQL: normalDDLPlan.Query,
- DDL: ddlStatement,
- NormalDDL: normalDDLPlan,
- OnlineDDL: onlineDDLPlan,
-
- DirectDDLEnabled: enableDirectDDL,
- OnlineDDLEnabled: enableOnlineDDL,
-
+ Keyspace: normalDDLPlan.Keyspace,
+ SQL: normalDDLPlan.Query,
+ DDL: ddlStatement,
+ NormalDDL: normalDDLPlan,
+ OnlineDDL: onlineDDLPlan,
+ Config: cfg,
CreateTempTable: ddlStatement.IsTemporary(),
}
tc := &tableCollector{}
@@ -80,8 +78,8 @@ func buildGeneralDDLPlan(ctx context.Context, sql string, ddlStatement sqlparser
return newPlanResult(eddl, tc.getTables()...), nil
}
-func buildByPassPlan(sql string, vschema plancontext.VSchema) (*planResult, error) {
- keyspace, err := vschema.DefaultKeyspace()
+func buildByPassPlan(sql string, vschema plancontext.VSchema, isDDL bool) (*planResult, error) {
+ keyspace, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
@@ -89,11 +87,12 @@ func buildByPassPlan(sql string, vschema plancontext.VSchema) (*planResult, erro
Keyspace: keyspace,
TargetDestination: vschema.Destination(),
Query: sql,
+ IsDDL: isDDL,
}
return newPlanResult(send), nil
}
-func buildDDLPlans(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*engine.Send, *engine.OnlineDDL, error) {
+func buildDDLPlans(ctx context.Context, sql string, ddlStatement sqlparser.DDLStatement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*engine.Send, *engine.OnlineDDL, error) {
var destination key.Destination
var keyspace *vindexes.Keyspace
var err error
@@ -112,9 +111,9 @@ func buildDDLPlans(ctx context.Context, sql string, ddlStatement sqlparser.DDLSt
}
err = checkFKError(vschema, ddlStatement, keyspace)
case *sqlparser.CreateView:
- destination, keyspace, err = buildCreateViewCommon(ctx, vschema, reservedVars, enableOnlineDDL, enableDirectDDL, ddl.Select, ddl)
+ destination, keyspace, err = buildCreateViewCommon(ctx, vschema, reservedVars, cfg, ddl.Select, ddl)
case *sqlparser.AlterView:
- destination, keyspace, err = buildCreateViewCommon(ctx, vschema, reservedVars, enableOnlineDDL, enableDirectDDL, ddl.Select, ddl)
+ destination, keyspace, err = buildCreateViewCommon(ctx, vschema, reservedVars, cfg, ddl.Select, ddl)
case *sqlparser.DropView:
destination, keyspace, err = buildDropView(vschema, ddlStatement)
case *sqlparser.DropTable:
@@ -196,7 +195,7 @@ func buildCreateViewCommon(
ctx context.Context,
vschema plancontext.VSchema,
reservedVars *sqlparser.ReservedVars,
- enableOnlineDDL, enableDirectDDL bool,
+ cfg dynamicconfig.DDL,
ddlSelect sqlparser.SelectStatement,
ddl sqlparser.DDLStatement,
) (key.Destination, *vindexes.Keyspace, error) {
@@ -213,7 +212,7 @@ func buildCreateViewCommon(
expressions = append(expressions, sqlparser.Clone(p.SelectExprs))
return nil
})
- selectPlan, err := createInstructionFor(ctx, sqlparser.String(ddlSelect), ddlSelect, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ selectPlan, err := createInstructionFor(ctx, sqlparser.String(ddlSelect), ddlSelect, reservedVars, vschema, cfg)
if err != nil {
return nil, nil, err
}
diff --git a/go/vt/vtgate/planbuilder/migration.go b/go/vt/vtgate/planbuilder/migration.go
index 6fb73a9039d..e64b990aa6b 100644
--- a/go/vt/vtgate/planbuilder/migration.go
+++ b/go/vt/vtgate/planbuilder/migration.go
@@ -27,6 +27,7 @@ import (
"vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/dynamicconfig"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
"vitess.io/vitess/go/vt/vtgate/vindexes"
@@ -80,8 +81,8 @@ func buildAlterMigrationThrottleAppPlan(query string, alterMigration *sqlparser.
}), nil
}
-func buildAlterMigrationPlan(query string, alterMigration *sqlparser.AlterMigration, vschema plancontext.VSchema, enableOnlineDDL bool) (*planResult, error) {
- if !enableOnlineDDL {
+func buildAlterMigrationPlan(query string, alterMigration *sqlparser.AlterMigration, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ if !cfg.OnlineEnabled() {
return nil, schema.ErrOnlineDDLDisabled
}
@@ -118,8 +119,8 @@ func buildAlterMigrationPlan(query string, alterMigration *sqlparser.AlterMigrat
return newPlanResult(send), nil
}
-func buildRevertMigrationPlan(query string, stmt *sqlparser.RevertMigration, vschema plancontext.VSchema, enableOnlineDDL bool) (*planResult, error) {
- if !enableOnlineDDL {
+func buildRevertMigrationPlan(query string, stmt *sqlparser.RevertMigration, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ if !cfg.OnlineEnabled() {
return nil, schema.ErrOnlineDDLDisabled
}
dest, ks, tabletType, err := vschema.TargetDestination("")
@@ -147,8 +148,8 @@ func buildRevertMigrationPlan(query string, stmt *sqlparser.RevertMigration, vsc
return newPlanResult(emig), nil
}
-func buildShowMigrationLogsPlan(query string, vschema plancontext.VSchema, enableOnlineDDL bool) (*planResult, error) {
- if !enableOnlineDDL {
+func buildShowMigrationLogsPlan(query string, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ if !cfg.OnlineEnabled() {
return nil, schema.ErrOnlineDDLDisabled
}
dest, ks, tabletType, err := vschema.TargetDestination("")
diff --git a/go/vt/vtgate/planbuilder/operator_transformers.go b/go/vt/vtgate/planbuilder/operator_transformers.go
index 546a9854f26..df14745e6b2 100644
--- a/go/vt/vtgate/planbuilder/operator_transformers.go
+++ b/go/vt/vtgate/planbuilder/operator_transformers.go
@@ -77,11 +77,31 @@ func transformToPrimitive(ctx *plancontext.PlanningContext, op operators.Operato
return transformSequential(ctx, op)
case *operators.DMLWithInput:
return transformDMLWithInput(ctx, op)
+ case *operators.RecurseCTE:
+ return transformRecurseCTE(ctx, op)
+ case *operators.PercentBasedMirror:
+ return transformPercentBasedMirror(ctx, op)
}
return nil, vterrors.VT13001(fmt.Sprintf("unknown type encountered: %T (transformToPrimitive)", op))
}
+func transformPercentBasedMirror(ctx *plancontext.PlanningContext, op *operators.PercentBasedMirror) (engine.Primitive, error) {
+ primitive, err := transformToPrimitive(ctx, op.Operator())
+ if err != nil {
+ return nil, err
+ }
+
+ target, err := transformToPrimitive(ctx.UseMirror(), op.Target())
+ // Mirroring is best-effort. If we encounter an error while building the
+ // mirror target primitive, proceed without mirroring.
+ if err != nil {
+ return primitive, nil
+ }
+
+ return engine.NewPercentBasedMirror(op.Percent, primitive, target), nil
+}
+
func transformDMLWithInput(ctx *plancontext.PlanningContext, op *operators.DMLWithInput) (engine.Primitive, error) {
input, err := transformToPrimitive(ctx, op.Source)
if err != nil {
@@ -149,7 +169,7 @@ func transformSequential(ctx *plancontext.PlanningContext, op *operators.Sequent
}
func transformInsertionSelection(ctx *plancontext.PlanningContext, op *operators.InsertSelection) (engine.Primitive, error) {
- rb, isRoute := op.Insert.(*operators.Route)
+ rb, isRoute := op.Insert().(*operators.Route)
if !isRoute {
return nil, vterrors.VT13001(fmt.Sprintf("Incorrect type encountered: %T (transformInsertionSelection)", op.Insert))
}
@@ -178,7 +198,7 @@ func transformInsertionSelection(ctx *plancontext.PlanningContext, op *operators
eins.Prefix, _, eins.Suffix = generateInsertShardedQuery(ins.AST)
- selectionPlan, err := transformToPrimitive(ctx, op.Select)
+ selectionPlan, err := transformToPrimitive(ctx, op.Select())
if err != nil {
return nil, err
}
@@ -288,7 +308,6 @@ func transformFkVerify(ctx *plancontext.PlanningContext, fkv *operators.FkVerify
Verify: verify,
Exec: inputLP,
}, nil
-
}
func transformAggregator(ctx *plancontext.PlanningContext, op *operators.Aggregator) (engine.Primitive, error) {
@@ -451,7 +470,6 @@ func getEvalEngineExpr(ctx *plancontext.PlanningContext, pe *operators.ProjExpr)
default:
return nil, vterrors.VT13001("project not planned for: %s", pe.String())
}
-
}
// newSimpleProjection creates a simple projections
@@ -527,7 +545,7 @@ func routeToEngineRoute(ctx *plancontext.PlanningContext, op *operators.Route, h
}
func newRoutingParams(ctx *plancontext.PlanningContext, opCode engine.Opcode) *engine.RoutingParameters {
- ks, _ := ctx.VSchema.DefaultKeyspace()
+ ks, _ := ctx.VSchema.SelectedKeyspace()
if ks == nil {
// if we don't have a selected keyspace, any keyspace will do
// this is used by operators that do not set the keyspace
@@ -981,6 +999,22 @@ func transformVindexPlan(ctx *plancontext.PlanningContext, op *operators.Vindex)
return prim, nil
}
+func transformRecurseCTE(ctx *plancontext.PlanningContext, op *operators.RecurseCTE) (engine.Primitive, error) {
+ seed, err := transformToPrimitive(ctx, op.Seed())
+ if err != nil {
+ return nil, err
+ }
+ term, err := transformToPrimitive(ctx, op.Term())
+ if err != nil {
+ return nil, err
+ }
+ return &engine.RecurseCTE{
+ Seed: seed,
+ Term: term,
+ Vars: op.Vars,
+ }, nil
+}
+
func generateQuery(statement sqlparser.Statement) string {
buf := sqlparser.NewTrackedBuffer(dmlFormatter)
statement.Format(buf)
diff --git a/go/vt/vtgate/planbuilder/operators/SQL_builder.go b/go/vt/vtgate/planbuilder/operators/SQL_builder.go
index 08cf3c4801c..fc91569981d 100644
--- a/go/vt/vtgate/planbuilder/operators/SQL_builder.go
+++ b/go/vt/vtgate/planbuilder/operators/SQL_builder.go
@@ -55,7 +55,36 @@ func ToSQL(ctx *plancontext.PlanningContext, op Operator) (_ sqlparser.Statement
return q.stmt, q.dmlOperator, nil
}
+// includeTable will return false if the table is a CTE, and it is not merged
+// it will return true if the table is not a CTE or if it is a CTE and it is merged
+func (qb *queryBuilder) includeTable(op *Table) bool {
+ if qb.ctx.SemTable == nil {
+ return true
+ }
+ tbl, err := qb.ctx.SemTable.TableInfoFor(op.QTable.ID)
+ if err != nil {
+ panic(err)
+ }
+ cteTbl, isCTE := tbl.(*semantics.CTETable)
+ if !isCTE {
+ return true
+ }
+
+ return cteTbl.Merged
+}
+
func (qb *queryBuilder) addTable(db, tableName, alias string, tableID semantics.TableSet, hints sqlparser.IndexHints) {
+ if tableID.NumberOfTables() == 1 && qb.ctx.SemTable != nil {
+ tblInfo, err := qb.ctx.SemTable.TableInfoFor(tableID)
+ if err != nil {
+ panic(err)
+ }
+ cte, isCTE := tblInfo.(*semantics.CTETable)
+ if isCTE {
+ tableName = cte.TableName
+ db = ""
+ }
+ }
tableExpr := sqlparser.TableName{
Name: sqlparser.NewIdentifierCS(tableName),
Qualifier: sqlparser.NewIdentifierCS(db),
@@ -105,6 +134,12 @@ func (qb *queryBuilder) addPredicate(expr sqlparser.Expr) {
addPred = stmt.AddWhere
case *sqlparser.Delete:
addPred = stmt.AddWhere
+ case nil:
+ // this would happen if we are adding a predicate on a dual query.
+ // we use this when building recursive CTE queries
+ sel := &sqlparser.Select{}
+ addPred = sel.AddWhere
+ qb.stmt = sel
default:
panic(fmt.Sprintf("cant add WHERE to %T", qb.stmt))
}
@@ -207,6 +242,27 @@ func (qb *queryBuilder) unionWith(other *queryBuilder, distinct bool) {
}
}
+func (qb *queryBuilder) recursiveCteWith(other *queryBuilder, name, alias string, distinct bool, columns sqlparser.Columns) {
+ cteUnion := &sqlparser.Union{
+ Left: qb.stmt.(sqlparser.SelectStatement),
+ Right: other.stmt.(sqlparser.SelectStatement),
+ Distinct: distinct,
+ }
+
+ qb.stmt = &sqlparser.Select{
+ With: &sqlparser.With{
+ Recursive: true,
+ CTEs: []*sqlparser.CommonTableExpr{{
+ ID: sqlparser.NewIdentifierCS(name),
+ Columns: columns,
+ Subquery: cteUnion,
+ }},
+ },
+ }
+
+ qb.addTable("", name, alias, "", nil)
+}
+
type FromStatement interface {
GetFrom() []sqlparser.TableExpr
SetFrom([]sqlparser.TableExpr)
@@ -401,6 +457,8 @@ func buildQuery(op Operator, qb *queryBuilder) {
buildDelete(op, qb)
case *Insert:
buildDML(op, qb)
+ case *RecurseCTE:
+ buildRecursiveCTE(op, qb)
default:
panic(vterrors.VT13001(fmt.Sprintf("unknown operator to convert to SQL: %T", op)))
}
@@ -492,6 +550,10 @@ func buildLimit(op *Limit, qb *queryBuilder) {
}
func buildTable(op *Table, qb *queryBuilder) {
+ if !qb.includeTable(op) {
+ return
+ }
+
dbName := ""
if op.QTable.IsInfSchema {
@@ -551,7 +613,16 @@ func buildApplyJoin(op *ApplyJoin, qb *queryBuilder) {
qbR := &queryBuilder{ctx: qb.ctx}
buildQuery(op.RHS, qbR)
- qb.joinWith(qbR, pred, op.JoinType)
+
+ switch {
+ // if we have a recursive cte, we might be missing a statement from one of the sides
+ case qbR.stmt == nil:
+ // do nothing
+ case qb.stmt == nil:
+ qb.stmt = qbR.stmt
+ default:
+ qb.joinWith(qbR, pred, op.JoinType)
+ }
}
func buildUnion(op *Union, qb *queryBuilder) {
@@ -636,6 +707,28 @@ func buildHorizon(op *Horizon, qb *queryBuilder) {
sqlparser.RemoveKeyspaceInCol(qb.stmt)
}
+func buildRecursiveCTE(op *RecurseCTE, qb *queryBuilder) {
+ predicates := slice.Map(op.Predicates, func(jc *plancontext.RecurseExpression) sqlparser.Expr {
+ // since we are adding these join predicates, we need to mark to broken up version (RHSExpr) of it as done
+ err := qb.ctx.SkipJoinPredicates(jc.Original)
+ if err != nil {
+ panic(err)
+ }
+ return jc.Original
+ })
+ pred := sqlparser.AndExpressions(predicates...)
+ buildQuery(op.Seed(), qb)
+ qbR := &queryBuilder{ctx: qb.ctx}
+ buildQuery(op.Term(), qbR)
+ qbR.addPredicate(pred)
+ infoFor, err := qb.ctx.SemTable.TableInfoFor(op.OuterID)
+ if err != nil {
+ panic(err)
+ }
+
+ qb.recursiveCteWith(qbR, op.Def.Name, infoFor.GetAliasedTableExpr().As.String(), op.Distinct, op.Def.Columns)
+}
+
func mergeHaving(h1, h2 *sqlparser.Where) *sqlparser.Where {
switch {
case h1 == nil && h2 == nil:
diff --git a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go
index 671f4b78954..73169369a41 100644
--- a/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go
+++ b/go/vt/vtgate/planbuilder/operators/aggregation_pushing.go
@@ -460,8 +460,8 @@ func createJoinPusher(rootAggr *Aggregator, operator Operator) *joinPusher {
return &joinPusher{
orig: rootAggr,
pushed: &Aggregator{
- Source: operator,
- QP: rootAggr.QP,
+ unaryOperator: newUnaryOp(operator),
+ QP: rootAggr.QP,
},
columns: initColReUse(len(rootAggr.Columns)),
tableID: TableID(operator),
diff --git a/go/vt/vtgate/planbuilder/operators/aggregator.go b/go/vt/vtgate/planbuilder/operators/aggregator.go
index bb969912f4f..f353ee02d1e 100644
--- a/go/vt/vtgate/planbuilder/operators/aggregator.go
+++ b/go/vt/vtgate/planbuilder/operators/aggregator.go
@@ -34,7 +34,7 @@ type (
// Both all aggregations and no grouping, and the inverse
// of all grouping and no aggregations are valid configurations of this operator
Aggregator struct {
- Source Operator
+ unaryOperator
Columns []*sqlparser.AliasedExpr
WithRollup bool
@@ -57,6 +57,9 @@ type (
// This is used to truncate the columns in the final result
ResultColumns int
+ // Truncate is set to true if the columns produced by this operator should be truncated if we added any additional columns
+ Truncate bool
+
QP *QueryProjection
DT *DerivedTable
@@ -72,17 +75,6 @@ func (a *Aggregator) Clone(inputs []Operator) Operator {
return &kopy
}
-func (a *Aggregator) Inputs() []Operator {
- return []Operator{a.Source}
-}
-
-func (a *Aggregator) SetInputs(operators []Operator) {
- if len(operators) != 1 {
- panic(fmt.Sprintf("unexpected number of operators as input in aggregator: %d", len(operators)))
- }
- a.Source = operators[0]
-}
-
func (a *Aggregator) AddPredicate(_ *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
return newFilter(a, expr)
}
@@ -151,6 +143,8 @@ func (a *Aggregator) checkOffset(offset int) {
}
func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, groupBy bool, ae *sqlparser.AliasedExpr) (offset int) {
+ a.planOffsets(ctx)
+
defer func() {
a.checkOffset(offset)
}()
@@ -199,6 +193,10 @@ func (a *Aggregator) AddColumn(ctx *plancontext.PlanningContext, reuse bool, gro
}
func (a *Aggregator) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int {
+ if !underRoute {
+ a.planOffsets(ctx)
+ }
+
if len(a.Columns) <= offset {
panic(vterrors.VT13001("offset out of range"))
}
@@ -221,7 +219,7 @@ func (a *Aggregator) AddWSColumn(ctx *plancontext.PlanningContext, offset int, u
}
if expr == nil {
- for _, aggr := range a.Aggregations {
+ for i, aggr := range a.Aggregations {
if aggr.ColOffset != offset {
continue
}
@@ -230,9 +228,13 @@ func (a *Aggregator) AddWSColumn(ctx *plancontext.PlanningContext, offset int, u
return aggr.WSOffset
}
- panic(vterrors.VT13001("expected to find a weight string for aggregation"))
+ a.Aggregations[i].WSOffset = len(a.Columns)
+ expr = a.Columns[offset].Expr
+ break
}
+ }
+ if expr == nil {
panic(vterrors.VT13001("could not find expression at offset"))
}
@@ -515,7 +517,7 @@ func (a *Aggregator) pushRemainingGroupingColumnsAndWeightStrings(ctx *planconte
continue
}
- offset := a.internalAddColumn(ctx, aeWrap(weightStringFor(gb.Inner)), false)
+ offset := a.internalAddWSColumn(ctx, a.Grouping[idx].ColOffset, aeWrap(weightStringFor(gb.Inner)))
a.Grouping[idx].WSOffset = offset
}
for idx, aggr := range a.Aggregations {
@@ -524,11 +526,28 @@ func (a *Aggregator) pushRemainingGroupingColumnsAndWeightStrings(ctx *planconte
}
arg := aggr.getPushColumn()
- offset := a.internalAddColumn(ctx, aeWrap(weightStringFor(arg)), false)
+ offset := a.internalAddWSColumn(ctx, aggr.ColOffset, aeWrap(weightStringFor(arg)))
+
a.Aggregations[idx].WSOffset = offset
}
}
+func (a *Aggregator) internalAddWSColumn(ctx *plancontext.PlanningContext, inOffset int, aliasedExpr *sqlparser.AliasedExpr) int {
+ if a.ResultColumns == 0 && a.Truncate {
+ // if we need to use `internalAddColumn`, it means we are adding columns that are not part of the original list,
+ // so we need to set the ResultColumns to the current length of the columns list
+ a.ResultColumns = len(a.Columns)
+ }
+
+ offset := a.Source.AddWSColumn(ctx, inOffset, false)
+
+ if offset == len(a.Columns) {
+ // if we get an offset at the end of our current column list, it means we added a new column
+ a.Columns = append(a.Columns, aliasedExpr)
+ }
+ return offset
+}
+
func (a *Aggregator) setTruncateColumnCount(offset int) {
a.ResultColumns = offset
}
@@ -538,7 +557,7 @@ func (a *Aggregator) getTruncateColumnCount() int {
}
func (a *Aggregator) internalAddColumn(ctx *plancontext.PlanningContext, aliasedExpr *sqlparser.AliasedExpr, addToGroupBy bool) int {
- if a.ResultColumns == 0 {
+ if a.ResultColumns == 0 && a.Truncate {
// if we need to use `internalAddColumn`, it means we are adding columns that are not part of the original list,
// so we need to set the ResultColumns to the current length of the columns list
a.ResultColumns = len(a.Columns)
diff --git a/go/vt/vtgate/planbuilder/operators/apply_join.go b/go/vt/vtgate/planbuilder/operators/apply_join.go
index ef36f6a6765..80bf74708a8 100644
--- a/go/vt/vtgate/planbuilder/operators/apply_join.go
+++ b/go/vt/vtgate/planbuilder/operators/apply_join.go
@@ -32,7 +32,7 @@ type (
// ApplyJoin is a nested loop join - for each row on the LHS,
// we'll execute the plan on the RHS, feeding data from left to right
ApplyJoin struct {
- LHS, RHS Operator
+ binaryOperator
// JoinType is permitted to store only 3 of the possible values
// NormalJoinType, StraightJoinType and LeftJoinType.
@@ -69,11 +69,10 @@ type (
// so they can be used for the result of this expression that is using data from both sides.
// All fields will be used for these
applyJoinColumn struct {
- Original sqlparser.Expr // this is the original expression being passed through
- LHSExprs []BindVarExpr // These are the expressions we are pushing to the left hand side which we'll receive as bind variables
- RHSExpr sqlparser.Expr // This the expression that we'll evaluate on the right hand side. This is nil, if the right hand side has nothing.
- DTColName *sqlparser.ColName // This is the output column name that the parent of JOIN will be seeing. If this is unset, then the colname is the String(Original). We set this when we push Projections with derived tables underneath a Join.
- GroupBy bool // if this is true, we need to push this down to our inputs with addToGroupBy set to true
+ Original sqlparser.Expr // this is the original expression being passed through
+ LHSExprs []BindVarExpr // These are the expressions we are pushing to the left hand side which we'll receive as bind variables
+ RHSExpr sqlparser.Expr // This the expression that we'll evaluate on the right hand side. This is nil, if the right hand side has nothing.
+ GroupBy bool // if this is true, we need to push this down to our inputs with addToGroupBy set to true
}
// BindVarExpr is an expression needed from one side of a join/subquery, and the argument name for it.
@@ -86,8 +85,7 @@ type (
func NewApplyJoin(ctx *plancontext.PlanningContext, lhs, rhs Operator, predicate sqlparser.Expr, joinType sqlparser.JoinType) *ApplyJoin {
aj := &ApplyJoin{
- LHS: lhs,
- RHS: rhs,
+ binaryOperator: newBinaryOp(lhs, rhs),
Vars: map[string]int{},
JoinType: joinType,
JoinColumns: &applyJoinColumns{},
@@ -114,16 +112,6 @@ func (aj *ApplyJoin) AddPredicate(ctx *plancontext.PlanningContext, expr sqlpars
return AddPredicate(ctx, aj, expr, false, newFilterSinglePredicate)
}
-// Inputs implements the Operator interface
-func (aj *ApplyJoin) Inputs() []Operator {
- return []Operator{aj.LHS, aj.RHS}
-}
-
-// SetInputs implements the Operator interface
-func (aj *ApplyJoin) SetInputs(inputs []Operator) {
- aj.LHS, aj.RHS = inputs[0], inputs[1]
-}
-
func (aj *ApplyJoin) GetLHS() Operator {
return aj.LHS
}
@@ -205,7 +193,6 @@ func (aj *ApplyJoin) getJoinColumnFor(ctx *plancontext.PlanningContext, orig *sq
rhs := TableID(aj.RHS)
both := lhs.Merge(rhs)
deps := ctx.SemTable.RecursiveDeps(e)
-
switch {
case deps.IsSolvedBy(lhs):
col.LHSExprs = []BindVarExpr{{Expr: e}}
@@ -225,8 +212,7 @@ func (aj *ApplyJoin) getJoinColumnFor(ctx *plancontext.PlanningContext, orig *sq
func applyJoinCompare(ctx *plancontext.PlanningContext, expr sqlparser.Expr) func(e applyJoinColumn) bool {
return func(e applyJoinColumn) bool {
- // e.DTColName is how the outside world will be using this expression. So we should check for an equality with that too.
- return ctx.SemTable.EqualsExprWithDeps(e.Original, expr) || ctx.SemTable.EqualsExprWithDeps(e.DTColName, expr)
+ return ctx.SemTable.EqualsExprWithDeps(e.Original, expr)
}
}
@@ -302,12 +288,18 @@ func (aj *ApplyJoin) planOffsets(ctx *plancontext.PlanningContext) Operator {
for _, col := range aj.JoinPredicates.columns {
for _, lhsExpr := range col.LHSExprs {
+ if _, found := aj.Vars[lhsExpr.Name]; found {
+ continue
+ }
offset := aj.LHS.AddColumn(ctx, true, false, aeWrap(lhsExpr.Expr))
aj.Vars[lhsExpr.Name] = offset
}
}
for _, lhsExpr := range aj.ExtraLHSVars {
+ if _, found := aj.Vars[lhsExpr.Name]; found {
+ continue
+ }
offset := aj.LHS.AddColumn(ctx, true, false, aeWrap(lhsExpr.Expr))
aj.Vars[lhsExpr.Name] = offset
}
@@ -441,11 +433,8 @@ func (jc applyJoinColumn) String() string {
lhs := slice.Map(jc.LHSExprs, func(e BindVarExpr) string {
return sqlparser.String(e.Expr)
})
- if jc.DTColName == nil {
- return fmt.Sprintf("[%s | %s | %s]", strings.Join(lhs, ", "), rhs, sqlparser.String(jc.Original))
- }
- return fmt.Sprintf("[%s | %s | %s | %s]", strings.Join(lhs, ", "), rhs, sqlparser.String(jc.Original), sqlparser.String(jc.DTColName))
+ return fmt.Sprintf("[%s | %s | %s]", strings.Join(lhs, ", "), rhs, sqlparser.String(jc.Original))
}
func (jc applyJoinColumn) IsPureLeft() bool {
@@ -461,16 +450,10 @@ func (jc applyJoinColumn) IsMixedLeftAndRight() bool {
}
func (jc applyJoinColumn) GetPureLeftExpr() sqlparser.Expr {
- if jc.DTColName != nil {
- return jc.DTColName
- }
return jc.LHSExprs[0].Expr
}
func (jc applyJoinColumn) GetRHSExpr() sqlparser.Expr {
- if jc.DTColName != nil {
- return jc.DTColName
- }
return jc.RHSExpr
}
diff --git a/go/vt/vtgate/planbuilder/operators/ast_to_op.go b/go/vt/vtgate/planbuilder/operators/ast_to_op.go
index 0d838610866..12c19bb72a6 100644
--- a/go/vt/vtgate/planbuilder/operators/ast_to_op.go
+++ b/go/vt/vtgate/planbuilder/operators/ast_to_op.go
@@ -26,8 +26,10 @@ import (
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
-const foreignKeyConstraintValues = "fkc_vals"
-const foreignKeyUpdateExpr = "fkc_upd"
+const (
+ foreignKeyConstraintValues = "fkc_vals"
+ foreignKeyUpdateExpr = "fkc_upd"
+)
// translateQueryToOp creates an operator tree that represents the input SELECT or UNION query
func translateQueryToOp(ctx *plancontext.PlanningContext, selStmt sqlparser.Statement) Operator {
@@ -47,19 +49,26 @@ func translateQueryToOp(ctx *plancontext.PlanningContext, selStmt sqlparser.Stat
}
}
+func translateQueryToOpWithMirroring(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) Operator {
+ op := translateQueryToOp(ctx, stmt)
+
+ if selStmt, ok := stmt.(sqlparser.SelectStatement); ok {
+ if mi := ctx.SemTable.GetMirrorInfo(); mi.Percent > 0 {
+ mirrorOp := translateQueryToOp(ctx.UseMirror(), selStmt)
+ op = NewPercentBasedMirror(mi.Percent, op, mirrorOp)
+ }
+ }
+
+ return op
+}
+
func createOperatorFromSelect(ctx *plancontext.PlanningContext, sel *sqlparser.Select) Operator {
op := crossJoin(ctx, sel.From)
- if sel.Where != nil {
- op = addWherePredicates(ctx, sel.Where.Expr, op)
- }
+ op = addWherePredicates(ctx, sel.GetWherePredicate(), op)
if sel.Comments != nil || sel.Lock != sqlparser.NoLock {
- op = &LockAndComment{
- Source: op,
- Comments: sel.Comments,
- Lock: sel.Lock,
- }
+ op = newLockAndComment(op, sel.Comments, sel.Lock)
}
op = newHorizon(op, sel)
@@ -73,15 +82,26 @@ func addWherePredicates(ctx *plancontext.PlanningContext, expr sqlparser.Expr, o
return sqc.getRootOperator(op, nil)
}
-func addWherePredsToSubQueryBuilder(ctx *plancontext.PlanningContext, expr sqlparser.Expr, op Operator, sqc *SubQueryBuilder) Operator {
+func addWherePredsToSubQueryBuilder(ctx *plancontext.PlanningContext, in sqlparser.Expr, op Operator, sqc *SubQueryBuilder) Operator {
outerID := TableID(op)
- exprs := sqlparser.SplitAndExpression(nil, expr)
- for _, expr := range exprs {
+ for _, expr := range sqlparser.SplitAndExpression(nil, in) {
sqlparser.RemoveKeyspaceInCol(expr)
+ expr = simplifyPredicates(ctx, expr)
subq := sqc.handleSubquery(ctx, expr, outerID)
if subq != nil {
continue
}
+ boolean := ctx.IsConstantBool(expr)
+ if boolean != nil {
+ if *boolean {
+ // If the predicate is true, we can ignore it.
+ continue
+ }
+
+ // If the predicate is false, we push down a false predicate to influence routing
+ expr = sqlparser.NewIntLiteral("0")
+ }
+
op = op.AddPredicate(ctx, expr)
addColumnEquality(ctx, expr)
}
@@ -148,8 +168,8 @@ func createOperatorFromUnion(ctx *plancontext.PlanningContext, node *sqlparser.U
if isRHSUnion {
panic(vterrors.VT12001("nesting of UNIONs on the right-hand side"))
}
- opLHS := translateQueryToOp(ctx, node.Left)
- opRHS := translateQueryToOp(ctx, node.Right)
+ opLHS := translateQueryToOpForUnion(ctx, node.Left)
+ opRHS := translateQueryToOpForUnion(ctx, node.Right)
lexprs := ctx.SemTable.SelectExprs(node.Left)
rexprs := ctx.SemTable.SelectExprs(node.Right)
@@ -158,6 +178,14 @@ func createOperatorFromUnion(ctx *plancontext.PlanningContext, node *sqlparser.U
return newHorizon(union, node)
}
+func translateQueryToOpForUnion(ctx *plancontext.PlanningContext, node sqlparser.SelectStatement) Operator {
+ op := translateQueryToOp(ctx, node)
+ if hz, ok := op.(*Horizon); ok {
+ hz.Truncate = true
+ }
+ return op
+}
+
// createOpFromStmt creates an operator from the given statement. It takes in two additional arguments—
// 1. verifyAllFKs: For this given statement, do we need to verify validity of all the foreign keys on the vtgate level.
// 2. fkToIgnore: The foreign key constraint to specifically ignore while planning the statement. This field is used in UPDATE CASCADE planning, wherein while planning the child update
@@ -246,24 +274,47 @@ func getOperatorFromAliasedTableExpr(ctx *plancontext.PlanningContext, tableExpr
panic(err)
}
- if vt, isVindex := tableInfo.(*semantics.VindexTable); isVindex {
- solves := tableID
+ switch tableInfo := tableInfo.(type) {
+ case *semantics.VindexTable:
return &Vindex{
Table: VindexTable{
TableID: tableID,
Alias: tableExpr,
Table: tbl,
- VTable: vt.Table.GetVindexTable(),
+ VTable: tableInfo.Table.GetVindexTable(),
},
- Vindex: vt.Vindex,
- Solved: solves,
+ Vindex: tableInfo.Vindex,
+ Solved: tableID,
}
+ case *semantics.CTETable:
+ return createDualCTETable(ctx, tableID, tableInfo)
+ case *semantics.RealTable:
+ if tableInfo.CTE != nil {
+ return createRecursiveCTE(ctx, tableInfo.CTE, tableID)
+ }
+
+ qg := newQueryGraph()
+ isInfSchema := tableInfo.IsInfSchema()
+ if ctx.IsMirrored() {
+ if mr := tableInfo.GetMirrorRule(); mr != nil {
+ newTbl := sqlparser.Clone(tbl)
+ newTbl.Qualifier = sqlparser.NewIdentifierCS(mr.Table.Keyspace.Name)
+ newTbl.Name = mr.Table.Name
+ if newTbl.Name.String() != tbl.Name.String() {
+ tableExpr = sqlparser.Clone(tableExpr)
+ tableExpr.As = tbl.Name
+ }
+ tbl = newTbl
+ } else {
+ panic(vterrors.VT13001(fmt.Sprintf("unable to find mirror rule for table: %T", tbl)))
+ }
+ }
+ qt := &QueryTable{Alias: tableExpr, Table: tbl, ID: tableID, IsInfSchema: isInfSchema}
+ qg.Tables = append(qg.Tables, qt)
+ return qg
+ default:
+ panic(vterrors.VT13001(fmt.Sprintf("unknown table type %T", tableInfo)))
}
- qg := newQueryGraph()
- isInfSchema := tableInfo.IsInfSchema()
- qt := &QueryTable{Alias: tableExpr, Table: tbl, ID: tableID, IsInfSchema: isInfSchema}
- qg.Tables = append(qg.Tables, qt)
- return qg
case *sqlparser.DerivedTable:
if onlyTable && tbl.Select.GetLimit() == nil {
tbl.Select.SetOrderBy(nil)
@@ -284,6 +335,56 @@ func getOperatorFromAliasedTableExpr(ctx *plancontext.PlanningContext, tableExpr
}
}
+func createDualCTETable(ctx *plancontext.PlanningContext, tableID semantics.TableSet, tableInfo *semantics.CTETable) Operator {
+ vschemaTable, _, _, _, _, err := ctx.VSchema.FindTableOrVindex(sqlparser.NewTableName("dual"))
+ if err != nil {
+ panic(err)
+ }
+ qtbl := &QueryTable{
+ ID: tableID,
+ Alias: tableInfo.ASTNode,
+ Table: sqlparser.NewTableName("dual"),
+ }
+ return createRouteFromVSchemaTable(ctx, qtbl, vschemaTable, false, nil)
+}
+
+func createRecursiveCTE(ctx *plancontext.PlanningContext, def *semantics.CTE, outerID semantics.TableSet) Operator {
+ union, ok := def.Query.(*sqlparser.Union)
+ if !ok {
+ panic(vterrors.VT13001("expected UNION in recursive CTE"))
+ }
+
+ seed := translateQueryToOp(ctx, union.Left)
+
+ // Push the CTE definition to the stack so that it can be used in the recursive part of the query
+ ctx.PushCTE(def, *def.IDForRecurse)
+
+ term := translateQueryToOp(ctx, union.Right)
+ horizon, ok := term.(*Horizon)
+ if !ok {
+ panic(vterrors.VT09027(def.Name))
+ }
+ term = horizon.Source
+ horizon.Source = nil // not sure about this
+ activeCTE, err := ctx.PopCTE()
+ if err != nil {
+ panic(err)
+ }
+
+ return newRecurse(ctx, def, seed, term, activeCTE.Predicates, horizon, idForRecursiveTable(ctx, def), outerID, union.Distinct)
+}
+
+func idForRecursiveTable(ctx *plancontext.PlanningContext, def *semantics.CTE) semantics.TableSet {
+ for i, table := range ctx.SemTable.Tables {
+ tbl, ok := table.(*semantics.CTETable)
+ if !ok || tbl.CTE.Name != def.Name {
+ continue
+ }
+ return semantics.SingleTableSet(i)
+ }
+ panic(vterrors.VT13001("recursive table not found"))
+}
+
func crossJoin(ctx *plancontext.PlanningContext, exprs sqlparser.TableExprs) Operator {
var output Operator
for _, tableExpr := range exprs {
diff --git a/go/vt/vtgate/planbuilder/operators/comments.go b/go/vt/vtgate/planbuilder/operators/comments.go
index 7e7749a61b5..9f0202c250a 100644
--- a/go/vt/vtgate/planbuilder/operators/comments.go
+++ b/go/vt/vtgate/planbuilder/operators/comments.go
@@ -26,25 +26,25 @@ import (
// LockAndComment contains any comments or locking directives we want on all queries down from this operator
type LockAndComment struct {
- Source Operator
+ unaryOperator
Comments *sqlparser.ParsedComments
Lock sqlparser.Lock
}
+func newLockAndComment(op Operator, comments *sqlparser.ParsedComments, lock sqlparser.Lock) Operator {
+ return &LockAndComment{
+ unaryOperator: newUnaryOp(op),
+ Comments: comments,
+ Lock: lock,
+ }
+}
+
func (l *LockAndComment) Clone(inputs []Operator) Operator {
klon := *l
klon.Source = inputs[0]
return &klon
}
-func (l *LockAndComment) Inputs() []Operator {
- return []Operator{l.Source}
-}
-
-func (l *LockAndComment) SetInputs(operators []Operator) {
- l.Source = operators[0]
-}
-
func (l *LockAndComment) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
l.Source = l.Source.AddPredicate(ctx, expr)
return l
diff --git a/go/vt/vtgate/planbuilder/operators/cte_merging.go b/go/vt/vtgate/planbuilder/operators/cte_merging.go
new file mode 100644
index 00000000000..cb19e06b2a7
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/operators/cte_merging.go
@@ -0,0 +1,95 @@
+/*
+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 operators
+
+import (
+ "vitess.io/vitess/go/vt/vtgate/engine"
+ "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+)
+
+func tryMergeRecurse(ctx *plancontext.PlanningContext, in *RecurseCTE) (Operator, *ApplyResult) {
+ op := tryMergeCTE(ctx, in.Seed(), in.Term(), in)
+ if op == nil {
+ return in, NoRewrite
+ }
+
+ return op, Rewrote("Merged CTE")
+}
+
+func tryMergeCTE(ctx *plancontext.PlanningContext, seed, term Operator, in *RecurseCTE) *Route {
+ seedRoute, termRoute, routingA, routingB, a, b, sameKeyspace := prepareInputRoutes(seed, term)
+ if seedRoute == nil {
+ return nil
+ }
+
+ switch {
+ case a == dual:
+ return mergeCTE(ctx, seedRoute, termRoute, routingB, in)
+ case b == dual:
+ return mergeCTE(ctx, seedRoute, termRoute, routingA, in)
+ case !sameKeyspace:
+ return nil
+ case a == anyShard:
+ return mergeCTE(ctx, seedRoute, termRoute, routingB, in)
+ case b == anyShard:
+ return mergeCTE(ctx, seedRoute, termRoute, routingA, in)
+ case a == sharded && b == sharded:
+ return tryMergeCTESharded(ctx, seedRoute, termRoute, in)
+ default:
+ return nil
+ }
+}
+
+func tryMergeCTESharded(ctx *plancontext.PlanningContext, seed, term *Route, in *RecurseCTE) *Route {
+ tblA := seed.Routing.(*ShardedRouting)
+ tblB := term.Routing.(*ShardedRouting)
+ switch tblA.RouteOpCode {
+ case engine.EqualUnique:
+ // If the two routes fully match, they can be merged together.
+ if tblB.RouteOpCode == engine.EqualUnique {
+ aVdx := tblA.SelectedVindex()
+ bVdx := tblB.SelectedVindex()
+ aExpr := tblA.VindexExpressions()
+ bExpr := tblB.VindexExpressions()
+ if aVdx == bVdx && gen4ValuesEqual(ctx, aExpr, bExpr) {
+ return mergeCTE(ctx, seed, term, tblA, in)
+ }
+ }
+ }
+
+ return nil
+}
+
+func mergeCTE(ctx *plancontext.PlanningContext, seed, term *Route, r Routing, in *RecurseCTE) *Route {
+ in.Def.Merged = true
+ hz := in.Horizon
+ hz.Source = term.Source
+ newTerm, _ := expandHorizon(ctx, hz)
+ cte := &RecurseCTE{
+ binaryOperator: newBinaryOp(seed.Source, newTerm),
+ Predicates: in.Predicates,
+ Def: in.Def,
+ LeftID: in.LeftID,
+ OuterID: in.OuterID,
+ Distinct: in.Distinct,
+ }
+ return &Route{
+ Routing: r,
+ unaryOperator: newUnaryOp(cte),
+ MergedWith: []*Route{term},
+ }
+}
diff --git a/go/vt/vtgate/planbuilder/operators/delete.go b/go/vt/vtgate/planbuilder/operators/delete.go
index 5bbf5218bd7..81e36d54315 100644
--- a/go/vt/vtgate/planbuilder/operators/delete.go
+++ b/go/vt/vtgate/planbuilder/operators/delete.go
@@ -45,9 +45,6 @@ func (d *Delete) Inputs() []Operator {
}
func (d *Delete) SetInputs(inputs []Operator) {
- if len(inputs) != 1 {
- panic(vterrors.VT13001("unexpected number of inputs for Delete operator"))
- }
d.Source = inputs[0]
}
@@ -55,8 +52,8 @@ func (d *Delete) GetOrdering(*plancontext.PlanningContext) []OrderBy {
return nil
}
-func (d *Delete) TablesUsed() []string {
- return SingleQualifiedIdentifier(d.Target.VTable.Keyspace, d.Target.VTable.Name)
+func (d *Delete) TablesUsed(in []string) []string {
+ return append(in, QualifiedString(d.Target.VTable.Keyspace, d.Target.VTable.Name.String()))
}
func (d *Delete) ShortDescription() string {
@@ -78,10 +75,7 @@ func createOperatorFromDelete(ctx *plancontext.PlanningContext, deleteStmt *sqlp
op, vTbl = createDeleteOperator(ctx, deleteStmt)
if deleteStmt.Comments != nil {
- op = &LockAndComment{
- Source: op,
- Comments: deleteStmt.Comments,
- }
+ op = newLockAndComment(op, deleteStmt.Comments, sqlparser.NoLock)
}
var err error
@@ -154,10 +148,7 @@ func createDeleteWithInputOp(ctx *plancontext.PlanningContext, del *sqlparser.De
}
if del.Comments != nil {
- op = &LockAndComment{
- Source: op,
- Comments: del.Comments,
- }
+ op = newLockAndComment(op, del.Comments, sqlparser.NoLock)
}
return op
}
@@ -260,16 +251,13 @@ func createDeleteOperator(ctx *plancontext.PlanningContext, del *sqlparser.Delet
Ignore: del.Ignore,
Target: targetTbl,
OwnedVindexQuery: ovq,
- Source: op,
},
}
if del.Limit != nil {
- addOrdering(ctx, del.OrderBy, delOp)
- delOp.Source = &Limit{
- Source: delOp.Source,
- AST: del.Limit,
- }
+ delOp.Source = newLimit(addOrdering(ctx, op, del.OrderBy), del.Limit, false)
+ } else {
+ delOp.Source = op
}
return sqc.getRootOperator(delOp, nil), vTbl
@@ -302,26 +290,24 @@ func makeColName(col sqlparser.IdentifierCI, table TargetTable, isMultiTbl bool)
return sqlparser.NewColName(col.String())
}
-func addOrdering(ctx *plancontext.PlanningContext, orderBy sqlparser.OrderBy, op Operator) {
+func addOrdering(ctx *plancontext.PlanningContext, op Operator, orderBy sqlparser.OrderBy) Operator {
es := &expressionSet{}
- ordering := &Ordering{}
- ordering.SetInputs(op.Inputs())
- for _, order := range orderBy {
- if sqlparser.IsNull(order.Expr) {
- // ORDER BY null can safely be ignored
+ var order []OrderBy
+ for _, ord := range orderBy {
+ if sqlparser.IsNull(ord.Expr) || !es.add(ctx, ord.Expr) {
+ // ORDER BY null, or expression repeated can safely be ignored
continue
}
- if !es.add(ctx, order.Expr) {
- continue
- }
- ordering.Order = append(ordering.Order, OrderBy{
- Inner: sqlparser.Clone(order),
- SimplifiedExpr: order.Expr,
+
+ order = append(order, OrderBy{
+ Inner: sqlparser.Clone(ord),
+ SimplifiedExpr: ord.Expr,
})
}
- if len(ordering.Order) > 0 {
- op.SetInputs([]Operator{ordering})
+ if len(order) == 0 {
+ return op
}
+ return newOrdering(op, order)
}
func updateQueryGraphWithSource(ctx *plancontext.PlanningContext, input Operator, tblID semantics.TableSet, vTbl *vindexes.Table) *vindexes.Table {
@@ -342,7 +328,7 @@ func updateQueryGraphWithSource(ctx *plancontext.PlanningContext, input Operator
if tbl.ID != tblID {
continue
}
- tbl.Alias = sqlparser.NewAliasedTableExpr(sqlparser.NewTableName(vTbl.Name.String()), tbl.Alias.As.String())
+ tbl.Alias = sqlparser.NewAliasedTableExpr(sqlparser.NewTableNameWithQualifier(vTbl.Name.String(), vTbl.Keyspace.Name), tbl.Alias.As.String())
tbl.Table, _ = tbl.Alias.TableName()
}
return op, Rewrote("change query table point to source table")
diff --git a/go/vt/vtgate/planbuilder/operators/distinct.go b/go/vt/vtgate/planbuilder/operators/distinct.go
index 4fd53725e10..52221498eea 100644
--- a/go/vt/vtgate/planbuilder/operators/distinct.go
+++ b/go/vt/vtgate/planbuilder/operators/distinct.go
@@ -26,8 +26,8 @@ import (
type (
Distinct struct {
- Source Operator
- QP *QueryProjection
+ unaryOperator
+ QP *QueryProjection
// When we go from AST to operator, we place DISTINCT ops in the required places in the op tree
// These are marked as `Required`, because they are semantically important to the results of the query.
@@ -45,6 +45,14 @@ type (
}
)
+func newDistinct(src Operator, qp *QueryProjection, required bool) *Distinct {
+ return &Distinct{
+ unaryOperator: newUnaryOp(src),
+ QP: qp,
+ Required: required,
+ }
+}
+
func (d *Distinct) planOffsets(ctx *plancontext.PlanningContext) Operator {
columns := d.GetColumns(ctx)
for idx, col := range columns {
@@ -66,22 +74,10 @@ func (d *Distinct) planOffsets(ctx *plancontext.PlanningContext) Operator {
}
func (d *Distinct) Clone(inputs []Operator) Operator {
- return &Distinct{
- Required: d.Required,
- Source: inputs[0],
- Columns: slices.Clone(d.Columns),
- QP: d.QP,
- PushedPerformance: d.PushedPerformance,
- ResultColumns: d.ResultColumns,
- }
-}
-
-func (d *Distinct) Inputs() []Operator {
- return []Operator{d.Source}
-}
-
-func (d *Distinct) SetInputs(operators []Operator) {
- d.Source = operators[0]
+ kopy := *d
+ kopy.Columns = slices.Clone(d.Columns)
+ kopy.Source = inputs[0]
+ return &kopy
}
func (d *Distinct) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
diff --git a/go/vt/vtgate/planbuilder/operators/dml_with_input.go b/go/vt/vtgate/planbuilder/operators/dml_with_input.go
index 3843e2f3fa8..720056f1964 100644
--- a/go/vt/vtgate/planbuilder/operators/dml_with_input.go
+++ b/go/vt/vtgate/planbuilder/operators/dml_with_input.go
@@ -50,9 +50,6 @@ func (d *DMLWithInput) Inputs() []Operator {
}
func (d *DMLWithInput) SetInputs(inputs []Operator) {
- if len(inputs) < 2 {
- panic("unexpected number of inputs for DMLWithInput operator")
- }
d.Source = inputs[0]
d.DML = inputs[1:]
}
diff --git a/go/vt/vtgate/planbuilder/operators/expressions.go b/go/vt/vtgate/planbuilder/operators/expressions.go
index 17b4bc7c3f1..38848693775 100644
--- a/go/vt/vtgate/planbuilder/operators/expressions.go
+++ b/go/vt/vtgate/planbuilder/operators/expressions.go
@@ -59,3 +59,65 @@ func breakExpressionInLHSandRHS(
col.Original = expr
return
}
+
+// nothingNeedsFetching will return true if all the nodes in the expression are constant
+func nothingNeedsFetching(ctx *plancontext.PlanningContext, expr sqlparser.Expr) (constant bool) {
+ constant = true
+ _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
+ if mustFetchFromInput(ctx, node) {
+ constant = false
+ }
+ return true, nil
+ }, expr)
+ return
+}
+
+func simplifyPredicates(ctx *plancontext.PlanningContext, in sqlparser.Expr) sqlparser.Expr {
+ var replace sqlparser.Expr
+
+ // if expr is constant true, replace with trueReplacement, if constant false, replace with falseReplacement
+ handleExpr := func(expr, trueReplacement, falseReplacement sqlparser.Expr) bool {
+ b := ctx.IsConstantBool(expr)
+ if b != nil {
+ if *b {
+ replace = trueReplacement
+ } else {
+ replace = falseReplacement
+ }
+ return true
+ }
+ return false
+ }
+
+ pre := func(node, _ sqlparser.SQLNode) bool {
+ switch node := node.(type) {
+ case *sqlparser.OrExpr:
+ if handleExpr(node.Left, sqlparser.NewIntLiteral("1"), node.Right) {
+ return false
+ }
+ if handleExpr(node.Right, sqlparser.NewIntLiteral("1"), node.Left) {
+ return false
+ }
+ case *sqlparser.AndExpr:
+ if handleExpr(node.Left, node.Right, sqlparser.NewIntLiteral("0")) {
+ return false
+ }
+ if handleExpr(node.Right, node.Left, sqlparser.NewIntLiteral("0")) {
+ return false
+ }
+ }
+ return true
+ }
+ post := func(cursor *sqlparser.CopyOnWriteCursor) {
+ if replace != nil {
+ cursor.Replace(replace)
+ replace = nil
+ }
+ }
+ output := sqlparser.CopyOnRewrite(in, pre, post, ctx.SemTable.CopySemanticInfo).(sqlparser.Expr)
+ if in != output {
+ // we need to do this, since one simplification might lead to another
+ return simplifyPredicates(ctx, output)
+ }
+ return output
+}
diff --git a/go/vt/vtgate/planbuilder/operators/filter.go b/go/vt/vtgate/planbuilder/operators/filter.go
index d68b2a43a24..d58d218908e 100644
--- a/go/vt/vtgate/planbuilder/operators/filter.go
+++ b/go/vt/vtgate/planbuilder/operators/filter.go
@@ -29,7 +29,7 @@ import (
)
type Filter struct {
- Source Operator
+ unaryOperator
Predicates []sqlparser.Expr
// PredicateWithOffsets is the evalengine expression that will finally be used.
@@ -45,28 +45,17 @@ func newFilterSinglePredicate(op Operator, expr sqlparser.Expr) Operator {
func newFilter(op Operator, expr ...sqlparser.Expr) Operator {
return &Filter{
- Source: op, Predicates: expr,
+ unaryOperator: newUnaryOp(op),
+ Predicates: expr,
}
}
// Clone implements the Operator interface
func (f *Filter) Clone(inputs []Operator) Operator {
- return &Filter{
- Source: inputs[0],
- Predicates: slices.Clone(f.Predicates),
- PredicateWithOffsets: f.PredicateWithOffsets,
- ResultColumns: f.ResultColumns,
- }
-}
-
-// Inputs implements the Operator interface
-func (f *Filter) Inputs() []Operator {
- return []Operator{f.Source}
-}
-
-// SetInputs implements the Operator interface
-func (f *Filter) SetInputs(ops []Operator) {
- f.Source = ops[0]
+ klon := *f
+ klon.Source = inputs[0]
+ klon.Predicates = slices.Clone(f.Predicates)
+ return &klon
}
// UnsolvedPredicates implements the unresolved interface
diff --git a/go/vt/vtgate/planbuilder/operators/fk_cascade.go b/go/vt/vtgate/planbuilder/operators/fk_cascade.go
index f24b59ca5ab..0aff5b3bea2 100644
--- a/go/vt/vtgate/planbuilder/operators/fk_cascade.go
+++ b/go/vt/vtgate/planbuilder/operators/fk_cascade.go
@@ -61,9 +61,6 @@ func (fkc *FkCascade) Inputs() []Operator {
// SetInputs implements the Operator interface
func (fkc *FkCascade) SetInputs(operators []Operator) {
- if len(operators) < 2 {
- panic("incorrect count of inputs for FkCascade")
- }
fkc.Parent = operators[0]
fkc.Selection = operators[1]
for idx, operator := range operators {
diff --git a/go/vt/vtgate/planbuilder/operators/fk_verify.go b/go/vt/vtgate/planbuilder/operators/fk_verify.go
index 8275a8d462f..a27f88f3335 100644
--- a/go/vt/vtgate/planbuilder/operators/fk_verify.go
+++ b/go/vt/vtgate/planbuilder/operators/fk_verify.go
@@ -52,9 +52,6 @@ func (fkv *FkVerify) Inputs() []Operator {
// SetInputs implements the Operator interface
func (fkv *FkVerify) SetInputs(operators []Operator) {
fkv.Input = operators[0]
- if len(fkv.Verify) != len(operators)-1 {
- panic("mismatched number of verify inputs")
- }
for i := 1; i < len(operators); i++ {
fkv.Verify[i-1].Op = operators[i]
}
diff --git a/go/vt/vtgate/planbuilder/operators/hash_join.go b/go/vt/vtgate/planbuilder/operators/hash_join.go
index 23d0d061e21..3761c4b87a6 100644
--- a/go/vt/vtgate/planbuilder/operators/hash_join.go
+++ b/go/vt/vtgate/planbuilder/operators/hash_join.go
@@ -31,7 +31,7 @@ import (
type (
HashJoin struct {
- LHS, RHS Operator
+ binaryOperator
// LeftJoin will be true in the case of an outer join
LeftJoin bool
@@ -79,10 +79,9 @@ var _ JoinOp = (*HashJoin)(nil)
func NewHashJoin(lhs, rhs Operator, outerJoin bool) *HashJoin {
hj := &HashJoin{
- LHS: lhs,
- RHS: rhs,
- LeftJoin: outerJoin,
- columns: &hashJoinColumns{},
+ binaryOperator: newBinaryOp(lhs, rhs),
+ LeftJoin: outerJoin,
+ columns: &hashJoinColumns{},
}
return hj
}
@@ -97,14 +96,6 @@ func (hj *HashJoin) Clone(inputs []Operator) Operator {
return &kopy
}
-func (hj *HashJoin) Inputs() []Operator {
- return []Operator{hj.LHS, hj.RHS}
-}
-
-func (hj *HashJoin) SetInputs(operators []Operator) {
- hj.LHS, hj.RHS = operators[0], operators[1]
-}
-
func (hj *HashJoin) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
return AddPredicate(ctx, hj, expr, false, newFilterSinglePredicate)
}
diff --git a/go/vt/vtgate/planbuilder/operators/hash_join_test.go b/go/vt/vtgate/planbuilder/operators/hash_join_test.go
index 2bf1d08d2b6..7325cc015a1 100644
--- a/go/vt/vtgate/planbuilder/operators/hash_join_test.go
+++ b/go/vt/vtgate/planbuilder/operators/hash_join_test.go
@@ -41,10 +41,9 @@ func TestJoinPredicates(t *testing.T) {
lhs := &fakeOp{id: lid}
rhs := &fakeOp{id: rid}
hj := &HashJoin{
- LHS: lhs,
- RHS: rhs,
- LeftJoin: false,
- columns: &hashJoinColumns{},
+ binaryOperator: newBinaryOp(lhs, rhs),
+ LeftJoin: false,
+ columns: &hashJoinColumns{},
}
cmp := &sqlparser.ComparisonExpr{
@@ -99,10 +98,9 @@ func TestOffsetPlanning(t *testing.T) {
for _, test := range tests {
t.Run(sqlparser.String(test.expr), func(t *testing.T) {
hj := &HashJoin{
- LHS: lhs,
- RHS: rhs,
- LeftJoin: false,
- columns: &hashJoinColumns{},
+ binaryOperator: newBinaryOp(lhs, rhs),
+ LeftJoin: false,
+ columns: &hashJoinColumns{},
}
hj.AddColumn(ctx, true, false, aeWrap(test.expr))
hj.planOffsets(ctx)
diff --git a/go/vt/vtgate/planbuilder/operators/helpers.go b/go/vt/vtgate/planbuilder/operators/helpers.go
index 31d9bcfd279..36b10c96ae1 100644
--- a/go/vt/vtgate/planbuilder/operators/helpers.go
+++ b/go/vt/vtgate/planbuilder/operators/helpers.go
@@ -18,6 +18,7 @@ package operators
import (
"fmt"
+ "slices"
"sort"
"vitess.io/vitess/go/vt/sqlparser"
@@ -82,20 +83,21 @@ func TableID(op Operator) (result semantics.TableSet) {
// TableUser is used to signal that this operator directly interacts with one or more tables
type TableUser interface {
- TablesUsed() []string
+ TablesUsed([]string) []string
}
func TablesUsed(op Operator) []string {
- addString, collect := collectSortedUniqueStrings()
+ var in []string
_ = Visit(op, func(this Operator) error {
if tbl, ok := this.(TableUser); ok {
- for _, u := range tbl.TablesUsed() {
- addString(u)
- }
+ in = tbl.TablesUsed(in)
}
return nil
})
- return collect()
+
+ slices.Sort(in)
+ compacted := slices.Compact(in)
+ return compacted
}
func CostOf(op Operator) (cost int) {
diff --git a/go/vt/vtgate/planbuilder/operators/horizon.go b/go/vt/vtgate/planbuilder/operators/horizon.go
index 7539704c2a9..292be1b37c5 100644
--- a/go/vt/vtgate/planbuilder/operators/horizon.go
+++ b/go/vt/vtgate/planbuilder/operators/horizon.go
@@ -18,6 +18,7 @@ package operators
import (
"errors"
+ "fmt"
"slices"
"vitess.io/vitess/go/vt/sqlparser"
@@ -34,7 +35,7 @@ import (
// Project/Aggregate/Sort/Limit operations, some which can be pushed down,
// and some that have to be evaluated at the vtgate level.
type Horizon struct {
- Source Operator
+ unaryOperator
// If this is a derived table, the two following fields will contain the tableID and name of it
TableId *semantics.TableSet
@@ -49,10 +50,15 @@ type Horizon struct {
// Columns needed to feed other plans
Columns []*sqlparser.ColName
ColumnsOffset []int
+
+ Truncate bool
}
func newHorizon(src Operator, query sqlparser.SelectStatement) *Horizon {
- return &Horizon{Source: src, Query: query}
+ return &Horizon{
+ unaryOperator: newUnaryOp(src),
+ Query: query,
+ }
}
// Clone implements the Operator interface
@@ -75,16 +81,6 @@ func (h *Horizon) IsMergeable(ctx *plancontext.PlanningContext) bool {
return isMergeable(ctx, h.Query, h)
}
-// Inputs implements the Operator interface
-func (h *Horizon) Inputs() []Operator {
- return []Operator{h.Source}
-}
-
-// SetInputs implements the Operator interface
-func (h *Horizon) SetInputs(ops []Operator) {
- h.Source = ops[0]
-}
-
func (h *Horizon) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
if _, isUNion := h.Source.(*Union); isUNion {
// If we have a derived table on top of a UNION, we can let the UNION do the expression rewriting
@@ -205,7 +201,7 @@ func (h *Horizon) getQP(ctx *plancontext.PlanningContext) *QueryProjection {
}
func (h *Horizon) ShortDescription() string {
- return h.Alias
+ return fmt.Sprintf("Horizon (Alias: %s)", h.Alias)
}
func (h *Horizon) introducesTableID() semantics.TableSet {
diff --git a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go
index fd3d992d0ca..dad5ad3a91a 100644
--- a/go/vt/vtgate/planbuilder/operators/horizon_expanding.go
+++ b/go/vt/vtgate/planbuilder/operators/horizon_expanding.go
@@ -44,17 +44,11 @@ func expandUnionHorizon(ctx *plancontext.PlanningContext, horizon *Horizon, unio
qp := horizon.getQP(ctx)
if len(qp.OrderExprs) > 0 {
- op = &Ordering{
- Source: op,
- Order: qp.OrderExprs,
- }
+ op = newOrdering(op, qp.OrderExprs)
}
if union.Limit != nil {
- op = &Limit{
- Source: op,
- AST: union.Limit,
- }
+ op = newLimit(op, union.Limit, false)
}
if horizon.TableId != nil {
@@ -84,6 +78,12 @@ func expandSelectHorizon(ctx *plancontext.PlanningContext, horizon *Horizon, sel
for _, order := range horizon.Query.GetOrderBy() {
qp.addDerivedColumn(ctx, order.Expr)
}
+ sel, isSel := horizon.Query.(*sqlparser.Select)
+ if isSel && sel.Having != nil {
+ for _, pred := range sqlparser.SplitAndExpression(nil, sel.Having.Expr) {
+ qp.addDerivedColumn(ctx, pred)
+ }
+ }
}
op := createProjectionFromSelect(ctx, horizon)
@@ -94,11 +94,7 @@ func expandSelectHorizon(ctx *plancontext.PlanningContext, horizon *Horizon, sel
}
if qp.NeedsDistinct() {
- op = &Distinct{
- Required: true,
- Source: op,
- QP: qp,
- }
+ op = newDistinct(op, qp, true)
extracted = append(extracted, "Distinct")
}
@@ -113,11 +109,7 @@ func expandSelectHorizon(ctx *plancontext.PlanningContext, horizon *Horizon, sel
}
if sel.Limit != nil {
- op = &Limit{
- Source: op,
- AST: sel.Limit,
- Top: true,
- }
+ op = newLimit(op, sel.Limit, true)
extracted = append(extracted, "Limit")
}
@@ -144,10 +136,7 @@ func expandOrderBy(ctx *plancontext.PlanningContext, op Operator, qp *QueryProje
// If the operator is not a projection, we cannot handle subqueries with aggregation if we are unable to push everything into a single route.
if !ok {
ctx.SemTable.NotSingleRouteErr = vterrors.VT12001("subquery with aggregation in order by")
- return &Ordering{
- Source: op,
- Order: qp.OrderExprs,
- }
+ return newOrdering(op, qp.OrderExprs)
} else {
// Add the new subquery expression to the projection
proj.addSubqueryExpr(ctx, aeWrap(newExpr), newExpr, subqs...)
@@ -169,10 +158,7 @@ func expandOrderBy(ctx *plancontext.PlanningContext, op Operator, qp *QueryProje
}
// Return the updated operator with the new order by expressions
- return &Ordering{
- Source: op,
- Order: newOrder,
- }
+ return newOrdering(op, newOrder)
}
// exposeOrderingColumn will expose the ordering column to the outer query
@@ -208,7 +194,7 @@ func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon *Horiz
}
if qp.NeedsAggregation() {
- return createProjectionWithAggr(ctx, qp, dt, horizon.src())
+ return createProjectionWithAggr(ctx, qp, dt, horizon)
}
projX := createProjectionWithoutAggr(ctx, qp, horizon.src())
@@ -216,16 +202,17 @@ func createProjectionFromSelect(ctx *plancontext.PlanningContext, horizon *Horiz
return projX
}
-func createProjectionWithAggr(ctx *plancontext.PlanningContext, qp *QueryProjection, dt *DerivedTable, src Operator) Operator {
+func createProjectionWithAggr(ctx *plancontext.PlanningContext, qp *QueryProjection, dt *DerivedTable, horizon *Horizon) Operator {
aggregations, complexAggr := qp.AggregationExpressions(ctx, true)
+ src := horizon.Source
aggrOp := &Aggregator{
- Source: src,
- Original: true,
- QP: qp,
- Grouping: qp.GetGrouping(),
- WithRollup: qp.WithRollup,
- Aggregations: aggregations,
- DT: dt,
+ unaryOperator: newUnaryOp(src),
+ Original: true,
+ QP: qp,
+ Grouping: qp.GetGrouping(),
+ WithRollup: qp.WithRollup,
+ Aggregations: aggregations,
+ DT: dt,
}
// Go through all aggregations and check for any subquery.
@@ -239,7 +226,11 @@ func createProjectionWithAggr(ctx *plancontext.PlanningContext, qp *QueryProject
if complexAggr {
return createProjectionForComplexAggregation(aggrOp, qp)
}
- return createProjectionForSimpleAggregation(ctx, aggrOp, qp)
+
+ addAllColumnsToAggregator(ctx, aggrOp, qp)
+ aggrOp.Truncate = horizon.Truncate
+
+ return aggrOp
}
func pullOutValueSubqueries(ctx *plancontext.PlanningContext, aggr Aggr, sqc *SubQueryBuilder, outerID semantics.TableSet) Aggr {
@@ -261,7 +252,7 @@ func pullOutValueSubqueries(ctx *plancontext.PlanningContext, aggr Aggr, sqc *Su
return aggr
}
-func createProjectionForSimpleAggregation(ctx *plancontext.PlanningContext, a *Aggregator, qp *QueryProjection) Operator {
+func addAllColumnsToAggregator(ctx *plancontext.PlanningContext, a *Aggregator, qp *QueryProjection) {
outer:
for colIdx, expr := range qp.SelectExprs {
ae, err := expr.GetAliasedExpr()
@@ -292,7 +283,6 @@ outer:
}
panic(vterrors.VT13001(fmt.Sprintf("Could not find the %s in aggregation in the original query", sqlparser.String(ae))))
}
- return a
}
func createProjectionForComplexAggregation(a *Aggregator, qp *QueryProjection) Operator {
@@ -368,7 +358,7 @@ func newStarProjection(src Operator, qp *QueryProjection) *Projection {
}
return &Projection{
- Source: src,
- Columns: StarProjections(cols),
+ unaryOperator: newUnaryOp(src),
+ Columns: StarProjections(cols),
}
}
diff --git a/go/vt/vtgate/planbuilder/operators/insert.go b/go/vt/vtgate/planbuilder/operators/insert.go
index 6832dc363d5..4ce37901a77 100644
--- a/go/vt/vtgate/planbuilder/operators/insert.go
+++ b/go/vt/vtgate/planbuilder/operators/insert.go
@@ -50,7 +50,7 @@ type Insert struct {
// that will appear in the result set of the select query.
VindexValueOffset [][]int
- noInputs
+ nullaryOperator
noColumns
noPredicates
}
@@ -96,8 +96,12 @@ func (i *Insert) Clone([]Operator) Operator {
}
}
-func (i *Insert) TablesUsed() []string {
- return SingleQualifiedIdentifier(i.VTable.Keyspace, i.VTable.Name)
+func (i *Insert) TablesUsed(in []string) []string {
+ return append(in, i.tableTarget())
+}
+
+func (i *Insert) tableTarget() string {
+ return QualifiedString(i.VTable.Keyspace, i.VTable.Name.String())
}
func (i *Insert) Statement() sqlparser.Statement {
@@ -363,8 +367,8 @@ func createInsertOperator(ctx *plancontext.PlanningContext, insStmt *sqlparser.I
AST: insStmt,
}
route := &Route{
- Source: insOp,
- Routing: routing,
+ unaryOperator: newUnaryOp(insOp),
+ Routing: routing,
}
// Table column list is nil then add all the columns
@@ -394,10 +398,7 @@ func createInsertOperator(ctx *plancontext.PlanningContext, insStmt *sqlparser.I
op = insertSelectPlan(ctx, insOp, route, insStmt, rows)
}
if insStmt.Comments != nil {
- op = &LockAndComment{
- Source: op,
- Comments: insStmt.Comments,
- }
+ op = newLockAndComment(op, insStmt.Comments, sqlparser.NoLock)
}
return op
}
@@ -420,17 +421,13 @@ func insertSelectPlan(
// output of the select plan will be used to insert rows into the table.
insertSelect := &InsertSelection{
- Select: &LockAndComment{
- Source: selOp,
- Lock: sqlparser.ShareModeLock,
- },
- Insert: routeOp,
+ binaryOperator: newBinaryOp(newLockAndComment(selOp, nil, sqlparser.ShareModeLock), routeOp),
}
// When the table you are streaming data from and table you are inserting from are same.
// Then due to locking of the index range on the table we might not be able to insert into the table.
// Therefore, instead of streaming, this flag will ensure the records are first read and then inserted.
- insertTbl := insOp.TablesUsed()[0]
+ insertTbl := insOp.tableTarget()
selTables := TablesUsed(selOp)
for _, tbl := range selTables {
if insertTbl == tbl {
diff --git a/go/vt/vtgate/planbuilder/operators/insert_selection.go b/go/vt/vtgate/planbuilder/operators/insert_selection.go
index 70bda0a990a..5f806bbda0b 100644
--- a/go/vt/vtgate/planbuilder/operators/insert_selection.go
+++ b/go/vt/vtgate/planbuilder/operators/insert_selection.go
@@ -23,8 +23,7 @@ import (
// InsertSelection operator represents an INSERT into SELECT FROM query.
// It holds the operators for running the selection and insertion.
type InsertSelection struct {
- Select Operator
- Insert Operator
+ binaryOperator
// ForceNonStreaming when true, select first then insert, this is to avoid locking rows by select for insert.
ForceNonStreaming bool
@@ -33,21 +32,13 @@ type InsertSelection struct {
noPredicates
}
-func (is *InsertSelection) Clone(inputs []Operator) Operator {
- return &InsertSelection{
- Select: inputs[0],
- Insert: inputs[1],
- ForceNonStreaming: is.ForceNonStreaming,
- }
-}
-
-func (is *InsertSelection) Inputs() []Operator {
- return []Operator{is.Select, is.Insert}
-}
+var _ Operator = (*InsertSelection)(nil)
-func (is *InsertSelection) SetInputs(inputs []Operator) {
- is.Select = inputs[0]
- is.Insert = inputs[1]
+func (is *InsertSelection) Clone(inputs []Operator) Operator {
+ klone := *is
+ klone.LHS = inputs[0]
+ klone.RHS = inputs[1]
+ return &klone
}
func (is *InsertSelection) ShortDescription() string {
@@ -61,4 +52,10 @@ func (is *InsertSelection) GetOrdering(*plancontext.PlanningContext) []OrderBy {
return nil
}
-var _ Operator = (*InsertSelection)(nil)
+func (is *InsertSelection) Select() Operator {
+ return is.LHS
+}
+
+func (is *InsertSelection) Insert() Operator {
+ return is.RHS
+}
diff --git a/go/vt/vtgate/planbuilder/operators/join.go b/go/vt/vtgate/planbuilder/operators/join.go
index 71d2e5a8048..ff4915527a7 100644
--- a/go/vt/vtgate/planbuilder/operators/join.go
+++ b/go/vt/vtgate/planbuilder/operators/join.go
@@ -17,14 +17,16 @@ limitations under the License.
package operators
import (
+ "vitess.io/vitess/go/slice"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+ "vitess.io/vitess/go/vt/vtgate/semantics"
)
// Join represents a join. If we have a predicate, this is an inner join. If no predicate exists, it is a cross join
type Join struct {
- LHS, RHS Operator
+ binaryOperator
Predicate sqlparser.Expr
// JoinType is permitted to store only 3 of the possible values
// NormalJoinType, StraightJoinType and LeftJoinType.
@@ -40,28 +42,13 @@ func (j *Join) Clone(inputs []Operator) Operator {
clone := *j
clone.LHS = inputs[0]
clone.RHS = inputs[1]
- return &Join{
- LHS: inputs[0],
- RHS: inputs[1],
- Predicate: j.Predicate,
- JoinType: j.JoinType,
- }
+ return &clone
}
func (j *Join) GetOrdering(*plancontext.PlanningContext) []OrderBy {
return nil
}
-// Inputs implements the Operator interface
-func (j *Join) Inputs() []Operator {
- return []Operator{j.LHS, j.RHS}
-}
-
-// SetInputs implements the Operator interface
-func (j *Join) SetInputs(ops []Operator) {
- j.LHS, j.RHS = ops[0], ops[1]
-}
-
func (j *Join) Compact(ctx *plancontext.PlanningContext) (Operator, *ApplyResult) {
if !j.JoinType.IsCommutative() {
// if we can't move tables around, we can't merge these inputs
@@ -87,7 +74,10 @@ func (j *Join) Compact(ctx *plancontext.PlanningContext) (Operator, *ApplyResult
func createStraightJoin(ctx *plancontext.PlanningContext, join *sqlparser.JoinTableExpr, lhs, rhs Operator) Operator {
// for inner joins we can treat the predicates as filters on top of the join
- joinOp := &Join{LHS: lhs, RHS: rhs, JoinType: join.Join}
+ joinOp := &Join{
+ binaryOperator: newBinaryOp(lhs, rhs),
+ JoinType: join.Join,
+ }
return addJoinPredicates(ctx, join.Condition.On, joinOp)
}
@@ -103,7 +93,10 @@ func createLeftOuterJoin(ctx *plancontext.PlanningContext, join *sqlparser.JoinT
join.Join = sqlparser.NaturalLeftJoinType
}
- joinOp := &Join{LHS: lhs, RHS: rhs, JoinType: join.Join}
+ joinOp := &Join{
+ binaryOperator: newBinaryOp(lhs, rhs),
+ JoinType: join.Join,
+ }
// mark the RHS as outer tables so we know which columns are nullable
ctx.OuterTables = ctx.OuterTables.Merge(TableID(rhs))
@@ -141,11 +134,49 @@ func addJoinPredicates(
if subq != nil {
continue
}
+
+ // if we are inside a CTE, we need to check if we depend on the recursion table
+ if cte := ctx.ActiveCTE(); cte != nil && ctx.SemTable.DirectDeps(pred).IsOverlapping(cte.Id) {
+ original := pred
+ pred = addCTEPredicate(ctx, pred, cte)
+ ctx.AddJoinPredicates(original, pred)
+ }
op = op.AddPredicate(ctx, pred)
}
return sqc.getRootOperator(op, nil)
}
+// addCTEPredicate breaks the expression into LHS and RHS
+func addCTEPredicate(
+ ctx *plancontext.PlanningContext,
+ pred sqlparser.Expr,
+ cte *plancontext.ContextCTE,
+) sqlparser.Expr {
+ expr := breakCTEExpressionInLhsAndRhs(ctx, pred, cte.Id)
+ cte.Predicates = append(cte.Predicates, expr)
+ return expr.RightExpr
+}
+
+func breakCTEExpressionInLhsAndRhs(ctx *plancontext.PlanningContext, pred sqlparser.Expr, lhsID semantics.TableSet) *plancontext.RecurseExpression {
+ col := breakExpressionInLHSandRHS(ctx, pred, lhsID)
+
+ lhsExprs := slice.Map(col.LHSExprs, func(bve BindVarExpr) plancontext.BindVarExpr {
+ col, ok := bve.Expr.(*sqlparser.ColName)
+ if !ok {
+ panic(vterrors.VT13001("expected column name"))
+ }
+ return plancontext.BindVarExpr{
+ Name: bve.Name,
+ Expr: col,
+ }
+ })
+ return &plancontext.RecurseExpression{
+ Original: col.Original,
+ RightExpr: col.RHSExpr,
+ LeftExprs: lhsExprs,
+ }
+}
+
func createJoin(ctx *plancontext.PlanningContext, LHS, RHS Operator) Operator {
lqg, lok := LHS.(*QueryGraph)
rqg, rok := RHS.(*QueryGraph)
@@ -157,7 +188,9 @@ func createJoin(ctx *plancontext.PlanningContext, LHS, RHS Operator) Operator {
}
return op
}
- return &Join{LHS: LHS, RHS: RHS}
+ return &Join{
+ binaryOperator: newBinaryOp(LHS, RHS),
+ }
}
func (j *Join) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
diff --git a/go/vt/vtgate/planbuilder/operators/join_merging.go b/go/vt/vtgate/planbuilder/operators/join_merging.go
index 6f2af8b5ff9..c035b7d11ed 100644
--- a/go/vt/vtgate/planbuilder/operators/join_merging.go
+++ b/go/vt/vtgate/planbuilder/operators/join_merging.go
@@ -18,7 +18,6 @@ package operators
import (
"fmt"
- "reflect"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
@@ -28,7 +27,7 @@ import (
// mergeJoinInputs checks whether two operators can be merged into a single one.
// If they can be merged, a new operator with the merged routing is returned
// If they cannot be merged, nil is returned.
-func mergeJoinInputs(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPredicates []sqlparser.Expr, m *joinMerger) *Route {
+func (jm *joinMerger) mergeJoinInputs(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPredicates []sqlparser.Expr) *Route {
lhsRoute, rhsRoute, routingA, routingB, a, b, sameKeyspace := prepareInputRoutes(lhs, rhs)
if lhsRoute == nil {
return nil
@@ -44,39 +43,39 @@ func mergeJoinInputs(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPr
rhsClone.AddPredicate(ctx, predicate)
}
}
- if !m.joinType.IsInner() && !rhsClone.Routing.OpCode().IsSingleShard() {
+ if !jm.joinType.IsInner() && !rhsClone.Routing.OpCode().IsSingleShard() {
return nil
}
- return m.merge(ctx, lhsRoute, rhsClone, rhsClone.Routing)
+ return jm.merge(ctx, lhsRoute, rhsClone, rhsClone.Routing)
// If a dual is on the right side.
case b == dual:
- return m.merge(ctx, lhsRoute, rhsRoute, routingA)
+ return jm.merge(ctx, lhsRoute, rhsRoute, routingA)
// As both are reference route. We need to merge the alternates as well.
case a == anyShard && b == anyShard && sameKeyspace:
- newrouting := mergeAnyShardRoutings(ctx, routingA.(*AnyShardRouting), routingB.(*AnyShardRouting), joinPredicates, m.joinType)
- return m.merge(ctx, lhsRoute, rhsRoute, newrouting)
+ newrouting := mergeAnyShardRoutings(ctx, routingA.(*AnyShardRouting), routingB.(*AnyShardRouting), joinPredicates, jm.joinType)
+ return jm.merge(ctx, lhsRoute, rhsRoute, newrouting)
// an unsharded/reference route can be merged with anything going to that keyspace
case a == anyShard && sameKeyspace:
- return m.merge(ctx, lhsRoute, rhsRoute, routingB)
+ return jm.merge(ctx, lhsRoute, rhsRoute, routingB)
case b == anyShard && sameKeyspace:
- return m.merge(ctx, lhsRoute, rhsRoute, routingA)
+ return jm.merge(ctx, lhsRoute, rhsRoute, routingA)
// None routing can always be merged, as long as we are aiming for the same keyspace
case a == none && sameKeyspace:
- return m.merge(ctx, lhsRoute, rhsRoute, routingA)
+ return jm.merge(ctx, lhsRoute, rhsRoute, routingA)
case b == none && sameKeyspace:
- return m.merge(ctx, lhsRoute, rhsRoute, routingB)
+ return jm.merge(ctx, lhsRoute, rhsRoute, routingB)
// infoSchema routing is complex, so we handle it in a separate method
case a == infoSchema && b == infoSchema:
- return tryMergeInfoSchemaRoutings(ctx, routingA, routingB, m, lhsRoute, rhsRoute)
+ return tryMergeInfoSchemaRoutings(ctx, routingA, routingB, jm, lhsRoute, rhsRoute)
// sharded routing is complex, so we handle it in a separate method
case a == sharded && b == sharded:
- return tryMergeShardedRouting(ctx, lhsRoute, rhsRoute, m, joinPredicates)
+ return tryMergeShardedRouting(ctx, lhsRoute, rhsRoute, jm, joinPredicates)
default:
return nil
@@ -112,7 +111,6 @@ func prepareInputRoutes(lhs Operator, rhs Operator) (*Route, *Route, Routing, Ro
lhsRoute, rhsRoute, routingA, routingB, sameKeyspace := getRoutesOrAlternates(lhsRoute, rhsRoute)
a, b := getRoutingType(routingA), getRoutingType(routingB)
-
return lhsRoute, rhsRoute, routingA, routingB, a, b, sameKeyspace
}
@@ -188,10 +186,6 @@ func getRoutesOrAlternates(lhsRoute, rhsRoute *Route) (*Route, *Route, Routing,
return lhsRoute, rhsRoute, routingA, routingB, sameKeyspace
}
-func getTypeName(myvar interface{}) string {
- return reflect.TypeOf(myvar).String()
-}
-
func getRoutingType(r Routing) routingType {
switch r.(type) {
case *InfoSchemaRouting:
@@ -242,8 +236,8 @@ func (jm *joinMerger) getApplyJoin(ctx *plancontext.PlanningContext, op1, op2 *R
func (jm *joinMerger) merge(ctx *plancontext.PlanningContext, op1, op2 *Route, r Routing) *Route {
return &Route{
- Source: jm.getApplyJoin(ctx, op1, op2),
- MergedWith: []*Route{op2},
- Routing: r,
+ unaryOperator: newUnaryOp(jm.getApplyJoin(ctx, op1, op2)),
+ MergedWith: []*Route{op2},
+ Routing: r,
}
}
diff --git a/go/vt/vtgate/planbuilder/operators/keys.go b/go/vt/vtgate/planbuilder/operators/keys.go
new file mode 100644
index 00000000000..8d75b260aeb
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/operators/keys.go
@@ -0,0 +1,319 @@
+/*
+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 operators
+
+import (
+ "encoding/json"
+ "fmt"
+ "slices"
+ "sort"
+ "strings"
+
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+)
+
+type (
+ Column struct {
+ Table string
+ Name string
+ }
+ ColumnUse struct {
+ Column Column
+ Uses sqlparser.ComparisonExprOperator
+ }
+ JoinPredicate struct {
+ LHS, RHS Column
+ Uses sqlparser.ComparisonExprOperator
+ }
+ VExplainKeys struct {
+ StatementType string `json:"statementType"`
+ TableName []string `json:"tableName,omitempty"`
+ GroupingColumns []Column `json:"groupingColumns,omitempty"`
+ FilterColumns []ColumnUse `json:"filterColumns,omitempty"`
+ SelectColumns []Column `json:"selectColumns,omitempty"`
+ JoinPredicates []JoinPredicate `json:"joinPredicates,omitempty"`
+ }
+)
+
+func newJoinPredicate(lhs, rhs Column, op sqlparser.ComparisonExprOperator) JoinPredicate {
+ // we want to try to keep the columns in the same order, no matter how the query was written
+ if lhs.String() > rhs.String() {
+ var success bool
+ op, success = op.SwitchSides()
+ if success {
+ lhs, rhs = rhs, lhs
+ }
+ }
+ return JoinPredicate{LHS: lhs, RHS: rhs, Uses: op}
+}
+
+func (c Column) MarshalJSON() ([]byte, error) {
+ if c.Table != "" {
+ return json.Marshal(fmt.Sprintf("%s.%s", c.Table, c.Name))
+ }
+ return json.Marshal(c.Name)
+}
+
+func (c *Column) UnmarshalJSON(data []byte) error {
+ var s string
+ if err := json.Unmarshal(data, &s); err != nil {
+ return err
+ }
+ parts := strings.Split(s, ".")
+ if len(parts) > 1 {
+ c.Table = parts[0]
+ c.Name = parts[1]
+ } else {
+ c.Name = s
+ }
+ return nil
+}
+
+func (cu ColumnUse) MarshalJSON() ([]byte, error) {
+ return json.Marshal(fmt.Sprintf("%s %s", cu.Column, cu.Uses.JSONString()))
+}
+
+func (cu *ColumnUse) UnmarshalJSON(data []byte) error {
+ var s string
+ if err := json.Unmarshal(data, &s); err != nil {
+ return err
+ }
+ spaceIdx := strings.LastIndex(s, " ")
+ if spaceIdx == -1 {
+ return fmt.Errorf("invalid ColumnUse format: %s", s)
+ }
+
+ for i := spaceIdx - 1; i >= 0; i-- {
+ // table.column not like
+ // table.`tricky not` like
+ if s[i] == '`' || s[i] == '.' {
+ break
+ }
+ if s[i] == ' ' {
+ spaceIdx = i
+ break
+ }
+ if i == 0 {
+ return fmt.Errorf("invalid ColumnUse format: %s", s)
+ }
+ }
+
+ colStr, opStr := s[:spaceIdx], s[spaceIdx+1:]
+
+ err := cu.Column.UnmarshalJSON([]byte(`"` + colStr + `"`))
+ if err != nil {
+ return fmt.Errorf("failed to unmarshal column: %w", err)
+ }
+
+ cu.Uses, err = sqlparser.ComparisonExprOperatorFromJson(strings.ToLower(opStr))
+ if err != nil {
+ return fmt.Errorf("failed to unmarshal operator: %w", err)
+ }
+ return nil
+}
+
+func (jp *JoinPredicate) MarshalJSON() ([]byte, error) {
+ return json.Marshal(jp.String())
+}
+
+func (jp *JoinPredicate) UnmarshalJSON(data []byte) error {
+ var s string
+ if err := json.Unmarshal(data, &s); err != nil {
+ return err
+ }
+ subStrings := strings.Split(s, " ")
+ if len(subStrings) != 3 {
+ return fmt.Errorf("invalid JoinPredicate format: %s", s)
+ }
+
+ op, err := sqlparser.ComparisonExprOperatorFromJson(subStrings[1])
+ if err != nil {
+ return fmt.Errorf("invalid comparison operator: %w", err)
+ }
+ jp.Uses = op
+
+ if err = jp.LHS.UnmarshalJSON([]byte(`"` + subStrings[0] + `"`)); err != nil {
+ return err
+ }
+ if err = jp.RHS.UnmarshalJSON([]byte(`"` + subStrings[2] + `"`)); err != nil {
+ return err
+ }
+ return nil
+}
+
+func (c Column) String() string {
+ return fmt.Sprintf("%s.%s", c.Table, c.Name)
+}
+
+func (cu ColumnUse) String() string {
+ return fmt.Sprintf("%s %s", cu.Column, cu.Uses.JSONString())
+}
+
+func (jp JoinPredicate) String() string {
+ return fmt.Sprintf("%s %s %s", jp.LHS.String(), jp.Uses.JSONString(), jp.RHS.String())
+}
+
+type columnUse struct {
+ col *sqlparser.ColName
+ use sqlparser.ComparisonExprOperator
+}
+
+type joinPredicate struct {
+ lhs *sqlparser.ColName
+ rhs *sqlparser.ColName
+ uses sqlparser.ComparisonExprOperator
+}
+
+func GetVExplainKeys(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) (result VExplainKeys) {
+ var groupingColumns, selectColumns []*sqlparser.ColName
+ var filterColumns, joinColumns []columnUse
+ var jps []joinPredicate
+
+ addPredicate := func(predicate sqlparser.Expr) {
+ predicates := sqlparser.SplitAndExpression(nil, predicate)
+ for _, expr := range predicates {
+ switch cmp := expr.(type) {
+ case *sqlparser.ComparisonExpr:
+ lhs, lhsOK := cmp.Left.(*sqlparser.ColName)
+ rhs, rhsOK := cmp.Right.(*sqlparser.ColName)
+
+ var output = &filterColumns
+ if lhsOK && rhsOK && ctx.SemTable.RecursiveDeps(lhs) != ctx.SemTable.RecursiveDeps(rhs) {
+ // If the columns are from different tables, they are considered join columns
+ output = &joinColumns
+ jps = append(jps, joinPredicate{lhs: lhs, rhs: rhs, uses: cmp.Operator})
+ }
+
+ if lhsOK {
+ *output = append(*output, columnUse{lhs, cmp.Operator})
+ }
+
+ if switchedOp, ok := cmp.Operator.SwitchSides(); rhsOK && ok {
+ *output = append(*output, columnUse{rhs, switchedOp})
+ }
+ case *sqlparser.BetweenExpr:
+ if col, ok := cmp.Left.(*sqlparser.ColName); ok {
+ // a BETWEEN 100 AND 200 is equivalent to a >= 100 AND a <= 200
+ filterColumns = append(filterColumns,
+ columnUse{col, sqlparser.GreaterEqualOp},
+ columnUse{col, sqlparser.LessEqualOp})
+ }
+ }
+
+ }
+ }
+
+ _ = sqlparser.VisitSQLNode(stmt, func(node sqlparser.SQLNode) (kontinue bool, err error) {
+ switch node := node.(type) {
+ case *sqlparser.Where:
+ addPredicate(node.Expr)
+ case *sqlparser.JoinCondition:
+ addPredicate(node.On)
+ case *sqlparser.GroupBy:
+ for _, expr := range node.Exprs {
+ col, ok := expr.(*sqlparser.ColName)
+ if ok {
+ groupingColumns = append(groupingColumns, col)
+ }
+ }
+ case *sqlparser.AliasedExpr:
+ _ = sqlparser.VisitSQLNode(node, func(e sqlparser.SQLNode) (kontinue bool, err error) {
+ if col, ok := e.(*sqlparser.ColName); ok {
+ selectColumns = append(selectColumns, col)
+ }
+ return true, nil
+ })
+ }
+
+ return true, nil
+ })
+
+ return VExplainKeys{
+ SelectColumns: getUniqueColNames(ctx, selectColumns),
+ GroupingColumns: getUniqueColNames(ctx, groupingColumns),
+ FilterColumns: getUniqueColUsages(ctx, filterColumns),
+ StatementType: sqlparser.ASTToStatementType(stmt).String(),
+ JoinPredicates: getUniqueJoinPredicates(ctx, jps),
+ }
+}
+
+func getUniqueJoinPredicates(ctx *plancontext.PlanningContext, joinPredicates []joinPredicate) []JoinPredicate {
+ var result []JoinPredicate
+ for _, predicate := range joinPredicates {
+ lhs := createColumn(ctx, predicate.lhs)
+ rhs := createColumn(ctx, predicate.rhs)
+ if lhs == nil || rhs == nil {
+ continue
+ }
+
+ result = append(result, newJoinPredicate(*lhs, *rhs, predicate.uses))
+ }
+
+ sort.Slice(result, func(i, j int) bool {
+ if result[i].LHS.Name == result[j].LHS.Name {
+ return result[i].RHS.Name < result[j].RHS.Name
+ }
+ return result[i].LHS.Name < result[j].LHS.Name
+ })
+
+ return slices.Compact(result)
+}
+
+func getUniqueColNames(ctx *plancontext.PlanningContext, inCols []*sqlparser.ColName) (columns []Column) {
+ for _, col := range inCols {
+ column := createColumn(ctx, col)
+ if column != nil {
+ columns = append(columns, *column)
+ }
+ }
+ sort.Slice(columns, func(i, j int) bool {
+ return columns[i].String() < columns[j].String()
+ })
+
+ return slices.Compact(columns)
+}
+
+func getUniqueColUsages(ctx *plancontext.PlanningContext, inCols []columnUse) (columns []ColumnUse) {
+ for _, col := range inCols {
+ column := createColumn(ctx, col.col)
+ if column != nil {
+ columns = append(columns, ColumnUse{Column: *column, Uses: col.use})
+ }
+ }
+
+ sort.Slice(columns, func(i, j int) bool {
+ return columns[i].Column.String() < columns[j].Column.String()
+ })
+ return slices.Compact(columns)
+}
+
+func createColumn(ctx *plancontext.PlanningContext, col *sqlparser.ColName) *Column {
+ tableInfo, err := ctx.SemTable.TableInfoForExpr(col)
+ if err != nil {
+ return nil
+ }
+ table := tableInfo.GetVindexTable()
+ if table == nil {
+ return nil
+ }
+ return &Column{
+ // we want the escaped versions of the names
+ Table: sqlparser.String(table.Name),
+ Name: sqlparser.String(col.Name),
+ }
+}
diff --git a/go/vt/vtgate/planbuilder/operators/keys_test.go b/go/vt/vtgate/planbuilder/operators/keys_test.go
new file mode 100644
index 00000000000..3fcd69f0b5b
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/operators/keys_test.go
@@ -0,0 +1,66 @@
+/*
+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 operators
+
+import (
+ "encoding/json"
+ "testing"
+
+ "github.com/google/go-cmp/cmp"
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/vt/sqlparser"
+)
+
+func TestMarshalUnmarshal(t *testing.T) {
+ // Test that marshalling and unmarshalling a struct works as expected
+ original := VExplainKeys{
+ StatementType: "SELECT",
+ TableName: []string{"users", "orders"},
+ GroupingColumns: []Column{
+ {Table: "orders", Name: "category"},
+ {Table: "users", Name: "department"},
+ },
+ FilterColumns: []ColumnUse{
+ {Column: Column{Table: "users", Name: "age"}, Uses: sqlparser.GreaterThanOp},
+ {Column: Column{Table: "orders", Name: "total"}, Uses: sqlparser.LessThanOp},
+ {Column: Column{Table: "orders", Name: "`tricky name not`"}, Uses: sqlparser.InOp},
+ },
+ SelectColumns: []Column{
+ {Table: "users", Name: "name"},
+ {Table: "users", Name: "email"},
+ {Table: "orders", Name: "amount"},
+ },
+ JoinPredicates: []JoinPredicate{
+ {LHS: Column{Table: "users", Name: "id"}, RHS: Column{Table: "orders", Name: "user_id"}, Uses: sqlparser.EqualOp},
+ },
+ }
+
+ jsonData, err := json.Marshal(original)
+ require.NoError(t, err)
+
+ t.Logf("Marshalled JSON: %s", jsonData)
+
+ var unmarshalled VExplainKeys
+ err = json.Unmarshal(jsonData, &unmarshalled)
+ require.NoError(t, err)
+
+ if diff := cmp.Diff(original, unmarshalled); diff != "" {
+ t.Errorf("Unmarshalled struct does not match original (-want +got):\n%s", diff)
+ t.FailNow()
+ }
+}
diff --git a/go/vt/vtgate/planbuilder/operators/limit.go b/go/vt/vtgate/planbuilder/operators/limit.go
index 1801e57c1c9..4549b85fcda 100644
--- a/go/vt/vtgate/planbuilder/operators/limit.go
+++ b/go/vt/vtgate/planbuilder/operators/limit.go
@@ -22,8 +22,8 @@ import (
)
type Limit struct {
- Source Operator
- AST *sqlparser.Limit
+ unaryOperator
+ AST *sqlparser.Limit
// Top is true if the limit is a top level limit. To optimise, we push LIMIT to the RHS of joins,
// but we need to still LIMIT the total result set to the top level limit.
@@ -33,21 +33,19 @@ type Limit struct {
Pushed bool
}
-func (l *Limit) Clone(inputs []Operator) Operator {
+func newLimit(op Operator, ast *sqlparser.Limit, top bool) *Limit {
return &Limit{
- Source: inputs[0],
- AST: sqlparser.Clone(l.AST),
- Top: l.Top,
- Pushed: l.Pushed,
+ unaryOperator: newUnaryOp(op),
+ AST: ast,
+ Top: top,
}
}
-func (l *Limit) Inputs() []Operator {
- return []Operator{l.Source}
-}
-
-func (l *Limit) SetInputs(operators []Operator) {
- l.Source = operators[0]
+func (l *Limit) Clone(inputs []Operator) Operator {
+ k := *l
+ k.Source = inputs[0]
+ k.AST = sqlparser.Clone(l.AST)
+ return &k
}
func (l *Limit) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
diff --git a/go/vt/vtgate/planbuilder/operators/mirror.go b/go/vt/vtgate/planbuilder/operators/mirror.go
new file mode 100644
index 00000000000..82a431af0a2
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/operators/mirror.go
@@ -0,0 +1,93 @@
+/*
+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 operators
+
+import (
+ "fmt"
+
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+)
+
+type (
+ PercentBasedMirror struct {
+ binaryOperator
+ Percent float32
+ }
+)
+
+var _ Operator = (*PercentBasedMirror)(nil)
+
+func (m *PercentBasedMirror) Operator() Operator {
+ return m.LHS
+}
+
+func (m *PercentBasedMirror) Target() Operator {
+ return m.RHS
+}
+
+func NewPercentBasedMirror(percent float32, operator, target Operator) *PercentBasedMirror {
+ return &PercentBasedMirror{
+ binaryOperator: newBinaryOp(operator, target),
+ Percent: percent,
+ }
+}
+
+// Clone will return a copy of this operator, protected so changed to the original will not impact the clone
+func (m *PercentBasedMirror) Clone(inputs []Operator) Operator {
+ cloneMirror := *m
+ cloneMirror.SetInputs(inputs)
+ return &cloneMirror
+}
+
+// AddPredicate is used to push predicates. It pushed it as far down as is possible in the tree.
+// If we encounter a join and the predicate depends on both sides of the join, the predicate will be split into two parts,
+// where data is fetched from the LHS of the join to be used in the evaluation on the RHS
+// TODO: we should remove this and replace it with rewriters
+func (m *PercentBasedMirror) AddPredicate(*plancontext.PlanningContext, sqlparser.Expr) Operator {
+ panic(vterrors.VT13001("not supported"))
+}
+
+func (m *PercentBasedMirror) AddColumn(*plancontext.PlanningContext, bool, bool, *sqlparser.AliasedExpr) int {
+ panic(vterrors.VT13001("not supported"))
+}
+
+func (m *PercentBasedMirror) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int {
+ return m.Operator().FindCol(ctx, expr, underRoute)
+}
+
+func (m *PercentBasedMirror) GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr {
+ return m.Operator().GetColumns(ctx)
+}
+
+func (m *PercentBasedMirror) GetSelectExprs(ctx *plancontext.PlanningContext) sqlparser.SelectExprs {
+ return m.Operator().GetSelectExprs(ctx)
+}
+
+func (m *PercentBasedMirror) ShortDescription() string {
+ return fmt.Sprintf("PercentBasedMirror (%.02f%%)", m.Percent)
+}
+
+func (m *PercentBasedMirror) GetOrdering(ctx *plancontext.PlanningContext) []OrderBy {
+ return m.Operator().GetOrdering(ctx)
+}
+
+// AddWSColumn implements Operator.
+func (m *PercentBasedMirror) AddWSColumn(*plancontext.PlanningContext, int, bool) int {
+ panic(vterrors.VT13001("not supported"))
+}
diff --git a/go/vt/vtgate/planbuilder/operators/offset_planning.go b/go/vt/vtgate/planbuilder/operators/offset_planning.go
index e8301c18823..62abff29e57 100644
--- a/go/vt/vtgate/planbuilder/operators/offset_planning.go
+++ b/go/vt/vtgate/planbuilder/operators/offset_planning.go
@@ -25,13 +25,13 @@ import (
"vitess.io/vitess/go/vt/vtgate/semantics"
)
+type offsettable interface {
+ Operator
+ planOffsets(ctx *plancontext.PlanningContext) Operator
+}
+
// planOffsets will walk the tree top down, adding offset information to columns in the tree for use in further optimization,
func planOffsets(ctx *plancontext.PlanningContext, root Operator) Operator {
- type offsettable interface {
- Operator
- planOffsets(ctx *plancontext.PlanningContext) Operator
- }
-
visitor := func(in Operator, _ semantics.TableSet, _ bool) (Operator, *ApplyResult) {
switch op := in.(type) {
case *Horizon:
@@ -120,50 +120,6 @@ func findAggregatorInSource(op Operator) *Aggregator {
}
}
-// addColumnsToInput adds columns needed by an operator to its input.
-// This happens only when the filter expression can be retrieved as an offset from the underlying mysql.
-func addColumnsToInput(ctx *plancontext.PlanningContext, root Operator) Operator {
-
- addColumnsNeededByFilter := func(in Operator, _ semantics.TableSet, _ bool) (Operator, *ApplyResult) {
- addedCols := false
- filter, ok := in.(*Filter)
- if !ok {
- return in, NoRewrite
- }
-
- var neededAggrs []sqlparser.Expr
- extractAggrs := func(cursor *sqlparser.CopyOnWriteCursor) {
- node := cursor.Node()
- if ctx.IsAggr(node) {
- neededAggrs = append(neededAggrs, node.(sqlparser.Expr))
- }
- }
-
- for _, expr := range filter.Predicates {
- _ = sqlparser.CopyOnRewrite(expr, dontEnterSubqueries, extractAggrs, nil)
- }
-
- if neededAggrs == nil {
- return in, NoRewrite
- }
-
- aggregator := findAggregatorInSource(filter.Source)
- for _, aggr := range neededAggrs {
- if aggregator.FindCol(ctx, aggr, false) == -1 {
- aggregator.addColumnWithoutPushing(ctx, aeWrap(aggr), false)
- addedCols = true
- }
- }
-
- if addedCols {
- return in, Rewrote("added columns because filter needs it")
- }
- return in, NoRewrite
- }
-
- return TopDown(root, TableID, addColumnsNeededByFilter, stopAtRoute)
-}
-
// isolateDistinctFromUnion will pull out the distinct from a union operator
func isolateDistinctFromUnion(_ *plancontext.PlanningContext, root Operator) Operator {
visitor := func(in Operator, _ semantics.TableSet, isRoot bool) (Operator, *ApplyResult) {
@@ -174,10 +130,7 @@ func isolateDistinctFromUnion(_ *plancontext.PlanningContext, root Operator) Ope
union.distinct = false
- distinct := &Distinct{
- Required: true,
- Source: union,
- }
+ distinct := newDistinct(union, nil, true)
return distinct, Rewrote("pulled out DISTINCT from union")
}
diff --git a/go/vt/vtgate/planbuilder/operators/operator.go b/go/vt/vtgate/planbuilder/operators/operator.go
index f1a38974c93..42658e4c52e 100644
--- a/go/vt/vtgate/planbuilder/operators/operator.go
+++ b/go/vt/vtgate/planbuilder/operators/operator.go
@@ -54,7 +54,8 @@ type (
// Inputs returns the inputs for this operator
Inputs() []Operator
- // SetInputs changes the inputs for this op
+ // SetInputs changes the inputs for this op.
+ // We don't need to check the size of the inputs, as the planner will ensure that the inputs are correct
SetInputs([]Operator)
// AddPredicate is used to push predicates. It pushed it as far down as is possible in the tree.
@@ -85,8 +86,46 @@ type (
// See GroupBy#SimplifiedExpr for more details about this
SimplifiedExpr sqlparser.Expr
}
+
+ unaryOperator struct {
+ Operator
+ Source Operator
+ }
+
+ binaryOperator struct {
+ Operator
+ LHS, RHS Operator
+ }
)
+func newUnaryOp(source Operator) unaryOperator {
+ return unaryOperator{Source: source}
+}
+
+func newBinaryOp(l, r Operator) binaryOperator {
+ return binaryOperator{
+ LHS: l,
+ RHS: r,
+ }
+}
+
+func (s *unaryOperator) Inputs() []Operator {
+ return []Operator{s.Source}
+}
+
+func (s *unaryOperator) SetInputs(operators []Operator) {
+ s.Source = operators[0]
+}
+
+func (b *binaryOperator) Inputs() []Operator {
+ return []Operator{b.LHS, b.RHS}
+}
+
+func (b *binaryOperator) SetInputs(operators []Operator) {
+ b.LHS = operators[0]
+ b.RHS = operators[1]
+}
+
// Map takes in a mapping function and applies it to both the expression in OrderBy.
func (ob OrderBy) Map(mappingFunc func(sqlparser.Expr) sqlparser.Expr) OrderBy {
return OrderBy{
diff --git a/go/vt/vtgate/planbuilder/operators/ordering.go b/go/vt/vtgate/planbuilder/operators/ordering.go
index 94c4f3dd846..7e40b420f9e 100644
--- a/go/vt/vtgate/planbuilder/operators/ordering.go
+++ b/go/vt/vtgate/planbuilder/operators/ordering.go
@@ -26,7 +26,7 @@ import (
)
type Ordering struct {
- Source Operator
+ unaryOperator
Offset []int
WOffset []int
@@ -35,21 +35,20 @@ type Ordering struct {
}
func (o *Ordering) Clone(inputs []Operator) Operator {
- return &Ordering{
- Source: inputs[0],
- Offset: slices.Clone(o.Offset),
- WOffset: slices.Clone(o.WOffset),
- Order: slices.Clone(o.Order),
- ResultColumns: o.ResultColumns,
- }
-}
+ klone := *o
+ klone.Source = inputs[0]
+ klone.Offset = slices.Clone(o.Offset)
+ klone.WOffset = slices.Clone(o.WOffset)
+ klone.Order = slices.Clone(o.Order)
-func (o *Ordering) Inputs() []Operator {
- return []Operator{o.Source}
+ return &klone
}
-func (o *Ordering) SetInputs(operators []Operator) {
- o.Source = operators[0]
+func newOrdering(src Operator, order []OrderBy) Operator {
+ return &Ordering{
+ unaryOperator: newUnaryOp(src),
+ Order: order,
+ }
}
func (o *Ordering) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
@@ -82,17 +81,25 @@ func (o *Ordering) GetOrdering(*plancontext.PlanningContext) []OrderBy {
}
func (o *Ordering) planOffsets(ctx *plancontext.PlanningContext) Operator {
+ var weightStrings []*OrderBy
+
for _, order := range o.Order {
offset := o.Source.AddColumn(ctx, true, false, aeWrap(order.SimplifiedExpr))
o.Offset = append(o.Offset, offset)
if !ctx.NeedsWeightString(order.SimplifiedExpr) {
- o.WOffset = append(o.WOffset, -1)
+ weightStrings = append(weightStrings, nil)
continue
}
+ weightStrings = append(weightStrings, &order)
+ }
- wsExpr := &sqlparser.WeightStringFuncExpr{Expr: order.SimplifiedExpr}
- offset = o.Source.AddColumn(ctx, true, false, aeWrap(wsExpr))
+ for i, order := range weightStrings {
+ if order == nil {
+ o.WOffset = append(o.WOffset, -1)
+ continue
+ }
+ offset := o.Source.AddWSColumn(ctx, o.Offset[i], false)
o.WOffset = append(o.WOffset, offset)
}
return nil
diff --git a/go/vt/vtgate/planbuilder/operators/phases.go b/go/vt/vtgate/planbuilder/operators/phases.go
index 9f2178bae05..eb6c42b8724 100644
--- a/go/vt/vtgate/planbuilder/operators/phases.go
+++ b/go/vt/vtgate/planbuilder/operators/phases.go
@@ -36,6 +36,7 @@ const (
initialPlanning
pullDistinctFromUnion
delegateAggregation
+ recursiveCTEHorizons
addAggrOrdering
cleanOutPerfDistinct
dmlWithInput
@@ -53,6 +54,8 @@ func (p Phase) String() string {
return "pull distinct from UNION"
case delegateAggregation:
return "split aggregation between vtgate and mysql"
+ case recursiveCTEHorizons:
+ return "expand recursive CTE horizons"
case addAggrOrdering:
return "optimize aggregations with ORDER BY"
case cleanOutPerfDistinct:
@@ -72,6 +75,8 @@ func (p Phase) shouldRun(s semantics.QuerySignature) bool {
return s.Union
case delegateAggregation:
return s.Aggregation
+ case recursiveCTEHorizons:
+ return s.RecursiveCTE
case addAggrOrdering:
return s.Aggregation
case cleanOutPerfDistinct:
@@ -93,6 +98,8 @@ func (p Phase) act(ctx *plancontext.PlanningContext, op Operator) Operator {
return enableDelegateAggregation(ctx, op)
case addAggrOrdering:
return addOrderingForAllAggregations(ctx, op)
+ case recursiveCTEHorizons:
+ return planRecursiveCTEHorizons(ctx, op)
case cleanOutPerfDistinct:
return removePerformanceDistinctAboveRoute(ctx, op)
case subquerySettling:
@@ -207,7 +214,7 @@ func removePerformanceDistinctAboveRoute(_ *plancontext.PlanningContext, op Oper
}
func enableDelegateAggregation(ctx *plancontext.PlanningContext, op Operator) Operator {
- return addColumnsToInput(ctx, op)
+ return prepareForAggregationPushing(ctx, op)
}
// addOrderingForAllAggregations is run we have pushed down Aggregators as far down as possible.
@@ -242,10 +249,7 @@ func addOrderingFor(aggrOp *Aggregator) {
SimplifiedExpr: aggrOp.DistinctExpr,
})
}
- aggrOp.Source = &Ordering{
- Source: aggrOp.Source,
- Order: orderBys,
- }
+ aggrOp.Source = newOrdering(aggrOp.Source, orderBys)
}
func needsOrdering(ctx *plancontext.PlanningContext, in *Aggregator) bool {
@@ -290,10 +294,101 @@ func addLiteralGroupingToRHS(in *ApplyJoin) (Operator, *ApplyResult) {
return nil
}
if len(aggr.Grouping) == 0 {
- gb := sqlparser.NewIntLiteral(".0")
+ gb := sqlparser.NewFloatLiteral(".0")
aggr.Grouping = append(aggr.Grouping, NewGroupBy(gb))
}
return nil
})
return in, NoRewrite
}
+
+// prepareForAggregationPushing adds columns needed by an operator to its input.
+// This happens only when the filter expression can be retrieved as an offset from the underlying mysql.
+func prepareForAggregationPushing(ctx *plancontext.PlanningContext, root Operator) Operator {
+ return TopDown(root, TableID, func(in Operator, _ semantics.TableSet, _ bool) (Operator, *ApplyResult) {
+ filter, ok := in.(*Filter)
+ if !ok {
+ return in, NoRewrite
+ }
+
+ var neededAggrs []sqlparser.Expr
+ extractAggrs := func(cursor *sqlparser.CopyOnWriteCursor) {
+ node := cursor.Node()
+ if ctx.IsAggr(node) {
+ neededAggrs = append(neededAggrs, node.(sqlparser.Expr))
+ }
+ }
+
+ for _, expr := range filter.Predicates {
+ _ = sqlparser.CopyOnRewrite(expr, dontEnterSubqueries, extractAggrs, nil)
+ }
+
+ if neededAggrs == nil {
+ return in, NoRewrite
+ }
+
+ addedCols := false
+ aggregator := findAggregatorInSource(filter.Source)
+ for _, aggr := range neededAggrs {
+ if aggregator.FindCol(ctx, aggr, false) == -1 {
+ aggregator.addColumnWithoutPushing(ctx, aeWrap(aggr), false)
+ addedCols = true
+ }
+ }
+
+ if addedCols {
+ return in, Rewrote("added columns because filter needs it")
+ }
+ return in, NoRewrite
+ }, stopAtRoute)
+}
+
+// prepareForAggregationPushing adds columns needed by an operator to its input.
+// This happens only when the filter expression can be retrieved as an offset from the underlying mysql.
+func planRecursiveCTEHorizons(ctx *plancontext.PlanningContext, root Operator) Operator {
+ return TopDown(root, TableID, func(in Operator, _ semantics.TableSet, _ bool) (Operator, *ApplyResult) {
+ // These recursive CTEs have not been pushed under a route, so we will have to evaluate it one the vtgate
+ // That means that we need to turn anything that is coming from the recursion into arguments
+ rcte, ok := in.(*RecurseCTE)
+ if !ok {
+ return in, NoRewrite
+ }
+ hz := rcte.Horizon
+ hz.Source = rcte.Term()
+ newTerm, _ := expandHorizon(ctx, hz)
+ pr := findProjection(newTerm)
+ ap, err := pr.GetAliasedProjections()
+ if err != nil {
+ panic(vterrors.VT09015())
+ }
+
+ // We need to break the expressions into LHS and RHS, and store them in the CTE for later use
+ projections := slice.Map(ap, func(p *ProjExpr) *plancontext.RecurseExpression {
+ recurseExpression := breakCTEExpressionInLhsAndRhs(ctx, p.EvalExpr, rcte.LeftID)
+ p.EvalExpr = recurseExpression.RightExpr
+ return recurseExpression
+ })
+ rcte.Projections = projections
+ rcte.RHS = newTerm
+ return rcte, Rewrote("expanded horizon on term side of recursive CTE")
+ }, stopAtRoute)
+}
+
+func findProjection(op Operator) *Projection {
+ for {
+ proj, ok := op.(*Projection)
+ if ok {
+ return proj
+ }
+ inputs := op.Inputs()
+ if len(inputs) != 1 {
+ panic(vterrors.VT13001("unexpected multiple inputs"))
+ }
+ src := inputs[0]
+ _, isRoute := src.(*Route)
+ if isRoute {
+ panic(vterrors.VT13001("failed to find the projection"))
+ }
+ op = src
+ }
+}
diff --git a/go/vt/vtgate/planbuilder/operators/plan_query.go b/go/vt/vtgate/planbuilder/operators/plan_query.go
index 4d371942c26..dc83c89c72c 100644
--- a/go/vt/vtgate/planbuilder/operators/plan_query.go
+++ b/go/vt/vtgate/planbuilder/operators/plan_query.go
@@ -43,11 +43,14 @@ import (
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+ "vitess.io/vitess/go/vt/vtgate/semantics"
)
type (
// helper type that implements Inputs() returning nil
- noInputs struct{}
+ nullaryOperator struct {
+ Operator
+ }
// helper type that implements AddColumn() returning an error
noColumns struct{}
@@ -60,7 +63,7 @@ type (
func PlanQuery(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) (result Operator, err error) {
defer PanicHandler(&err)
- op := translateQueryToOp(ctx, stmt)
+ op := translateQueryToOpWithMirroring(ctx, stmt)
if DebugOperatorTree {
fmt.Println("Initial tree:")
@@ -71,13 +74,35 @@ func PlanQuery(ctx *plancontext.PlanningContext, stmt sqlparser.Statement) (resu
checkValid(op)
op = planQuery(ctx, op)
- _, isRoute := op.(*Route)
- if !isRoute && ctx.SemTable.NotSingleRouteErr != nil {
- // If we got here, we don't have a single shard plan
- return nil, ctx.SemTable.NotSingleRouteErr
+ if err := checkSingleRouteError(ctx, op); err != nil {
+ return nil, err
+ }
+
+ return op, nil
+}
+
+// checkSingleRouteError checks if the query has a NotSingleRouteErr and more than one route, and returns an error if it does
+func checkSingleRouteError(ctx *plancontext.PlanningContext, op Operator) error {
+ if ctx.SemTable.NotSingleRouteErr == nil {
+ return nil
+ }
+ routes := 0
+ visitF := func(op Operator, _ semantics.TableSet, _ bool) (Operator, *ApplyResult) {
+ switch op.(type) {
+ case *Route:
+ routes++
+ }
+ return op, NoRewrite
+ }
+
+ // we'll walk the tree and count the number of routes
+ TopDown(op, TableID, visitF, stopAtRoute)
+
+ if routes <= 1 {
+ return nil
}
- return op, err
+ return ctx.SemTable.NotSingleRouteErr
}
func PanicHandler(err *error) {
@@ -94,14 +119,14 @@ func PanicHandler(err *error) {
}
// Inputs implements the Operator interface
-func (noInputs) Inputs() []Operator {
+func (nullaryOperator) Inputs() []Operator {
return nil
}
// SetInputs implements the Operator interface
-func (noInputs) SetInputs(ops []Operator) {
+func (nullaryOperator) SetInputs(ops []Operator) {
if len(ops) > 0 {
- panic("the noInputs operator does not have inputs")
+ panic("the nullaryOperator operator does not have inputs")
}
}
diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go
index d8ede63b612..e894ab433b4 100644
--- a/go/vt/vtgate/planbuilder/operators/projection.go
+++ b/go/vt/vtgate/planbuilder/operators/projection.go
@@ -32,7 +32,7 @@ import (
// Projection is used when we need to evaluate expressions on the vtgate
// It uses the evalengine to accomplish its goal
type Projection struct {
- Source Operator
+ unaryOperator
// Columns contain the expressions as viewed from the outside of this operator
Columns ProjCols
@@ -87,7 +87,7 @@ type (
ProjExpr struct {
Original *sqlparser.AliasedExpr // this is the expression the user asked for. should only be used to decide on the column alias
- EvalExpr sqlparser.Expr // EvalExpr is the expression that will be evaluated at runtime
+ EvalExpr sqlparser.Expr // EvalExpr represents the expression evaluated at runtime or used when the ProjExpr is pushed under a route
ColExpr sqlparser.Expr // ColExpr is used during planning to figure out which column this ProjExpr is representing
Info ExprInfo // Here we store information about evalengine, offsets or subqueries
}
@@ -127,8 +127,8 @@ func newProjExprWithInner(ae *sqlparser.AliasedExpr, in sqlparser.Expr) *ProjExp
func newAliasedProjection(src Operator) *Projection {
return &Projection{
- Source: src,
- Columns: AliasedProjections{},
+ unaryOperator: newUnaryOp(src),
+ Columns: AliasedProjections{},
}
}
@@ -383,8 +383,18 @@ func (p *Projection) addColumn(
return p.addProjExpr(pe)
}
- // we need to push down this column to our input
- inputOffset := p.Source.AddColumn(ctx, true, addToGroupBy, ae)
+ var inputOffset int
+ if nothingNeedsFetching(ctx, expr) {
+ // if we don't need to fetch anything, we could just evaluate it in the projection
+ // we still check if it's there - if it is, we can, we should use it
+ inputOffset = p.Source.FindCol(ctx, expr, false)
+ if inputOffset < 0 {
+ return p.addProjExpr(pe)
+ }
+ } else {
+ // we need to push down this column to our input
+ inputOffset = p.Source.AddColumn(ctx, true, addToGroupBy, ae)
+ }
pe.Info = Offset(inputOffset) // since we already know the offset, let's save the information
return p.addProjExpr(pe)
@@ -395,20 +405,9 @@ func (po *EvalEngine) expr() {}
func (po SubQueryExpression) expr() {}
func (p *Projection) Clone(inputs []Operator) Operator {
- return &Projection{
- Source: inputs[0],
- Columns: p.Columns, // TODO don't think we need to deep clone here
- DT: p.DT,
- FromAggr: p.FromAggr,
- }
-}
-
-func (p *Projection) Inputs() []Operator {
- return []Operator{p.Source}
-}
-
-func (p *Projection) SetInputs(operators []Operator) {
- p.Source = operators[0]
+ klone := *p
+ klone.Source = inputs[0]
+ return &klone
}
func (p *Projection) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.Expr) Operator {
diff --git a/go/vt/vtgate/planbuilder/operators/projection_pushing.go b/go/vt/vtgate/planbuilder/operators/projection_pushing.go
index b5c6a10bb78..1d1a5a04501 100644
--- a/go/vt/vtgate/planbuilder/operators/projection_pushing.go
+++ b/go/vt/vtgate/planbuilder/operators/projection_pushing.go
@@ -320,7 +320,7 @@ func splitSubqueryExpression(
alias string,
) applyJoinColumn {
col := join.getJoinColumnFor(ctx, pe.Original, pe.ColExpr, false)
- return pushDownSplitJoinCol(col, lhs, pe, alias, rhs)
+ return pushDownSplitJoinCol(col, lhs, rhs, pe, alias)
}
func splitUnexploredExpression(
@@ -334,23 +334,49 @@ func splitUnexploredExpression(
original := sqlparser.Clone(pe.Original)
expr := pe.ColExpr
- var colName *sqlparser.ColName
- if dt != nil {
- if !pe.isSameInAndOut(ctx) {
- panic(vterrors.VT13001("derived table columns must be the same in and out"))
- }
- colName = sqlparser.NewColNameWithQualifier(pe.Original.ColumnName(), sqlparser.NewTableName(dt.Alias))
- ctx.SemTable.CopySemanticInfo(expr, colName)
- }
-
// Get a applyJoinColumn for the current expression.
col := join.getJoinColumnFor(ctx, original, expr, false)
- col.DTColName = colName
- return pushDownSplitJoinCol(col, lhs, pe, alias, rhs)
+ if dt == nil {
+ return pushDownSplitJoinCol(col, lhs, rhs, pe, alias)
+ }
+
+ if !pe.isSameInAndOut(ctx) {
+ panic(vterrors.VT13001("derived table columns must be the same in and out"))
+ }
+ // we are pushing a derived projection through a join. that means that after this rewrite, we are on top of the
+ // derived table divider, and can only see the projected columns, not the underlying expressions
+ colName := sqlparser.NewColNameWithQualifier(pe.Original.ColumnName(), sqlparser.NewTableName(dt.Alias))
+ ctx.SemTable.CopySemanticInfo(expr, colName)
+ col.Original = colName
+ if alias == "" {
+ alias = pe.Original.ColumnName()
+ }
+
+ // Update the left and right child columns and names based on the applyJoinColumn type.
+ switch {
+ case col.IsPureLeft():
+ lhs.add(pe, alias)
+ col.LHSExprs[0].Expr = colName
+ case col.IsPureRight():
+ rhs.add(pe, alias)
+ col.RHSExpr = colName
+ default:
+ for _, lhsExpr := range col.LHSExprs {
+ ae := aeWrap(lhsExpr.Expr)
+ columnName := ae.ColumnName()
+ lhs.add(newProjExpr(ae), columnName)
+ }
+ innerPE := newProjExprWithInner(pe.Original, col.RHSExpr)
+ innerPE.ColExpr = col.RHSExpr
+ col.RHSExpr = colName
+ innerPE.Info = pe.Info
+ rhs.add(innerPE, alias)
+ }
+ return col
}
-func pushDownSplitJoinCol(col applyJoinColumn, lhs *projector, pe *ProjExpr, alias string, rhs *projector) applyJoinColumn {
+func pushDownSplitJoinCol(col applyJoinColumn, lhs, rhs *projector, pe *ProjExpr, alias string) applyJoinColumn {
// Update the left and right child columns and names based on the applyJoinColumn type.
switch {
case col.IsPureLeft():
diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go
index e88fb53edb3..5fe0c7773c1 100644
--- a/go/vt/vtgate/planbuilder/operators/query_planning.go
+++ b/go/vt/vtgate/planbuilder/operators/query_planning.go
@@ -102,11 +102,21 @@ func runRewriters(ctx *plancontext.PlanningContext, root Operator) Operator {
return tryPushDelete(in)
case *Update:
return tryPushUpdate(in)
+ case *RecurseCTE:
+ return tryMergeRecurse(ctx, in)
+
default:
return in, NoRewrite
}
}
+ if pbm, ok := root.(*PercentBasedMirror); ok {
+ pbm.SetInputs([]Operator{
+ runRewriters(ctx, pbm.Operator()),
+ runRewriters(ctx.UseMirror(), pbm.Target()),
+ })
+ }
+
return FixedPointBottomUp(root, TableID, visitor, stopAtRoute)
}
@@ -150,27 +160,15 @@ func pushLockAndComment(l *LockAndComment) (Operator, *ApplyResult) {
src.Lock = l.Lock.GetHighestOrderLock(src.Lock)
return src, Rewrote("put lock and comment into route")
case *SubQueryContainer:
- src.Outer = &LockAndComment{
- Source: src.Outer,
- Comments: l.Comments,
- Lock: l.Lock,
- }
+ src.Outer = newLockAndComment(src.Outer, l.Comments, l.Lock)
for _, sq := range src.Inner {
- sq.Subquery = &LockAndComment{
- Source: sq.Subquery,
- Comments: l.Comments,
- Lock: l.Lock,
- }
+ sq.Subquery = newLockAndComment(sq.Subquery, l.Comments, l.Lock)
}
return src, Rewrote("push lock and comment into subquery container")
default:
inputs := src.Inputs()
for i, op := range inputs {
- inputs[i] = &LockAndComment{
- Source: op,
- Comments: l.Comments,
- Lock: l.Lock,
- }
+ inputs[i] = newLockAndComment(op, l.Comments, l.Lock)
}
src.SetInputs(inputs)
return src, Rewrote("pushed down lock and comments")
@@ -426,10 +424,7 @@ func createPushedLimit(ctx *plancontext.PlanningContext, src Operator, orig *Lim
pushedLimit.Rowcount = getLimitExpression(ctx, plus)
pushedLimit.Offset = nil
}
- return &Limit{
- Source: src,
- AST: pushedLimit,
- }
+ return newLimit(src, pushedLimit, false)
}
// getLimitExpression is a helper function to simplify an expression using the evalengine
@@ -496,11 +491,8 @@ func setUpperLimit(in *Limit) (Operator, *ApplyResult) {
return SkipChildren
}
case *Route:
- newSrc := &Limit{
- Source: op.Source,
- AST: &sqlparser.Limit{Rowcount: sqlparser.NewArgument(engine.UpperLimitStr)},
- }
- op.Source = newSrc
+ ast := &sqlparser.Limit{Rowcount: sqlparser.NewArgument(engine.UpperLimitStr)}
+ op.Source = newLimit(op.Source, ast, false)
result = result.Merge(Rewrote("push upper limit under route"))
return SkipChildren
}
@@ -733,7 +725,6 @@ func pushFilterUnderProjection(ctx *plancontext.PlanningContext, filter *Filter,
}
}
return Swap(filter, projection, "push filter under projection")
-
}
func tryPushDistinct(in *Distinct) (Operator, *ApplyResult) {
@@ -753,7 +744,7 @@ func tryPushDistinct(in *Distinct) (Operator, *ApplyResult) {
return in, NoRewrite
}
- src.Source = &Distinct{Source: src.Source}
+ src.Source = newDistinct(src.Source, nil, false)
in.PushedPerformance = true
return in, Rewrote("added distinct under route - kept original")
@@ -763,14 +754,14 @@ func tryPushDistinct(in *Distinct) (Operator, *ApplyResult) {
return src, Rewrote("remove double distinct")
case *Union:
for i := range src.Sources {
- src.Sources[i] = &Distinct{Source: src.Sources[i]}
+ src.Sources[i] = newDistinct(src.Sources[i], nil, false)
}
in.PushedPerformance = true
return in, Rewrote("push down distinct under union")
case JoinOp:
- src.SetLHS(&Distinct{Source: src.GetLHS()})
- src.SetRHS(&Distinct{Source: src.GetRHS()})
+ src.SetLHS(newDistinct(src.GetLHS(), nil, false))
+ src.SetRHS(newDistinct(src.GetRHS(), nil, false))
in.PushedPerformance = true
if in.Required {
@@ -821,10 +812,7 @@ func tryPushUnion(ctx *plancontext.PlanningContext, op *Union) (Operator, *Apply
return result, Rewrote("push union under route")
}
- return &Distinct{
- Source: result,
- Required: true,
- }, Rewrote("push union under route")
+ return newDistinct(result, nil, true), Rewrote("push union under route")
}
if len(sources) == len(op.Sources) {
diff --git a/go/vt/vtgate/planbuilder/operators/querygraph.go b/go/vt/vtgate/planbuilder/operators/querygraph.go
index 8e8572f7dfa..98cd9ada64a 100644
--- a/go/vt/vtgate/planbuilder/operators/querygraph.go
+++ b/go/vt/vtgate/planbuilder/operators/querygraph.go
@@ -42,7 +42,7 @@ type (
// NoDeps contains the predicates that can be evaluated anywhere.
NoDeps sqlparser.Expr
- noInputs
+ nullaryOperator
noColumns
}
diff --git a/go/vt/vtgate/planbuilder/operators/recurse_cte.go b/go/vt/vtgate/planbuilder/operators/recurse_cte.go
new file mode 100644
index 00000000000..61474b663d6
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/operators/recurse_cte.go
@@ -0,0 +1,211 @@
+/*
+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 operators
+
+import (
+ "fmt"
+ "slices"
+ "strings"
+
+ "golang.org/x/exp/maps"
+
+ "vitess.io/vitess/go/slice"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
+ "vitess.io/vitess/go/vt/vtgate/semantics"
+)
+
+// RecurseCTE is used to represent a recursive CTE
+type RecurseCTE struct {
+ binaryOperator
+
+ // Def is the CTE definition according to the semantics
+ Def *semantics.CTE
+
+ // Expressions are the predicates that are needed on the recurse side of the CTE
+ Predicates []*plancontext.RecurseExpression
+ Projections []*plancontext.RecurseExpression
+
+ // Vars is the map of variables that are sent between the two parts of the recursive CTE
+ // It's filled in at offset planning time
+ Vars map[string]int
+
+ // MyTableID is the id of the CTE
+ MyTableInfo *semantics.CTETable
+
+ // Horizon is stored here until we either expand it or push it under a route
+ Horizon *Horizon
+
+ // The LeftID is the id of the left side of the CTE
+ LeftID,
+
+ // The OuterID is the id for this use of the CTE
+ OuterID semantics.TableSet
+
+ // Distinct is used to determine if the result set should be distinct
+ Distinct bool
+}
+
+var _ Operator = (*RecurseCTE)(nil)
+
+func newRecurse(
+ ctx *plancontext.PlanningContext,
+ def *semantics.CTE,
+ seed, term Operator,
+ predicates []*plancontext.RecurseExpression,
+ horizon *Horizon,
+ leftID, outerID semantics.TableSet,
+ distinct bool,
+) *RecurseCTE {
+ for _, pred := range predicates {
+ ctx.AddJoinPredicates(pred.Original, pred.RightExpr)
+ }
+ return &RecurseCTE{
+ binaryOperator: newBinaryOp(seed, term),
+ Def: def,
+ Predicates: predicates,
+ Horizon: horizon,
+ LeftID: leftID,
+ OuterID: outerID,
+ Distinct: distinct,
+ }
+}
+
+func (r *RecurseCTE) Clone(inputs []Operator) Operator {
+ klone := *r
+ klone.LHS = inputs[0]
+ klone.RHS = inputs[1]
+ klone.Vars = maps.Clone(r.Vars)
+ klone.Predicates = slices.Clone(r.Predicates)
+ klone.Projections = slices.Clone(r.Projections)
+ return &klone
+}
+
+func (r *RecurseCTE) AddPredicate(_ *plancontext.PlanningContext, e sqlparser.Expr) Operator {
+ return newFilter(r, e)
+}
+
+func (r *RecurseCTE) AddColumn(ctx *plancontext.PlanningContext, _, _ bool, expr *sqlparser.AliasedExpr) int {
+ r.makeSureWeHaveTableInfo(ctx)
+ e := semantics.RewriteDerivedTableExpression(expr.Expr, r.MyTableInfo)
+ offset := r.Seed().FindCol(ctx, e, false)
+ if offset == -1 {
+ panic(vterrors.VT13001("CTE column not found"))
+ }
+ return offset
+}
+
+func (r *RecurseCTE) makeSureWeHaveTableInfo(ctx *plancontext.PlanningContext) {
+ if r.MyTableInfo == nil {
+ for _, table := range ctx.SemTable.Tables {
+ cte, ok := table.(*semantics.CTETable)
+ if !ok {
+ continue
+ }
+ if cte.CTE == r.Def {
+ r.MyTableInfo = cte
+ break
+ }
+ }
+ if r.MyTableInfo == nil {
+ panic(vterrors.VT13001("CTE not found"))
+ }
+ }
+}
+
+func (r *RecurseCTE) AddWSColumn(ctx *plancontext.PlanningContext, offset int, underRoute bool) int {
+ seed := r.Seed().AddWSColumn(ctx, offset, underRoute)
+ term := r.Term().AddWSColumn(ctx, offset, underRoute)
+ if seed != term {
+ panic(vterrors.VT13001("CTE columns don't match"))
+ }
+ return seed
+}
+
+func (r *RecurseCTE) FindCol(ctx *plancontext.PlanningContext, expr sqlparser.Expr, underRoute bool) int {
+ r.makeSureWeHaveTableInfo(ctx)
+ expr = semantics.RewriteDerivedTableExpression(expr, r.MyTableInfo)
+ return r.Seed().FindCol(ctx, expr, underRoute)
+}
+
+func (r *RecurseCTE) GetColumns(ctx *plancontext.PlanningContext) []*sqlparser.AliasedExpr {
+ return r.Seed().GetColumns(ctx)
+}
+
+func (r *RecurseCTE) GetSelectExprs(ctx *plancontext.PlanningContext) sqlparser.SelectExprs {
+ return r.Seed().GetSelectExprs(ctx)
+}
+
+func (r *RecurseCTE) ShortDescription() string {
+ distinct := ""
+ if r.Distinct {
+ distinct = "distinct "
+ }
+ if len(r.Vars) > 0 {
+ return fmt.Sprintf("%s%v", distinct, r.Vars)
+ }
+ expressions := slice.Map(r.expressions(), func(expr *plancontext.RecurseExpression) string {
+ return sqlparser.String(expr.Original)
+ })
+ return fmt.Sprintf("%s%v %v", distinct, r.Def.Name, strings.Join(expressions, ", "))
+}
+
+func (r *RecurseCTE) GetOrdering(*plancontext.PlanningContext) []OrderBy {
+ // RecurseCTE is a special case. It never guarantees any ordering.
+ return nil
+}
+
+func (r *RecurseCTE) expressions() []*plancontext.RecurseExpression {
+ return append(r.Predicates, r.Projections...)
+}
+
+func (r *RecurseCTE) planOffsets(ctx *plancontext.PlanningContext) Operator {
+ r.Vars = make(map[string]int)
+ columns := r.Seed().GetColumns(ctx)
+ for _, expr := range r.expressions() {
+ outer:
+ for _, lhsExpr := range expr.LeftExprs {
+ _, found := r.Vars[lhsExpr.Name]
+ if found {
+ continue
+ }
+
+ for offset, column := range columns {
+ if lhsExpr.Expr.Name.EqualString(column.ColumnName()) {
+ r.Vars[lhsExpr.Name] = offset
+ continue outer
+ }
+ }
+
+ panic(vterrors.VT13001("couldn't find column"))
+ }
+ }
+ return r
+}
+
+func (r *RecurseCTE) introducesTableID() semantics.TableSet {
+ return r.OuterID
+}
+
+func (r *RecurseCTE) Seed() Operator {
+ return r.LHS
+}
+
+func (r *RecurseCTE) Term() Operator {
+ return r.RHS
+}
diff --git a/go/vt/vtgate/planbuilder/operators/rewriters.go b/go/vt/vtgate/planbuilder/operators/rewriters.go
index ab9fe66e368..1864eb952e1 100644
--- a/go/vt/vtgate/planbuilder/operators/rewriters.go
+++ b/go/vt/vtgate/planbuilder/operators/rewriters.go
@@ -164,6 +164,16 @@ func TopDown(
// Swap takes a tree like a->b->c and swaps `a` and `b`, so we end up with b->a->c
func Swap(parent, child Operator, message string) (Operator, *ApplyResult) {
+ unaryParent, isUnary := parent.(*unaryOperator)
+ if isUnary {
+ unaryChild, isUnary := child.(*unaryOperator)
+ if isUnary {
+ // If both the parent and child are unary operators, we can just swap the sources
+ unaryParent.Source, unaryChild.Source = unaryChild.Source, unaryParent.Source
+ return parent, Rewrote(message)
+ }
+ }
+
c := child.Inputs()
if len(c) != 1 {
panic(vterrors.VT13001("Swap can only be used on single input operators"))
@@ -200,34 +210,60 @@ func bottomUp(
return root, NoRewrite
}
- oldInputs := root.Inputs()
var anythingChanged *ApplyResult
- newInputs := make([]Operator, len(oldInputs))
- childID := rootID
-
- // noLHSTableSet is used to mark which operators that do not send data from the LHS to the RHS
- // It's only UNION at this moment, but this package can't depend on the actual operators, so
- // we use this interface to avoid direct dependencies
- type noLHSTableSet interface{ NoLHSTableSet() }
-
- for i, operator := range oldInputs {
- // We merge the table set of all the LHS above the current root so that we can
- // send it down to the current RHS.
- // We don't want to send the LHS table set to the RHS if the root is a UNION.
- // Some operators, like SubQuery, can have multiple child operators on the RHS
- if _, isUnion := root.(noLHSTableSet); !isUnion && i > 0 {
- childID = childID.Merge(resolveID(oldInputs[0]))
+
+ switch root := root.(type) {
+ case nullaryOperator:
+ // no inputs, nothing to do
+ case *unaryOperator:
+ newSource, changed := bottomUp(root.Source, rootID, resolveID, rewriter, shouldVisit, false)
+ if DebugOperatorTree && changed.Changed() {
+ fmt.Println(ToTree(newSource))
}
- in, changed := bottomUp(operator, childID, resolveID, rewriter, shouldVisit, false)
+ anythingChanged = anythingChanged.Merge(changed)
+ root.Source = newSource
+ case *binaryOperator:
+ newLHS, changed := bottomUp(root.LHS, rootID, resolveID, rewriter, shouldVisit, false)
if DebugOperatorTree && changed.Changed() {
- fmt.Println(ToTree(in))
+ fmt.Println(ToTree(newLHS))
}
anythingChanged = anythingChanged.Merge(changed)
- newInputs[i] = in
- }
+ root.LHS = newLHS
+ newRHS, changed := bottomUp(root.RHS, rootID, resolveID, rewriter, shouldVisit, false)
+ if DebugOperatorTree && changed.Changed() {
+ fmt.Println(ToTree(newRHS))
+ }
+ anythingChanged = anythingChanged.Merge(changed)
+ root.RHS = newRHS
+ default:
+ oldInputs := root.Inputs()
+ newInputs := make([]Operator, len(oldInputs))
+ childID := rootID
+
+ // noLHSTableSet is used to mark which operators that do not send data from the LHS to the RHS
+ // It's only UNION at this moment, but this package can't depend on the actual operators, so
+ // we use this interface to avoid direct dependencies
+ type noLHSTableSet interface{ NoLHSTableSet() }
+
+ for i, operator := range oldInputs {
+ // We merge the table set of all the LHS above the current root so that we can
+ // send it down to the current RHS.
+ // We don't want to send the LHS table set to the RHS if the root is a UNION.
+ // Some operators, like SubQuery, can have multiple child operators on the RHS
+ if _, isUnion := root.(noLHSTableSet); !isUnion && i > 0 {
+ childID = childID.Merge(resolveID(oldInputs[0]))
+ }
+ in, changed := bottomUp(operator, childID, resolveID, rewriter, shouldVisit, false)
+ if DebugOperatorTree && changed.Changed() {
+ fmt.Println(ToTree(in))
+ }
+ anythingChanged = anythingChanged.Merge(changed)
+ newInputs[i] = in
+ }
- if anythingChanged.Changed() {
- root = root.Clone(newInputs)
+ if anythingChanged.Changed() {
+ root.SetInputs(newInputs)
+ }
}
newOp, treeIdentity := rewriter(root, rootID, isRoot)
@@ -247,21 +283,42 @@ func breakableTopDown(
var anythingChanged *ApplyResult
- oldInputs := newOp.Inputs()
- newInputs := make([]Operator, len(oldInputs))
- for i, oldInput := range oldInputs {
- newInputs[i], identity, err = breakableTopDown(oldInput, rewriter)
+ switch newOp := newOp.(type) {
+ case nullaryOperator:
+ // no inputs, nothing to do
+ case *unaryOperator:
+ newSource, identity, err := breakableTopDown(newOp.Source, rewriter)
+ if err != nil {
+ return nil, NoRewrite, err
+ }
anythingChanged = anythingChanged.Merge(identity)
+ newOp.Source = newSource
+ case *binaryOperator:
+ newLHS, identity, err := breakableTopDown(newOp.LHS, rewriter)
if err != nil {
return nil, NoRewrite, err
}
+ anythingChanged = anythingChanged.Merge(identity)
+ newRHS, identity, err := breakableTopDown(newOp.RHS, rewriter)
+ if err != nil {
+ return nil, NoRewrite, err
+ }
+ anythingChanged = anythingChanged.Merge(identity)
+ newOp.LHS = newLHS
+ newOp.RHS = newRHS
+ default:
+ oldInputs := newOp.Inputs()
+ newInputs := make([]Operator, len(oldInputs))
+ for i, oldInput := range oldInputs {
+ newInputs[i], identity, err = breakableTopDown(oldInput, rewriter)
+ if err != nil {
+ return nil, NoRewrite, err
+ }
+ anythingChanged = anythingChanged.Merge(identity)
+ }
}
- if anythingChanged.Changed() {
- return newOp, NoRewrite, nil
- }
-
- return newOp.Clone(newInputs), anythingChanged, nil
+ return newOp, anythingChanged, nil
}
// topDown is a helper function that recursively traverses the operator tree from the
@@ -285,24 +342,39 @@ func topDown(
root = newOp
}
- oldInputs := root.Inputs()
- newInputs := make([]Operator, len(oldInputs))
- childID := rootID
-
- type noLHSTableSet interface{ NoLHSTableSet() }
-
- for i, operator := range oldInputs {
- if _, isUnion := root.(noLHSTableSet); !isUnion && i > 0 {
- childID = childID.Merge(resolveID(oldInputs[0]))
- }
- in, changed := topDown(operator, childID, resolveID, rewriter, shouldVisit, false)
+ switch newOp := newOp.(type) {
+ case nullaryOperator:
+ // no inputs, nothing to do
+ case *unaryOperator:
+ newSource, changed := topDown(newOp.Source, rootID, resolveID, rewriter, shouldVisit, false)
anythingChanged = anythingChanged.Merge(changed)
- newInputs[i] = in
- }
+ newOp.Source = newSource
+ case *binaryOperator:
+ newLHS, changed := topDown(newOp.LHS, rootID, resolveID, rewriter, shouldVisit, false)
+ anythingChanged = anythingChanged.Merge(changed)
+ newRHS, changed := topDown(newOp.RHS, rootID, resolveID, rewriter, shouldVisit, false)
+ anythingChanged = anythingChanged.Merge(changed)
+ newOp.LHS, newOp.RHS = newLHS, newRHS
+ default:
+ oldInputs := root.Inputs()
+ newInputs := make([]Operator, len(oldInputs))
+ childID := rootID
+
+ type noLHSTableSet interface{ NoLHSTableSet() }
+
+ for i, operator := range oldInputs {
+ if _, isUnion := root.(noLHSTableSet); !isUnion && i > 0 {
+ childID = childID.Merge(resolveID(oldInputs[0]))
+ }
+ in, changed := topDown(operator, childID, resolveID, rewriter, shouldVisit, false)
+ anythingChanged = anythingChanged.Merge(changed)
+ newInputs[i] = in
+ }
- if anythingChanged != NoRewrite {
- return root.Clone(newInputs), anythingChanged
+ if anythingChanged != NoRewrite {
+ root.SetInputs(newInputs)
+ }
}
- return root, NoRewrite
+ return root, anythingChanged
}
diff --git a/go/vt/vtgate/planbuilder/operators/route.go b/go/vt/vtgate/planbuilder/operators/route.go
index 1c91077c2e4..e398fb05607 100644
--- a/go/vt/vtgate/planbuilder/operators/route.go
+++ b/go/vt/vtgate/planbuilder/operators/route.go
@@ -19,12 +19,10 @@ package operators
import (
"fmt"
- "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/slice"
"vitess.io/vitess/go/vt/key"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/sqlparser"
- "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/evalengine"
@@ -35,7 +33,7 @@ import (
type (
Route struct {
- Source Operator
+ unaryOperator
// Routes that have been merged into this one.
MergedWith []*Route
@@ -121,7 +119,7 @@ func UpdateRoutingLogic(ctx *plancontext.PlanningContext, expr sqlparser.Expr, r
}
nr := &NoneRouting{keyspace: ks}
- if isConstantFalse(ctx.VSchema.Environment(), expr, ctx.VSchema.ConnCollation()) {
+ if b := ctx.IsConstantBool(expr); b != nil && !*b {
return nr
}
@@ -163,31 +161,6 @@ func UpdateRoutingLogic(ctx *plancontext.PlanningContext, expr sqlparser.Expr, r
return exit()
}
-// isConstantFalse checks whether this predicate can be evaluated at plan-time. If it returns `false` or `null`,
-// we know that the query will not return anything, and this can be used to produce better plans
-func isConstantFalse(env *vtenv.Environment, expr sqlparser.Expr, collation collations.ID) bool {
- eenv := evalengine.EmptyExpressionEnv(env)
- eexpr, err := evalengine.Translate(expr, &evalengine.Config{
- Collation: collation,
- Environment: env,
- })
- if err != nil {
- return false
- }
- eres, err := eenv.Evaluate(eexpr)
- if err != nil {
- return false
- }
- if eres.Value(collation).IsNull() {
- return false
- }
- b, err := eres.ToBooleanStrict()
- if err != nil {
- return false
- }
- return !b
-}
-
// Cost implements the Operator interface
func (r *Route) Cost() int {
return r.Routing.Cost()
@@ -201,16 +174,6 @@ func (r *Route) Clone(inputs []Operator) Operator {
return &cloneRoute
}
-// Inputs implements the Operator interface
-func (r *Route) Inputs() []Operator {
- return []Operator{r.Source}
-}
-
-// SetInputs implements the Operator interface
-func (r *Route) SetInputs(ops []Operator) {
- r.Source = ops[0]
-}
-
func createOption(
colVindex *vindexes.ColumnVindex,
vfunc func(*vindexes.ColumnVindex) vindexes.Vindex,
@@ -459,10 +422,10 @@ func createRouteFromVSchemaTable(
}
}
plan := &Route{
- Source: &Table{
+ unaryOperator: newUnaryOp(&Table{
QTable: queryTable,
VTable: vschemaTable,
- },
+ }),
}
// We create the appropriate Routing struct here, depending on the type of table we are dealing with.
@@ -716,8 +679,8 @@ func wrapInDerivedProjection(
columns = append(columns, sqlparser.NewIdentifierCI(fmt.Sprintf("c%d", i)))
}
derivedProj := &Projection{
- Source: op,
- Columns: AliasedProjections(slice.Map(unionColumns, newProjExpr)),
+ unaryOperator: newUnaryOp(op),
+ Columns: AliasedProjections(slice.Map(unionColumns, newProjExpr)),
DT: &DerivedTable{
TableID: ctx.SemTable.NewTableId(),
Alias: "dt",
@@ -754,14 +717,11 @@ func (r *Route) GetOrdering(ctx *plancontext.PlanningContext) []OrderBy {
// TablesUsed returns tables used by MergedWith routes, which are not included
// in Inputs() and thus not a part of the operator tree
-func (r *Route) TablesUsed() []string {
- addString, collect := collectSortedUniqueStrings()
+func (r *Route) TablesUsed(in []string) []string {
for _, mw := range r.MergedWith {
- for _, u := range TablesUsed(mw) {
- addString(u)
- }
+ in = append(in, TablesUsed(mw)...)
}
- return collect()
+ return in
}
func isSpecialOrderBy(o OrderBy) bool {
diff --git a/go/vt/vtgate/planbuilder/operators/route_planning.go b/go/vt/vtgate/planbuilder/operators/route_planning.go
index 22db69f287b..90eb16e1562 100644
--- a/go/vt/vtgate/planbuilder/operators/route_planning.go
+++ b/go/vt/vtgate/planbuilder/operators/route_planning.go
@@ -189,8 +189,8 @@ func createInfSchemaRoute(ctx *plancontext.PlanningContext, table *QueryTable) O
routing = UpdateRoutingLogic(ctx, pred, routing)
}
return &Route{
- Source: src,
- Routing: routing,
+ unaryOperator: newUnaryOp(src),
+ Routing: routing,
}
}
@@ -288,7 +288,8 @@ func requiresSwitchingSides(ctx *plancontext.PlanningContext, op Operator) (requ
}
func mergeOrJoin(ctx *plancontext.PlanningContext, lhs, rhs Operator, joinPredicates []sqlparser.Expr, joinType sqlparser.JoinType) (Operator, *ApplyResult) {
- newPlan := mergeJoinInputs(ctx, lhs, rhs, joinPredicates, newJoinMerge(joinPredicates, joinType))
+ jm := newJoinMerge(joinPredicates, joinType)
+ newPlan := jm.mergeJoinInputs(ctx, lhs, rhs, joinPredicates)
if newPlan != nil {
return newPlan, Rewrote("merge routes into single operator")
}
diff --git a/go/vt/vtgate/planbuilder/operators/subquery.go b/go/vt/vtgate/planbuilder/operators/subquery.go
index b919bbfaed9..9610a2b10c9 100644
--- a/go/vt/vtgate/planbuilder/operators/subquery.go
+++ b/go/vt/vtgate/planbuilder/operators/subquery.go
@@ -232,11 +232,7 @@ var subqueryNotAtTopErr = vterrors.VT12001("unmergable subquery can not be insid
func (sq *SubQuery) addLimit() {
// for a correlated subquery, we can add a limit 1 to the subquery
- sq.Subquery = &Limit{
- Source: sq.Subquery,
- AST: &sqlparser.Limit{Rowcount: sqlparser.NewIntLiteral("1")},
- Top: true,
- }
+ sq.Subquery = newLimit(sq.Subquery, &sqlparser.Limit{Rowcount: sqlparser.NewIntLiteral("1")}, true)
}
func (sq *SubQuery) settleFilter(ctx *plancontext.PlanningContext, outer Operator) Operator {
diff --git a/go/vt/vtgate/planbuilder/operators/subquery_planning.go b/go/vt/vtgate/planbuilder/operators/subquery_planning.go
index 0893afbeead..a2aca74fb6e 100644
--- a/go/vt/vtgate/planbuilder/operators/subquery_planning.go
+++ b/go/vt/vtgate/planbuilder/operators/subquery_planning.go
@@ -633,7 +633,7 @@ func (s *subqueryRouteMerger) merge(ctx *plancontext.PlanningContext, inner, out
if !s.subq.TopLevel {
// if the subquery we are merging isn't a top level predicate, we can't use it for routing
return &Route{
- Source: outer.Source,
+ unaryOperator: newUnaryOp(outer.Source),
MergedWith: mergedWith(inner, outer),
Routing: outer.Routing,
Ordering: outer.Ordering,
@@ -651,7 +651,7 @@ func (s *subqueryRouteMerger) merge(ctx *plancontext.PlanningContext, inner, out
src = s.rewriteASTExpression(ctx, inner)
}
return &Route{
- Source: src,
+ unaryOperator: newUnaryOp(src),
MergedWith: mergedWith(inner, outer),
Routing: r,
Ordering: s.outer.Ordering,
diff --git a/go/vt/vtgate/planbuilder/operators/table.go b/go/vt/vtgate/planbuilder/operators/table.go
index 3ecd4982ece..4391380480c 100644
--- a/go/vt/vtgate/planbuilder/operators/table.go
+++ b/go/vt/vtgate/planbuilder/operators/table.go
@@ -33,7 +33,7 @@ type (
VTable *vindexes.Table
Columns []*sqlparser.ColName
- noInputs
+ nullaryOperator
}
ColNameColumns interface {
GetColNames() []*sqlparser.ColName
@@ -107,11 +107,11 @@ func (to *Table) AddCol(col *sqlparser.ColName) {
to.Columns = append(to.Columns, col)
}
-func (to *Table) TablesUsed() []string {
+func (to *Table) TablesUsed(in []string) []string {
if sqlparser.SystemSchema(to.QTable.Table.Qualifier.String()) {
- return nil
+ return in
}
- return SingleQualifiedIdentifier(to.VTable.Keyspace, to.VTable.Name)
+ return append(in, QualifiedString(to.VTable.Keyspace, to.VTable.Name.String()))
}
func addColumn(ctx *plancontext.PlanningContext, op ColNameColumns, e sqlparser.Expr) int {
diff --git a/go/vt/vtgate/planbuilder/operators/union_merging.go b/go/vt/vtgate/planbuilder/operators/union_merging.go
index 20c20673665..000d176b61a 100644
--- a/go/vt/vtgate/planbuilder/operators/union_merging.go
+++ b/go/vt/vtgate/planbuilder/operators/union_merging.go
@@ -222,9 +222,9 @@ func createMergedUnion(
union := newUnion([]Operator{lhsRoute.Source, rhsRoute.Source}, []sqlparser.SelectExprs{lhsExprs, rhsExprs}, cols, distinct)
selectExprs := unionSelects(lhsExprs)
return &Route{
- Source: union,
- MergedWith: []*Route{rhsRoute},
- Routing: routing,
+ unaryOperator: newUnaryOp(union),
+ MergedWith: []*Route{rhsRoute},
+ Routing: routing,
}, selectExprs
}
diff --git a/go/vt/vtgate/planbuilder/operators/update.go b/go/vt/vtgate/planbuilder/operators/update.go
index e843155246c..dd0a86c2de2 100644
--- a/go/vt/vtgate/planbuilder/operators/update.go
+++ b/go/vt/vtgate/planbuilder/operators/update.go
@@ -66,9 +66,6 @@ func (u *Update) Inputs() []Operator {
}
func (u *Update) SetInputs(inputs []Operator) {
- if len(inputs) != 1 {
- panic(vterrors.VT13001("unexpected number of inputs for Update operator"))
- }
u.Source = inputs[0]
}
@@ -89,8 +86,8 @@ func (u *Update) GetOrdering(*plancontext.PlanningContext) []OrderBy {
return nil
}
-func (u *Update) TablesUsed() []string {
- return SingleQualifiedIdentifier(u.Target.VTable.Keyspace, u.Target.VTable.Name)
+func (u *Update) TablesUsed(in []string) []string {
+ return append(in, QualifiedString(u.Target.VTable.Keyspace, u.Target.VTable.Name.String()))
}
func (u *Update) ShortDescription() string {
@@ -113,11 +110,7 @@ func createOperatorFromUpdate(ctx *plancontext.PlanningContext, updStmt *sqlpars
var targetTbl TargetTable
op, targetTbl, updClone = createUpdateOperator(ctx, updStmt)
- op = &LockAndComment{
- Source: op,
- Comments: updStmt.Comments,
- Lock: sqlparser.ShareModeLock,
- }
+ op = newLockAndComment(op, updStmt.Comments, sqlparser.ShareModeLock)
parentFks = ctx.SemTable.GetParentForeignKeysForTableSet(targetTbl.ID)
childFks = ctx.SemTable.GetChildForeignKeysForTableSet(targetTbl.ID)
@@ -206,10 +199,7 @@ func createUpdateWithInputOp(ctx *plancontext.PlanningContext, upd *sqlparser.Up
}
if upd.Comments != nil {
- op = &LockAndComment{
- Source: op,
- Comments: upd.Comments,
- }
+ op = newLockAndComment(op, upd.Comments, sqlparser.NoLock)
}
return op
}
@@ -388,7 +378,6 @@ func createUpdateOperator(ctx *plancontext.PlanningContext, updStmt *sqlparser.U
Ignore: updStmt.Ignore,
Target: targetTbl,
OwnedVindexQuery: ovq,
- Source: op,
},
Assignments: assignments,
ChangedVindexValues: cvv,
@@ -397,14 +386,13 @@ func createUpdateOperator(ctx *plancontext.PlanningContext, updStmt *sqlparser.U
}
if len(updStmt.OrderBy) > 0 {
- addOrdering(ctx, updStmt.OrderBy, updOp)
+ updOp.Source = addOrdering(ctx, op, updStmt.OrderBy)
+ } else {
+ updOp.Source = op
}
if updStmt.Limit != nil {
- updOp.Source = &Limit{
- Source: updOp.Source,
- AST: updStmt.Limit,
- }
+ updOp.Source = newLimit(updOp.Source, updStmt.Limit, false)
}
return sqc.getRootOperator(updOp, nil), targetTbl, updClone
diff --git a/go/vt/vtgate/planbuilder/operators/vindex.go b/go/vt/vtgate/planbuilder/operators/vindex.go
index fd907fdad27..30f13701df6 100644
--- a/go/vt/vtgate/planbuilder/operators/vindex.go
+++ b/go/vt/vtgate/planbuilder/operators/vindex.go
@@ -35,7 +35,7 @@ type (
Columns []*sqlparser.ColName
Value sqlparser.Expr
- noInputs
+ nullaryOperator
}
// VindexTable contains information about the vindex table we want to query
@@ -164,8 +164,8 @@ func (v *Vindex) AddPredicate(ctx *plancontext.PlanningContext, expr sqlparser.E
// TablesUsed implements the Operator interface.
// It is not keyspace-qualified.
-func (v *Vindex) TablesUsed() []string {
- return []string{v.Table.Table.Name.String()}
+func (v *Vindex) TablesUsed(in []string) []string {
+ return append(in, v.Table.Table.Name.String())
}
func (v *Vindex) ShortDescription() string {
diff --git a/go/vt/vtgate/planbuilder/plan_test.go b/go/vt/vtgate/planbuilder/plan_test.go
index f49994d37b2..7135f4dff29 100644
--- a/go/vt/vtgate/planbuilder/plan_test.go
+++ b/go/vt/vtgate/planbuilder/plan_test.go
@@ -74,17 +74,16 @@ func TestPlanTestSuite(t *testing.T) {
func (s *planTestSuite) TestPlan() {
defer utils.EnsureNoLeaks(s.T())
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- TabletType_: topodatapb.TabletType_PRIMARY,
- SysVarEnabled: true,
- TestBuilder: TestBuilder,
- Env: vtenv.NewTestEnv(),
- }
- s.addPKs(vschemaWrapper.V, "user", []string{"user", "music"})
- s.addPKsProvided(vschemaWrapper.V, "user", []string{"user_extra"}, []string{"id", "user_id"})
- s.addPKsProvided(vschemaWrapper.V, "ordering", []string{"order"}, []string{"oid", "region_id"})
- s.addPKsProvided(vschemaWrapper.V, "ordering", []string{"order_event"}, []string{"oid", "ename"})
+
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ s.addPKs(vschema, "user", []string{"user", "music"})
+ s.addPKsProvided(vschema, "user", []string{"user_extra"}, []string{"id", "user_id"})
+ s.addPKsProvided(vschema, "ordering", []string{"order"}, []string{"oid", "region_id"})
+ s.addPKsProvided(vschema, "ordering", []string{"order_event"}, []string{"oid", "ename"})
// You will notice that some tests expect user.Id instead of user.id.
// This is because we now pre-create vindex columns in the symbol
@@ -92,77 +91,73 @@ func (s *planTestSuite) TestPlan() {
// the column is named as Id. This is to make sure that
// column names are case-preserved, but treated as
// case-insensitive even if they come from the vschema.
- s.testFile("aggr_cases.json", vschemaWrapper, false)
- s.testFile("dml_cases.json", vschemaWrapper, false)
- s.testFile("from_cases.json", vschemaWrapper, false)
- s.testFile("filter_cases.json", vschemaWrapper, false)
- s.testFile("postprocess_cases.json", vschemaWrapper, false)
- s.testFile("select_cases.json", vschemaWrapper, false)
- s.testFile("symtab_cases.json", vschemaWrapper, false)
- s.testFile("unsupported_cases.json", vschemaWrapper, false)
- s.testFile("unknown_schema_cases.json", vschemaWrapper, false)
- s.testFile("vindex_func_cases.json", vschemaWrapper, false)
- s.testFile("wireup_cases.json", vschemaWrapper, false)
- s.testFile("memory_sort_cases.json", vschemaWrapper, false)
- s.testFile("use_cases.json", vschemaWrapper, false)
- s.testFile("set_cases.json", vschemaWrapper, false)
- s.testFile("union_cases.json", vschemaWrapper, false)
- s.testFile("large_union_cases.json", vschemaWrapper, false)
- s.testFile("transaction_cases.json", vschemaWrapper, false)
- s.testFile("lock_cases.json", vschemaWrapper, false)
- s.testFile("large_cases.json", vschemaWrapper, false)
- s.testFile("ddl_cases_no_default_keyspace.json", vschemaWrapper, false)
- s.testFile("flush_cases_no_default_keyspace.json", vschemaWrapper, false)
- s.testFile("show_cases_no_default_keyspace.json", vschemaWrapper, false)
- s.testFile("stream_cases.json", vschemaWrapper, false)
- s.testFile("info_schema80_cases.json", vschemaWrapper, false)
- s.testFile("reference_cases.json", vschemaWrapper, false)
- s.testFile("vexplain_cases.json", vschemaWrapper, false)
- s.testFile("misc_cases.json", vschemaWrapper, false)
- s.testFile("cte_cases.json", vschemaWrapper, false)
+ s.testFile("aggr_cases.json", vw, false)
+ s.testFile("dml_cases.json", vw, false)
+ s.testFile("from_cases.json", vw, false)
+ s.testFile("filter_cases.json", vw, false)
+ s.testFile("postprocess_cases.json", vw, false)
+ s.testFile("select_cases.json", vw, false)
+ s.testFile("symtab_cases.json", vw, false)
+ s.testFile("unsupported_cases.json", vw, false)
+ s.testFile("unknown_schema_cases.json", vw, false)
+ s.testFile("vindex_func_cases.json", vw, false)
+ s.testFile("wireup_cases.json", vw, false)
+ s.testFile("memory_sort_cases.json", vw, false)
+ s.testFile("use_cases.json", vw, false)
+ s.testFile("set_cases.json", vw, false)
+ s.testFile("union_cases.json", vw, false)
+ s.testFile("large_union_cases.json", vw, false)
+ s.testFile("transaction_cases.json", vw, false)
+ s.testFile("lock_cases.json", vw, false)
+ s.testFile("large_cases.json", vw, false)
+ s.testFile("ddl_cases_no_default_keyspace.json", vw, false)
+ s.testFile("flush_cases_no_default_keyspace.json", vw, false)
+ s.testFile("show_cases_no_default_keyspace.json", vw, false)
+ s.testFile("stream_cases.json", vw, false)
+ s.testFile("info_schema80_cases.json", vw, false)
+ s.testFile("reference_cases.json", vw, false)
+ s.testFile("vexplain_cases.json", vw, false)
+ s.testFile("misc_cases.json", vw, false)
+ s.testFile("cte_cases.json", vw, false)
}
// TestForeignKeyPlanning tests the planning of foreign keys in a managed mode by Vitess.
func (s *planTestSuite) TestForeignKeyPlanning() {
+ env := vtenv.NewTestEnv()
vschema := loadSchema(s.T(), "vschemas/schema.json", true)
- s.setFks(vschema)
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: vschema,
- TestBuilder: TestBuilder,
- Env: vtenv.NewTestEnv(),
- }
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("foreignkey_cases.json", vschemaWrapper, false)
+ s.setFks(vschema)
+ s.testFile("foreignkey_cases.json", vw, false)
}
// TestForeignKeyChecksOn tests the planning when the session variable for foreign_key_checks is set to ON.
func (s *planTestSuite) TestForeignKeyChecksOn() {
+ env := vtenv.NewTestEnv()
vschema := loadSchema(s.T(), "vschemas/schema.json", true)
- s.setFks(vschema)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
fkChecksState := true
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: vschema,
- TestBuilder: TestBuilder,
- ForeignKeyChecksState: &fkChecksState,
- Env: vtenv.NewTestEnv(),
- }
+ vw.ForeignKeyChecksState = &fkChecksState
- s.testFile("foreignkey_checks_on_cases.json", vschemaWrapper, false)
+ s.setFks(vschema)
+ s.testFile("foreignkey_checks_on_cases.json", vw, false)
}
// TestForeignKeyChecksOff tests the planning when the session variable for foreign_key_checks is set to OFF.
func (s *planTestSuite) TestForeignKeyChecksOff() {
+ env := vtenv.NewTestEnv()
vschema := loadSchema(s.T(), "vschemas/schema.json", true)
- s.setFks(vschema)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
fkChecksState := false
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: vschema,
- TestBuilder: TestBuilder,
- ForeignKeyChecksState: &fkChecksState,
- Env: vtenv.NewTestEnv(),
- }
+ vw.ForeignKeyChecksState = &fkChecksState
- s.testFile("foreignkey_checks_off_cases.json", vschemaWrapper, false)
+ s.setFks(vschema)
+ s.testFile("foreignkey_checks_off_cases.json", vw, false)
}
func (s *planTestSuite) setFks(vschema *vindexes.VSchema) {
@@ -266,120 +261,127 @@ func (s *planTestSuite) TestSystemTables57() {
MySQLServerVersion: "5.7.9",
})
require.NoError(s.T(), err)
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Env: env,
- }
- s.testFile("info_schema57_cases.json", vschemaWrapper, false)
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ s.testFile("info_schema57_cases.json", vw, false)
}
func (s *planTestSuite) TestSysVarSetDisabled() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- SysVarEnabled: false,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.SysVarEnabled = false
- s.testFile("set_sysvar_disabled_cases.json", vschemaWrapper, false)
+ s.testFile("set_sysvar_disabled_cases.json", vw, false)
}
func (s *planTestSuite) TestViews() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- EnableViews: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("view_cases.json", vschemaWrapper, false)
+ vw.EnableViews = true
+
+ s.testFile("view_cases.json", vw, false)
}
func (s *planTestSuite) TestOne() {
reset := operators.EnableDebugPrinting()
defer reset()
- lv := loadSchema(s.T(), "vschemas/schema.json", true)
- s.setFks(lv)
- s.addPKs(lv, "user", []string{"user", "music"})
- s.addPKs(lv, "main", []string{"unsharded"})
- s.addPKsProvided(lv, "user", []string{"user_extra"}, []string{"id", "user_id"})
- s.addPKsProvided(lv, "ordering", []string{"order"}, []string{"oid", "region_id"})
- s.addPKsProvided(lv, "ordering", []string{"order_event"}, []string{"oid", "ename"})
- vschema := &vschemawrapper.VSchemaWrapper{
- V: lv,
- TestBuilder: TestBuilder,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("onecase.json", vschema, false)
+ s.setFks(vschema)
+ s.addPKs(vschema, "user", []string{"user", "music"})
+ s.addPKs(vschema, "main", []string{"unsharded"})
+ s.addPKsProvided(vschema, "user", []string{"user_extra"}, []string{"id", "user_id"})
+ s.addPKsProvided(vschema, "ordering", []string{"order"}, []string{"oid", "region_id"})
+ s.addPKsProvided(vschema, "ordering", []string{"order_event"}, []string{"oid", "ename"})
+
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneTPCC() {
reset := operators.EnableDebugPrinting()
defer reset()
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/tpcc_schema.json", true),
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/tpcc_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("onecase.json", vschema, false)
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneWithMainAsDefault() {
reset := operators.EnableDebugPrinting()
defer reset()
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "main",
- Sharded: false,
- },
- Env: vtenv.NewTestEnv(),
- }
- s.testFile("onecase.json", vschema, false)
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("main")
+ vw.Keyspace = &vindexes.Keyspace{Name: "main"}
+
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneWithSecondUserAsDefault() {
reset := operators.EnableDebugPrinting()
defer reset()
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "second_user",
- Sharded: true,
- },
- Env: vtenv.NewTestEnv(),
+
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("second_user")
+ vw.Keyspace = &vindexes.Keyspace{
+ Name: "second_user",
+ Sharded: true,
}
- s.testFile("onecase.json", vschema, false)
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneWithUserAsDefault() {
reset := operators.EnableDebugPrinting()
defer reset()
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "user",
- Sharded: true,
- },
- Env: vtenv.NewTestEnv(),
+
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("user")
+ vw.Keyspace = &vindexes.Keyspace{
+ Name: "user",
+ Sharded: true,
}
- s.testFile("onecase.json", vschema, false)
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneWithTPCHVSchema() {
reset := operators.EnableDebugPrinting()
defer reset()
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/tpch_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
- s.testFile("onecase.json", vschema, false)
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestOneWith57Version() {
@@ -390,52 +392,47 @@ func (s *planTestSuite) TestOneWith57Version() {
MySQLServerVersion: "5.7.9",
})
require.NoError(s.T(), err)
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Env: env,
- }
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("onecase.json", vschema, false)
+ s.testFile("onecase.json", vw, false)
}
func (s *planTestSuite) TestRubyOnRailsQueries() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/rails_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/rails_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("rails_cases.json", vschemaWrapper, false)
+ s.testFile("rails_cases.json", vw, false)
}
func (s *planTestSuite) TestOLTP() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/oltp_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/oltp_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("oltp_cases.json", vschemaWrapper, false)
+ s.testFile("oltp_cases.json", vw, false)
}
func (s *planTestSuite) TestTPCC() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/tpcc_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/tpcc_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("tpcc_cases.json", vschemaWrapper, false)
+ s.testFile("tpcc_cases.json", vw, false)
}
func (s *planTestSuite) TestTPCH() {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/tpch_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/tpch_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("tpch_cases.json", vschemaWrapper, false)
+ s.testFile("tpch_cases.json", vw, false)
}
func BenchmarkOLTP(b *testing.B) {
@@ -451,19 +448,14 @@ func BenchmarkTPCH(b *testing.B) {
}
func benchmarkWorkload(b *testing.B, name string) {
- vschemaWrapper := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(b, "vschemas/"+name+"_schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(b, "vschemas/"+name+"_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(b, err)
testCases := readJSONTests(name + "_cases.json")
b.ResetTimer()
- for _, version := range plannerVersions {
- b.Run(version.String(), func(b *testing.B) {
- benchmarkPlanner(b, version, testCases, vschemaWrapper)
- })
- }
+ benchmarkPlanner(b, Gen4, testCases, vw)
}
func (s *planTestSuite) TestBypassPlanningShardTargetFromFile() {
@@ -482,35 +474,33 @@ func (s *planTestSuite) TestBypassPlanningShardTargetFromFile() {
}
func (s *planTestSuite) TestBypassPlanningKeyrangeTargetFromFile() {
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
keyRange, _ := key.ParseShardingSpec("-")
+ vw.Dest = key.DestinationExactKeyRange{KeyRange: keyRange[0]}
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "main",
- Sharded: false,
- },
- TabletType_: topodatapb.TabletType_PRIMARY,
- Dest: key.DestinationExactKeyRange{KeyRange: keyRange[0]},
- Env: vtenv.NewTestEnv(),
- }
+ vw.Vcursor.SetTarget("main")
+ vw.Keyspace = &vindexes.Keyspace{Name: "main"}
- s.testFile("bypass_keyrange_cases.json", vschema, false)
+ s.testFile("bypass_keyrange_cases.json", vw, false)
}
func (s *planTestSuite) TestWithDefaultKeyspaceFromFile() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
+
// We are testing this separately so we can set a default keyspace
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "main",
- Sharded: false,
- },
- TabletType_: topodatapb.TabletType_PRIMARY,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("main")
+ vw.Keyspace = &vindexes.Keyspace{Name: "main"}
+
ts := memorytopo.NewServer(ctx, "cell1")
ts.CreateKeyspace(ctx, "main", &topodatapb.Keyspace{})
ts.CreateKeyspace(ctx, "user", &topodatapb.Keyspace{})
@@ -525,70 +515,92 @@ func (s *planTestSuite) TestWithDefaultKeyspaceFromFile() {
})
require.True(s.T(), created)
- s.testFile("alterVschema_cases.json", vschema, false)
- s.testFile("ddl_cases.json", vschema, false)
- s.testFile("migration_cases.json", vschema, false)
- s.testFile("flush_cases.json", vschema, false)
- s.testFile("show_cases.json", vschema, false)
- s.testFile("call_cases.json", vschema, false)
+ s.testFile("alterVschema_cases.json", vw, false)
+ s.testFile("ddl_cases.json", vw, false)
+ s.testFile("migration_cases.json", vw, false)
+ s.testFile("flush_cases.json", vw, false)
+ s.testFile("show_cases.json", vw, false)
+ s.testFile("call_cases.json", vw, false)
}
func (s *planTestSuite) TestWithDefaultKeyspaceFromFileSharded() {
// We are testing this separately so we can set a default keyspace
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "second_user",
- Sharded: true,
- },
- TabletType_: topodatapb.TabletType_PRIMARY,
- Env: vtenv.NewTestEnv(),
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("second_user")
+ vw.Keyspace = &vindexes.Keyspace{
+ Name: "second_user",
+ Sharded: true,
}
- s.testFile("select_cases_with_default.json", vschema, false)
+ s.testFile("select_cases_with_default.json", vw, false)
}
func (s *planTestSuite) TestWithUserDefaultKeyspaceFromFileSharded() {
// We are testing this separately so we can set a default keyspace
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "user",
- Sharded: true,
- },
- TabletType_: topodatapb.TabletType_PRIMARY,
- Env: vtenv.NewTestEnv(),
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("user")
+ vw.Keyspace = &vindexes.Keyspace{
+ Name: "user",
+ Sharded: true,
}
- s.testFile("select_cases_with_user_as_default.json", vschema, false)
+ s.testFile("select_cases_with_user_as_default.json", vw, false)
+ s.testFile("dml_cases_with_user_as_default.json", vw, false)
}
func (s *planTestSuite) TestWithSystemSchemaAsDefaultKeyspace() {
// We are testing this separately so we can set a default keyspace
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{Name: "information_schema"},
- TabletType_: topodatapb.TabletType_PRIMARY,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Keyspace = &vindexes.Keyspace{Name: "information_schema"}
- s.testFile("sysschema_default.json", vschema, false)
+ s.testFile("sysschema_default.json", vw, false)
}
func (s *planTestSuite) TestOtherPlanningFromFile() {
// We are testing this separately so we can set a default keyspace
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(s.T(), "vschemas/schema.json", true),
- Keyspace: &vindexes.Keyspace{
- Name: "main",
- Sharded: false,
- },
- TabletType_: topodatapb.TabletType_PRIMARY,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ vw.Vcursor.SetTarget("main")
+ vw.Keyspace = &vindexes.Keyspace{Name: "main"}
+
+ s.testFile("other_read_cases.json", vw, false)
+ s.testFile("other_admin_cases.json", vw, false)
+}
+
+func (s *planTestSuite) TestMirrorPlanning() {
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/mirror_schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
+
+ s.testFile("mirror_cases.json", vw, false)
+}
+
+func (s *planTestSuite) TestOneMirror() {
+ reset := operators.EnableDebugPrinting()
+ defer reset()
+
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(s.T(), "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(s.T(), err)
- s.testFile("other_read_cases.json", vschema, false)
- s.testFile("other_admin_cases.json", vschema, false)
+ s.testFile("onecase.json", vw, false)
}
func loadSchema(t testing.TB, filename string, setCollation bool) *vindexes.VSchema {
@@ -637,21 +649,12 @@ func createFkDefinition(childCols []string, parentTableName string, parentCols [
}
}
-type (
- planTest struct {
- Comment string `json:"comment,omitempty"`
- Query string `json:"query,omitempty"`
- Plan json.RawMessage `json:"plan,omitempty"`
- Skip bool `json:"skip,omitempty"`
- }
-)
-
func (s *planTestSuite) testFile(filename string, vschema *vschemawrapper.VSchemaWrapper, render bool) {
opts := jsondiff.DefaultConsoleOptions()
s.T().Run(filename, func(t *testing.T) {
failed := false
- var expected []planTest
+ var expected []PlanTest
for _, tcase := range readJSONTests(filename) {
testName := tcase.Comment
if testName == "" {
@@ -660,9 +663,10 @@ func (s *planTestSuite) testFile(filename string, vschema *vschemawrapper.VSchem
if tcase.Query == "" {
continue
}
- current := planTest{
- Comment: testName,
+ current := PlanTest{
+ Comment: tcase.Comment,
Query: tcase.Query,
+ SkipE2E: tcase.SkipE2E,
}
vschema.Version = Gen4
out := getPlanOutput(tcase, vschema, render)
@@ -683,7 +687,7 @@ func (s *planTestSuite) testFile(filename string, vschema *vschemawrapper.VSchem
if tcase.Skip {
t.Skip(message)
} else {
- t.Errorf(message)
+ t.Error(message)
}
} else if tcase.Skip {
t.Errorf("query is correct even though it is skipped:\n %s", tcase.Query)
@@ -708,8 +712,8 @@ func (s *planTestSuite) testFile(filename string, vschema *vschemawrapper.VSchem
})
}
-func readJSONTests(filename string) []planTest {
- var output []planTest
+func readJSONTests(filename string) []PlanTest {
+ var output []PlanTest
file, err := os.Open(locateFile(filename))
if err != nil {
panic(err)
@@ -723,7 +727,7 @@ func readJSONTests(filename string) []planTest {
return output
}
-func getPlanOutput(tcase planTest, vschema *vschemawrapper.VSchemaWrapper, render bool) (out string) {
+func getPlanOutput(tcase PlanTest, vschema *vschemawrapper.VSchemaWrapper, render bool) (out string) {
defer func() {
if r := recover(); r != nil {
out = fmt.Sprintf("panicked: %v\n%s", r, string(debug.Stack()))
@@ -761,33 +765,29 @@ func locateFile(name string) string {
var benchMarkFiles = []string{"from_cases.json", "filter_cases.json", "large_cases.json", "aggr_cases.json", "select_cases.json", "union_cases.json"}
func BenchmarkPlanner(b *testing.B) {
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(b, "vschemas/schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(b, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(b, err)
+
for _, filename := range benchMarkFiles {
testCases := readJSONTests(filename)
b.Run(filename+"-gen4", func(b *testing.B) {
- benchmarkPlanner(b, Gen4, testCases, vschema)
- })
- b.Run(filename+"-gen4left2right", func(b *testing.B) {
- benchmarkPlanner(b, Gen4Left2Right, testCases, vschema)
+ benchmarkPlanner(b, Gen4, testCases, vw)
})
}
}
func BenchmarkSemAnalysis(b *testing.B) {
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(b, "vschemas/schema.json", true),
- SysVarEnabled: true,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(b, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(b, err)
for i := 0; i < b.N; i++ {
for _, filename := range benchMarkFiles {
for _, tc := range readJSONTests(filename) {
- exerciseAnalyzer(tc.Query, vschema.CurrentDb(), vschema)
+ exerciseAnalyzer(tc.Query, vw.CurrentDb(), vw)
}
}
}
@@ -812,12 +812,10 @@ func exerciseAnalyzer(query, database string, s semantics.SchemaInformation) {
}
func BenchmarkSelectVsDML(b *testing.B) {
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(b, "vschemas/schema.json", true),
- SysVarEnabled: true,
- Version: Gen4,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(b, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(b, err)
dmlCases := readJSONTests("dml_cases.json")
selectCases := readJSONTests("select_cases.json")
@@ -831,15 +829,37 @@ func BenchmarkSelectVsDML(b *testing.B) {
})
b.Run("DML (random sample, N=32)", func(b *testing.B) {
- benchmarkPlanner(b, Gen4, dmlCases[:32], vschema)
+ benchmarkPlanner(b, Gen4, dmlCases[:32], vw)
})
b.Run("Select (random sample, N=32)", func(b *testing.B) {
- benchmarkPlanner(b, Gen4, selectCases[:32], vschema)
+ benchmarkPlanner(b, Gen4, selectCases[:32], vw)
+ })
+}
+
+func BenchmarkBaselineVsMirrored(b *testing.B) {
+ env := vtenv.NewTestEnv()
+ baseline := loadSchema(b, "vschemas/mirror_schema.json", true)
+ baseline.MirrorRules = map[string]*vindexes.MirrorRule{}
+ bvw, err := vschemawrapper.NewVschemaWrapper(env, baseline, TestBuilder)
+ require.NoError(b, err)
+
+ mirroredSchema := loadSchema(b, "vschemas/mirror_schema.json", true)
+ mvw, err := vschemawrapper.NewVschemaWrapper(env, mirroredSchema, TestBuilder)
+ require.NoError(b, err)
+
+ cases := readJSONTests("mirror_cases.json")
+
+ b.Run("Baseline", func(b *testing.B) {
+ benchmarkPlanner(b, Gen4, cases, bvw)
+ })
+
+ b.Run("Mirrored", func(b *testing.B) {
+ benchmarkPlanner(b, Gen4, cases, mvw)
})
}
-func benchmarkPlanner(b *testing.B, version plancontext.PlannerVersion, testCases []planTest, vschema *vschemawrapper.VSchemaWrapper) {
+func benchmarkPlanner(b *testing.B, version plancontext.PlannerVersion, testCases []PlanTest, vschema *vschemawrapper.VSchemaWrapper) {
b.ReportAllocs()
for n := 0; n < b.N; n++ {
for _, tcase := range testCases {
diff --git a/go/vt/vtgate/planbuilder/plancontext/planning_context.go b/go/vt/vtgate/planbuilder/plancontext/planning_context.go
index 58be17febab..016f5c877cf 100644
--- a/go/vt/vtgate/planbuilder/plancontext/planning_context.go
+++ b/go/vt/vtgate/planbuilder/plancontext/planning_context.go
@@ -66,6 +66,19 @@ type PlanningContext struct {
// OuterTables contains the tables that are outer to the current query
// Used to set the nullable flag on the columns
OuterTables semantics.TableSet
+
+ // This is a stack of CTEs being built. It's used when we have CTEs inside CTEs,
+ // to remember which is the CTE currently being assembled
+ CurrentCTE []*ContextCTE
+
+ // mirror contains a mirrored clone of this planning context.
+ mirror *PlanningContext
+
+ // isMirrored indicates that mirrored tables should be used.
+ isMirrored bool
+
+ emptyEnv *evalengine.ExpressionEnv
+ constantCfg *evalengine.Config
}
// CreatePlanningContext initializes a new PlanningContext with the given parameters.
@@ -78,7 +91,7 @@ func CreatePlanningContext(stmt sqlparser.Statement,
version querypb.ExecuteOptions_PlannerVersion,
) (*PlanningContext, error) {
ksName := ""
- if ks, _ := vschema.DefaultKeyspace(); ks != nil {
+ if ks, _ := vschema.SelectedKeyspace(); ks != nil {
ksName = ks.Name
}
@@ -376,3 +389,112 @@ func (ctx *PlanningContext) ContainsAggr(e sqlparser.SQLNode) (hasAggr bool) {
}, e)
return
}
+
+func (ctx *PlanningContext) IsMirrored() bool {
+ return ctx.isMirrored
+}
+
+type ContextCTE struct {
+ *semantics.CTE
+ Id semantics.TableSet
+ Predicates []*RecurseExpression
+}
+
+type RecurseExpression struct {
+ Original sqlparser.Expr
+ RightExpr sqlparser.Expr
+ LeftExprs []BindVarExpr
+}
+
+type BindVarExpr struct {
+ Name string
+ Expr *sqlparser.ColName
+}
+
+func (ctx *PlanningContext) PushCTE(def *semantics.CTE, id semantics.TableSet) {
+ ctx.CurrentCTE = append(ctx.CurrentCTE, &ContextCTE{
+ CTE: def,
+ Id: id,
+ })
+}
+
+func (ctx *PlanningContext) PopCTE() (*ContextCTE, error) {
+ if len(ctx.CurrentCTE) == 0 {
+ return nil, vterrors.VT13001("no CTE to pop")
+ }
+ activeCTE := ctx.CurrentCTE[len(ctx.CurrentCTE)-1]
+ ctx.CurrentCTE = ctx.CurrentCTE[:len(ctx.CurrentCTE)-1]
+ return activeCTE, nil
+}
+
+func (ctx *PlanningContext) ActiveCTE() *ContextCTE {
+ if len(ctx.CurrentCTE) == 0 {
+ return nil
+ }
+ return ctx.CurrentCTE[len(ctx.CurrentCTE)-1]
+}
+
+func (ctx *PlanningContext) UseMirror() *PlanningContext {
+ if ctx.isMirrored {
+ panic(vterrors.VT13001("cannot mirror already mirrored planning context"))
+ }
+ if ctx.mirror != nil {
+ return ctx.mirror
+ }
+ ctx.mirror = &PlanningContext{
+ ReservedVars: ctx.ReservedVars,
+ SemTable: ctx.SemTable,
+ VSchema: ctx.VSchema,
+ joinPredicates: map[sqlparser.Expr][]sqlparser.Expr{},
+ skipPredicates: map[sqlparser.Expr]any{},
+ PlannerVersion: ctx.PlannerVersion,
+ ReservedArguments: map[sqlparser.Expr]string{},
+ VerifyAllFKs: ctx.VerifyAllFKs,
+ MergedSubqueries: ctx.MergedSubqueries,
+ CurrentPhase: ctx.CurrentPhase,
+ Statement: ctx.Statement,
+ OuterTables: ctx.OuterTables,
+ CurrentCTE: ctx.CurrentCTE,
+ emptyEnv: ctx.emptyEnv,
+ isMirrored: true,
+ }
+ return ctx.mirror
+}
+
+// IsConstantBool checks whether this predicate can be evaluated at plan-time.
+// If it can, it returns the constant value.
+func (ctx *PlanningContext) IsConstantBool(expr sqlparser.Expr) *bool {
+ if !ctx.SemTable.RecursiveDeps(expr).IsEmpty() {
+ // we have column dependencies, so we can be pretty sure
+ // we won't be able to use the evalengine to check if this is constant false
+ return nil
+ }
+ env := ctx.VSchema.Environment()
+ collation := ctx.VSchema.ConnCollation()
+ if ctx.constantCfg == nil {
+ ctx.constantCfg = &evalengine.Config{
+ Collation: collation,
+ Environment: env,
+ NoCompilation: true,
+ }
+ }
+ eexpr, err := evalengine.Translate(expr, ctx.constantCfg)
+ if ctx.emptyEnv == nil {
+ ctx.emptyEnv = evalengine.EmptyExpressionEnv(env)
+ }
+ if err != nil {
+ return nil
+ }
+ eres, err := ctx.emptyEnv.Evaluate(eexpr)
+ if err != nil {
+ return nil
+ }
+ if eres.Value(collation).IsNull() {
+ return nil
+ }
+ b, err := eres.ToBooleanStrict()
+ if err != nil {
+ return nil
+ }
+ return &b
+}
diff --git a/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go b/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go
index 3ab58cba724..e5e96b0a4be 100644
--- a/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go
+++ b/go/vt/vtgate/planbuilder/plancontext/planning_context_test.go
@@ -201,7 +201,7 @@ func (v *vschema) FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.Ta
panic("implement me")
}
-func (v *vschema) DefaultKeyspace() (*vindexes.Keyspace, error) {
+func (v *vschema) SelectedKeyspace() (*vindexes.Keyspace, error) {
// TODO implement me
panic("implement me")
}
@@ -365,4 +365,9 @@ func (v *vschema) GetAggregateUDFs() []string {
panic("implement me")
}
+// FindMirrorRule implements VSchema.
+func (v *vschema) FindMirrorRule(tablename sqlparser.TableName) (*vindexes.MirrorRule, error) {
+ panic("unimplemented")
+}
+
var _ VSchema = (*vschema)(nil)
diff --git a/go/vt/vtgate/planbuilder/plancontext/vschema.go b/go/vt/vtgate/planbuilder/plancontext/vschema.go
index 8ac4c57bfd7..b4560424718 100644
--- a/go/vt/vtgate/planbuilder/plancontext/vschema.go
+++ b/go/vt/vtgate/planbuilder/plancontext/vschema.go
@@ -27,7 +27,9 @@ type VSchema interface {
FindTable(tablename sqlparser.TableName) (*vindexes.Table, string, topodatapb.TabletType, key.Destination, error)
FindView(name sqlparser.TableName) sqlparser.SelectStatement
FindTableOrVindex(tablename sqlparser.TableName) (*vindexes.Table, vindexes.Vindex, string, topodatapb.TabletType, key.Destination, error)
- DefaultKeyspace() (*vindexes.Keyspace, error)
+
+ // SelectedKeyspace returns the current keyspace if set, otherwise returns an error
+ SelectedKeyspace() (*vindexes.Keyspace, error)
TargetString() string
Destination() key.Destination
TabletType() topodatapb.TabletType
@@ -96,6 +98,10 @@ type VSchema interface {
// GetAggregateUDFs returns the list of aggregate UDFs.
GetAggregateUDFs() []string
+
+ // FindMirrorRule finds the mirror rule for the requested keyspace, table
+ // name, and the tablet type in the VSchema.
+ FindMirrorRule(tablename sqlparser.TableName) (*vindexes.MirrorRule, error)
}
// PlannerNameToVersion returns the numerical representation of the planner
diff --git a/go/vt/vtgate/planbuilder/select.go b/go/vt/vtgate/planbuilder/select.go
index 6927c5315ac..409343f2760 100644
--- a/go/vt/vtgate/planbuilder/select.go
+++ b/go/vt/vtgate/planbuilder/select.go
@@ -46,7 +46,7 @@ func gen4SelectStmtPlanner(
}
if p != nil {
used := "dual"
- keyspace, ksErr := vschema.DefaultKeyspace()
+ keyspace, ksErr := vschema.SelectedKeyspace()
if ksErr == nil {
// we are just getting the ks to log the correct table use.
// no need to fail this if we can't find the default keyspace
@@ -101,7 +101,7 @@ func gen4SelectStmtPlanner(
func gen4planSQLCalcFoundRows(vschema plancontext.VSchema, sel *sqlparser.Select, query string, reservedVars *sqlparser.ReservedVars) (*planResult, error) {
ksName := ""
- if ks, _ := vschema.DefaultKeyspace(); ks != nil {
+ if ks, _ := vschema.SelectedKeyspace(); ks != nil {
ksName = ks.Name
}
semTable, err := semantics.Analyze(sel, ksName, vschema)
@@ -205,7 +205,7 @@ func newBuildSelectPlan(
return nil, nil, err
}
- if ks, _ := ctx.SemTable.SingleUnshardedKeyspace(); ks != nil {
+ if ks, ok := ctx.SemTable.CanTakeSelectUnshardedShortcut(); ok {
plan, tablesUsed, err = selectUnshardedShortcut(ctx, selStmt, ks)
if err != nil {
return nil, nil, err
@@ -214,7 +214,6 @@ func newBuildSelectPlan(
return plan, tablesUsed, err
}
- // From this point on, we know it is not an unsharded query and return the NotUnshardedErr if there is any
if ctx.SemTable.NotUnshardedErr != nil {
return nil, nil, ctx.SemTable.NotUnshardedErr
}
diff --git a/go/vt/vtgate/planbuilder/show.go b/go/vt/vtgate/planbuilder/show.go
index f79da3a2034..40cf7b2411f 100644
--- a/go/vt/vtgate/planbuilder/show.go
+++ b/go/vt/vtgate/planbuilder/show.go
@@ -45,14 +45,14 @@ const (
func buildShowPlan(sql string, stmt *sqlparser.Show, _ *sqlparser.ReservedVars, vschema plancontext.VSchema) (*planResult, error) {
if vschema.Destination() != nil {
- return buildByPassPlan(sql, vschema)
+ return buildByPassPlan(sql, vschema, false)
}
var prim engine.Primitive
var err error
switch show := stmt.Internal.(type) {
case *sqlparser.ShowTransactionStatus:
- prim, err = buildShowTransactionStatusPlan(show, vschema)
+ prim, err = buildShowTransactionStatusPlan(show)
case *sqlparser.ShowBasic:
prim, err = buildShowBasicPlan(show, vschema)
case *sqlparser.ShowCreate:
@@ -70,8 +70,9 @@ func buildShowPlan(sql string, stmt *sqlparser.Show, _ *sqlparser.ReservedVars,
}
// buildShowTransactionStatusPlan builds the transaction status plan
-func buildShowTransactionStatusPlan(show *sqlparser.ShowTransactionStatus, vschema plancontext.VSchema) (engine.Primitive, error) {
+func buildShowTransactionStatusPlan(show *sqlparser.ShowTransactionStatus) (engine.Primitive, error) {
return &engine.TransactionStatus{
+ Keyspace: show.Keyspace,
TransactionID: show.TransactionID,
}, nil
}
@@ -675,7 +676,7 @@ func buildVschemaKeyspacesPlan(vschema plancontext.VSchema) (engine.Primitive, e
func buildVschemaTablesPlan(vschema plancontext.VSchema) (engine.Primitive, error) {
vs := vschema.GetVSchema()
- ks, err := vschema.DefaultKeyspace()
+ ks, err := vschema.SelectedKeyspace()
if err != nil {
return nil, err
}
diff --git a/go/vt/vtgate/planbuilder/show_test.go b/go/vt/vtgate/planbuilder/show_test.go
index bfdb9a623a0..c3651aaa1cd 100644
--- a/go/vt/vtgate/planbuilder/show_test.go
+++ b/go/vt/vtgate/planbuilder/show_test.go
@@ -32,10 +32,13 @@ import (
)
func TestBuildDBPlan(t *testing.T) {
- vschema := &vschemawrapper.VSchemaWrapper{
- Keyspace: &vindexes.Keyspace{Name: "main"},
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(t, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(t, err)
+
+ vw.Vcursor.SetTarget("main")
+ vw.Keyspace = &vindexes.Keyspace{Name: "main"}
testCases := []struct {
query string
@@ -54,7 +57,7 @@ func TestBuildDBPlan(t *testing.T) {
require.NoError(t, err)
show := parserOut.(*sqlparser.Show)
- primitive, err := buildDBPlan(show.Internal.(*sqlparser.ShowBasic), vschema)
+ primitive, err := buildDBPlan(show.Internal.(*sqlparser.ShowBasic), vw)
require.NoError(t, err)
result, err := primitive.TryExecute(context.Background(), nil, nil, false)
diff --git a/go/vt/vtgate/planbuilder/simplifier_test.go b/go/vt/vtgate/planbuilder/simplifier_test.go
index 305c18896e3..dce21b3e175 100644
--- a/go/vt/vtgate/planbuilder/simplifier_test.go
+++ b/go/vt/vtgate/planbuilder/simplifier_test.go
@@ -38,21 +38,21 @@ func TestSimplifyBuggyQuery(t *testing.T) {
query := "select distinct count(distinct a), count(distinct 4) from user left join unsharded on 0 limit 5"
// select 0 from unsharded union select 0 from `user` union select 0 from unsharded
// select 0 from unsharded union (select 0 from `user` union select 0 from unsharded)
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(t, "vschemas/schema.json", true),
- Version: Gen4,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(t, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(t, err)
+
stmt, reserved, err := sqlparser.NewTestParser().Parse2(query)
require.NoError(t, err)
- rewritten, _ := sqlparser.RewriteAST(sqlparser.Clone(stmt), vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
+ rewritten, _ := sqlparser.RewriteAST(sqlparser.Clone(stmt), vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
reservedVars := sqlparser.NewReservedVars("vtg", reserved)
simplified := simplifier.SimplifyStatement(
stmt.(sqlparser.SelectStatement),
- vschema.CurrentDb(),
- vschema,
- keepSameError(query, reservedVars, vschema, rewritten.BindVarNeeds),
+ vw.CurrentDb(),
+ vw,
+ keepSameError(query, reservedVars, vw, rewritten.BindVarNeeds),
)
fmt.Println(sqlparser.String(simplified))
@@ -61,21 +61,22 @@ func TestSimplifyBuggyQuery(t *testing.T) {
func TestSimplifyPanic(t *testing.T) {
t.Skip("not needed to run")
query := "(select id from unsharded union select id from unsharded_auto) union (select id from unsharded_auto union select name from unsharded)"
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(t, "vschemas/schema.json", true),
- Version: Gen4,
- Env: vtenv.NewTestEnv(),
- }
+
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(t, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(t, err)
+
stmt, reserved, err := sqlparser.NewTestParser().Parse2(query)
require.NoError(t, err)
- rewritten, _ := sqlparser.RewriteAST(sqlparser.Clone(stmt), vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
+ rewritten, _ := sqlparser.RewriteAST(sqlparser.Clone(stmt), vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
reservedVars := sqlparser.NewReservedVars("vtg", reserved)
simplified := simplifier.SimplifyStatement(
stmt.(sqlparser.SelectStatement),
- vschema.CurrentDb(),
- vschema,
- keepPanicking(query, reservedVars, vschema, rewritten.BindVarNeeds),
+ vw.CurrentDb(),
+ vw,
+ keepPanicking(query, reservedVars, vw, rewritten.BindVarNeeds),
)
fmt.Println(sqlparser.String(simplified))
@@ -83,11 +84,11 @@ func TestSimplifyPanic(t *testing.T) {
func TestUnsupportedFile(t *testing.T) {
t.Skip("run manually to see if any queries can be simplified")
- vschema := &vschemawrapper.VSchemaWrapper{
- V: loadSchema(t, "vschemas/schema.json", true),
- Version: Gen4,
- Env: vtenv.NewTestEnv(),
- }
+ env := vtenv.NewTestEnv()
+ vschema := loadSchema(t, "vschemas/schema.json", true)
+ vw, err := vschemawrapper.NewVschemaWrapper(env, vschema, TestBuilder)
+ require.NoError(t, err)
+
fmt.Println(vschema)
for _, tcase := range readJSONTests("unsupported_cases.txt") {
t.Run(tcase.Query, func(t *testing.T) {
@@ -99,11 +100,10 @@ func TestUnsupportedFile(t *testing.T) {
t.Skip()
return
}
- rewritten, err := sqlparser.RewriteAST(stmt, vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
+ rewritten, err := sqlparser.RewriteAST(stmt, vw.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
if err != nil {
t.Skip()
}
- vschema.CurrentDb()
reservedVars := sqlparser.NewReservedVars("vtg", reserved)
ast := rewritten.AST
@@ -111,9 +111,9 @@ func TestUnsupportedFile(t *testing.T) {
stmt, _, _ = sqlparser.NewTestParser().Parse2(tcase.Query)
simplified := simplifier.SimplifyStatement(
stmt.(sqlparser.SelectStatement),
- vschema.CurrentDb(),
- vschema,
- keepSameError(tcase.Query, reservedVars, vschema, rewritten.BindVarNeeds),
+ vw.CurrentDb(),
+ vw,
+ keepSameError(tcase.Query, reservedVars, vw, rewritten.BindVarNeeds),
)
if simplified == nil {
@@ -135,12 +135,12 @@ func keepSameError(query string, reservedVars *sqlparser.ReservedVars, vschema *
}
rewritten, _ := sqlparser.RewriteAST(stmt, vschema.CurrentDb(), sqlparser.SQLSelectLimitUnset, "", nil, nil, nil)
ast := rewritten.AST
- _, expected := BuildFromStmt(context.Background(), query, ast, reservedVars, vschema, rewritten.BindVarNeeds, true, true)
+ _, expected := BuildFromStmt(context.Background(), query, ast, reservedVars, vschema, rewritten.BindVarNeeds, staticConfig{})
if expected == nil {
panic("query does not fail to plan")
}
return func(statement sqlparser.SelectStatement) bool {
- _, myErr := BuildFromStmt(context.Background(), query, statement, reservedVars, vschema, needs, true, true)
+ _, myErr := BuildFromStmt(context.Background(), query, statement, reservedVars, vschema, needs, staticConfig{})
if myErr == nil {
return false
}
@@ -162,7 +162,7 @@ func keepPanicking(query string, reservedVars *sqlparser.ReservedVars, vschema *
}
}()
log.Errorf("trying %s", sqlparser.String(statement))
- _, _ = BuildFromStmt(context.Background(), query, statement, reservedVars, vschema, needs, true, true)
+ _, _ = BuildFromStmt(context.Background(), query, statement, reservedVars, vschema, needs, staticConfig{})
log.Errorf("did not panic")
return false
diff --git a/go/vt/vtgate/planbuilder/test_helper.go b/go/vt/vtgate/planbuilder/test_helper.go
new file mode 100644
index 00000000000..25d6b7306d1
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/test_helper.go
@@ -0,0 +1,27 @@
+/*
+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 planbuilder
+
+import "encoding/json"
+
+type PlanTest struct {
+ Comment string `json:"comment,omitempty"`
+ Query string `json:"query,omitempty"`
+ Plan json.RawMessage `json:"plan,omitempty"`
+ Skip bool `json:"skip,omitempty"`
+ SkipE2E bool `json:"skip_e2e,omitempty"`
+}
diff --git a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json
index 628a959af1d..49a03a8f05a 100644
--- a/go/vt/vtgate/planbuilder/testdata/aggr_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/aggr_cases.json
@@ -615,6 +615,35 @@
]
}
},
+ {
+ "comment": "using HAVING inside a derived table still produces viable plans",
+ "query": "select id from (select id from user group by id having (count(user.id) = 2) limit 2 offset 0) subquery_for_limit",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select id from (select id from user group by id having (count(user.id) = 2) limit 2 offset 0) subquery_for_limit",
+ "Instructions": {
+ "OperatorType": "Limit",
+ "Count": "2",
+ "Offset": "0",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id from (select id, count(`user`.id) = 2 from `user` where 1 != 1 group by id) as subquery_for_limit where 1 != 1",
+ "Query": "select id from (select id, count(`user`.id) = 2 from `user` group by id having count(`user`.id) = 2) as subquery_for_limit limit 2",
+ "Table": "`user`"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "user.user"
+ ]
+ }
+ },
{
"comment": "sum with distinct no unique vindex",
"query": "select col1, sum(distinct col2) from user group by col1",
@@ -689,13 +718,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "1 ASC",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "count_distinct(1|3) AS k",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -911,19 +940,44 @@
"Table": "`user`, user_extra"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.`name` from music where 1 != 1",
- "Query": "select music.`name` from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.`name` from music where 1 != 1",
+ "Query": "select music.`name` from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
}
@@ -2149,13 +2203,13 @@
"Instructions": {
"OperatorType": "Filter",
"Predicate": "count(*) <= 10",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS a",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -2187,13 +2241,13 @@
"Instructions": {
"OperatorType": "Filter",
"Predicate": "count(*) = 1.00",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum_count_star(0) AS a",
"GroupBy": "(1|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -2918,16 +2972,16 @@
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "L:0,L:1,L:2",
+ "JoinColumnIndexes": "L:0,L:1,L:3",
"JoinVars": {
- "u2_val2": 3
+ "u2_val2": 2
},
"TableName": "`user`_`user`_music",
"Inputs": [
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "L:0,L:1,L:2,R:0",
+ "JoinColumnIndexes": "L:0,L:1,R:0,L:2",
"JoinVars": {
"u_val2": 1
},
@@ -2963,19 +3017,44 @@
]
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select 1 from music as m where 1 != 1",
- "Query": "select 1 from music as m where m.id = :u2_val2",
- "Table": "music",
"Values": [
":u2_val2"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select 1 from music as m where 1 != 1",
+ "Query": "select 1 from music as m where m.id = :u2_val2",
+ "Table": "music"
+ }
+ ]
}
]
}
@@ -3097,13 +3176,13 @@
"Instructions": {
"OperatorType": "Filter",
"Predicate": "count(*) = 3",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS count(*)",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -3135,13 +3214,13 @@
"Instructions": {
"OperatorType": "Filter",
"Predicate": "sum(foo) + sum(bar) = 42",
+ "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum(1) AS sum(foo), sum(2) AS sum(bar)",
"GroupBy": "(0|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -3173,13 +3252,13 @@
"Instructions": {
"OperatorType": "Filter",
"Predicate": "sum(`user`.foo) + sum(bar) = 42",
+ "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum(1) AS fooSum, sum(2) AS barSum",
"GroupBy": "(0|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -3218,7 +3297,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS count(*)",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -3257,7 +3335,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count(1) AS count(u.`name`)",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Projection",
@@ -3362,7 +3439,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS count(*)",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Projection",
@@ -3621,8 +3697,10 @@
"Aggregates": "count_star(0) AS count(*)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "3",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -3635,8 +3713,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1",
- "Query": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where id > 12) as x limit 10",
+ "FieldQuery": "select 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1",
+ "Query": "select 1 from (select phone, id, city from `user` where id > 12) as x limit 10",
"Table": "`user`"
}
]
@@ -3734,8 +3812,12 @@
"ResultColumns": 2,
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,2,3",
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":1 as val1",
+ "1 as 1",
+ ":2 as weight_string(val1)"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -3748,9 +3830,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1",
- "OrderBy": "(1|3) ASC",
- "Query": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit 2",
+ "FieldQuery": "select x.id, x.val1, weight_string(x.val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1",
+ "OrderBy": "(1|2) ASC",
+ "Query": "select x.id, x.val1, weight_string(x.val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit 2",
"Table": "`user`"
}
]
@@ -3771,25 +3853,42 @@
"QueryType": "SELECT",
"Original": "select * from (select id from user having count(*) = 1) s",
"Instructions": {
- "OperatorType": "Filter",
- "Predicate": "count(*) = 1",
- "ResultColumns": 1,
+ "OperatorType": "SimpleProjection",
+ "ColumnNames": [
+ "0:id"
+ ],
+ "Columns": "0",
"Inputs": [
{
- "OperatorType": "Aggregate",
- "Variant": "Scalar",
- "Aggregates": "any_value(0) AS id, sum_count_star(1) AS count(*)",
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":0 as id",
+ "count(*) = 1 as count(*) = 1"
+ ],
"Inputs": [
{
- "OperatorType": "Route",
- "Variant": "Scatter",
- "Keyspace": {
- "Name": "user",
- "Sharded": true
- },
- "FieldQuery": "select id, count(*) from `user` where 1 != 1",
- "Query": "select id, count(*) from `user`",
- "Table": "`user`"
+ "OperatorType": "Filter",
+ "Predicate": "count(*) = 1",
+ "Inputs": [
+ {
+ "OperatorType": "Aggregate",
+ "Variant": "Scalar",
+ "Aggregates": "any_value(0) AS id, sum_count_star(1) AS count(*), any_value(2)",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id, count(*), 1 from `user` where 1 != 1",
+ "Query": "select id, count(*), 1 from `user`",
+ "Table": "`user`"
+ }
+ ]
+ }
+ ]
}
]
}
@@ -4112,7 +4211,6 @@
"Variant": "Ordered",
"Aggregates": "max(1|3) AS bazo",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -4151,7 +4249,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count(1) AS bazo",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -4999,13 +5096,13 @@
"Collations": [
"0"
],
+ "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum_count_star(0) AS count(*)",
"GroupBy": "1",
- "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "Route",
@@ -5681,7 +5778,6 @@
"OperatorType": "Aggregate",
"Variant": "Ordered",
"GroupBy": "0, (1|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "SimpleProjection",
@@ -5690,7 +5786,7 @@
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "any_value(0) AS id, sum_count_star(1) AS a, any_value(2)",
+ "Aggregates": "any_value(0|2) AS id, sum_count_star(1) AS a",
"Inputs": [
{
"OperatorType": "Route",
@@ -5699,9 +5795,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select id, count(*) as a, weight_string(id) from `user` where 1 != 1",
- "OrderBy": "1 ASC, (0|2) ASC",
- "Query": "select id, count(*) as a, weight_string(id) from `user` order by count(*) asc, id asc",
+ "FieldQuery": "select dt.c0 as id, dt.c1 as a, weight_string(dt.c0), weight_string(dt.c0) from (select id, count(*) as a from `user` where 1 != 1) as dt(c0, c1) where 1 != 1",
+ "OrderBy": "1 ASC, (0|3) ASC",
+ "Query": "select dt.c0 as id, dt.c1 as a, weight_string(dt.c0), weight_string(dt.c0) from (select id, count(*) as a from `user` order by count(*) asc, id asc) as dt(c0, c1)",
"Table": "`user`"
}
]
@@ -5846,16 +5942,15 @@
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "group_concat(1) AS group_concat(u.bar), any_value(2) AS baz, any_value(4)",
+ "Aggregates": "group_concat(1) AS group_concat(u.bar), any_value(2|4) AS baz",
"GroupBy": "(0|3)",
- "ResultColumns": 5,
"Inputs": [
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "L:0,L:1,L:2,L:3,L:4",
+ "JoinColumnIndexes": "L:0,L:1,L:2,L:4,L:5",
"JoinVars": {
- "u_col": 5
+ "u_col": 3
},
"TableName": "`user`_music",
"Inputs": [
@@ -5866,9 +5961,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select u.foo, u.bar, u.baz, weight_string(u.foo), weight_string(u.baz), u.col from `user` as u where 1 != 1",
- "OrderBy": "(0|3) ASC",
- "Query": "select u.foo, u.bar, u.baz, weight_string(u.foo), weight_string(u.baz), u.col from `user` as u order by u.foo asc",
+ "FieldQuery": "select u.foo, u.bar, u.baz, u.col, weight_string(u.foo), weight_string(u.baz) from `user` as u where 1 != 1",
+ "OrderBy": "(0|4) ASC",
+ "Query": "select u.foo, u.bar, u.baz, u.col, weight_string(u.foo), weight_string(u.baz) from `user` as u order by u.foo asc",
"Table": "`user`"
},
{
@@ -6104,18 +6199,19 @@
"Variant": "Ordered",
"Aggregates": "count_star(0)",
"GroupBy": "1",
- "ResultColumns": 1,
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "2,1",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1",
+ "0 as .0"
+ ],
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "sum_count_star(0) AS count(*), any_value(2)",
+ "Aggregates": "sum_count_star(0) AS count(*)",
"GroupBy": "1",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -6124,8 +6220,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select count(*), .0, 1 from `user` where 1 != 1 group by .0",
- "Query": "select count(*), .0, 1 from `user` group by .0",
+ "FieldQuery": "select count(*), .0 from `user` where 1 != 1 group by .0",
+ "Query": "select count(*), .0 from `user` group by .0",
"Table": "`user`"
}
]
@@ -6289,7 +6385,6 @@
"Variant": "Ordered",
"Aggregates": "sum(1) AS avg(id), sum_count(2) AS count(foo), sum_count(3) AS count(id)",
"GroupBy": "(0|4)",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Route",
@@ -6339,7 +6434,6 @@
"Variant": "Ordered",
"Aggregates": "sum(1) AS avg(id), sum_count(2) AS count(foo), sum_count(3) AS count(id)",
"GroupBy": "(0|4)",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Route",
@@ -6523,7 +6617,6 @@
"OperatorType": "Aggregate",
"Variant": "Scalar",
"Aggregates": "min(0|2) AS min_id, max(1|3) AS max_id",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -6663,70 +6756,59 @@
"OrderBy": "(4|6) ASC, (5|7) ASC",
"Inputs": [
{
- "OperatorType": "Projection",
- "Expressions": [
- "count(*) as count(*)",
- "count(*) as count(*)",
- "`user`.col as col",
- "ue.col as col",
- "`user`.foo as foo",
- "ue.bar as bar",
- "weight_string(`user`.foo) as weight_string(`user`.foo)",
- "weight_string(ue.bar) as weight_string(ue.bar)"
- ],
+ "OperatorType": "Join",
+ "Variant": "HashLeftJoin",
+ "Collation": "binary",
+ "ComparisonType": "INT16",
+ "JoinColumnIndexes": "-1,1,-2,2,-3,3,-3,4",
+ "Predicate": "`user`.col = ue.col",
+ "TableName": "`user`_user_extra",
"Inputs": [
{
- "OperatorType": "Join",
- "Variant": "HashLeftJoin",
- "Collation": "binary",
- "ComparisonType": "INT16",
- "JoinColumnIndexes": "-1,1,-2,2,-3,3,-3,3",
- "Predicate": "`user`.col = ue.col",
- "TableName": "`user`_user_extra",
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select count(*), `user`.col, `user`.foo from `user` where 1 != 1 group by `user`.col, `user`.foo",
+ "Query": "select count(*), `user`.col, `user`.foo from `user` group by `user`.col, `user`.foo",
+ "Table": "`user`"
+ },
+ {
+ "OperatorType": "Aggregate",
+ "Variant": "Ordered",
+ "Aggregates": "count_star(0)",
+ "GroupBy": "1, (2|3)",
"Inputs": [
{
- "OperatorType": "Route",
- "Variant": "Scatter",
- "Keyspace": {
- "Name": "user",
- "Sharded": true
- },
- "FieldQuery": "select count(*), `user`.col, `user`.foo from `user` where 1 != 1 group by `user`.col, `user`.foo",
- "Query": "select count(*), `user`.col, `user`.foo from `user` group by `user`.col, `user`.foo",
- "Table": "`user`"
- },
- {
- "OperatorType": "Aggregate",
- "Variant": "Ordered",
- "Aggregates": "count_star(0)",
- "GroupBy": "1, (2|3)",
- "ResultColumns": 3,
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1",
+ ":0 as col",
+ ":1 as bar",
+ ":2 as weight_string(ue.bar)"
+ ],
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "2,0,1,3",
+ "OperatorType": "Sort",
+ "Variant": "Memory",
+ "OrderBy": "0 ASC, (1|2) ASC",
"Inputs": [
{
- "OperatorType": "Sort",
- "Variant": "Memory",
- "OrderBy": "0 ASC, (1|3) ASC",
+ "OperatorType": "Limit",
+ "Count": "10",
"Inputs": [
{
- "OperatorType": "Limit",
- "Count": "10",
- "Inputs": [
- {
- "OperatorType": "Route",
- "Variant": "Scatter",
- "Keyspace": {
- "Name": "user",
- "Sharded": true
- },
- "FieldQuery": "select ue.col, ue.bar, 1, weight_string(ue.bar) from (select col, bar from user_extra where 1 != 1) as ue where 1 != 1",
- "Query": "select ue.col, ue.bar, 1, weight_string(ue.bar) from (select col, bar from user_extra) as ue limit 10",
- "Table": "user_extra"
- }
- ]
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select ue.col, ue.bar, weight_string(ue.bar) from (select col, bar from user_extra where 1 != 1) as ue where 1 != 1",
+ "Query": "select ue.col, ue.bar, weight_string(ue.bar) from (select col, bar from user_extra) as ue limit 10",
+ "Table": "user_extra"
}
]
}
@@ -7271,8 +7353,10 @@
"Aggregates": "count_star(0) AS count(*)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "2",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -7286,9 +7370,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1",
- "OrderBy": "(1|3) DESC",
- "Query": "select subquery_for_count.one, subquery_for_count.id, 1, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit 25",
+ "FieldQuery": "select subquery_for_count.one, subquery_for_count.id, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where 1 != 1) as subquery_for_count where 1 != 1",
+ "OrderBy": "(1|2) DESC",
+ "Query": "select subquery_for_count.one, subquery_for_count.id, weight_string(subquery_for_count.id) from (select 1 as one, id from `user` where `user`.is_not_deleted = true) as subquery_for_count order by subquery_for_count.id desc limit 25",
"Table": "`user`"
}
]
diff --git a/go/vt/vtgate/planbuilder/testdata/cte_cases.json b/go/vt/vtgate/planbuilder/testdata/cte_cases.json
index 1fd398012e3..12f92f5a9f4 100644
--- a/go/vt/vtgate/planbuilder/testdata/cte_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/cte_cases.json
@@ -223,8 +223,10 @@
"Aggregates": "count_star(0) AS count(*)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "3",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -237,8 +239,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1",
- "Query": "select x.phone, x.id, x.city, 1 from (select phone, id, city from `user` where id > 12) as x limit 10",
+ "FieldQuery": "select 1 from (select phone, id, city from `user` where 1 != 1) as x where 1 != 1",
+ "Query": "select 1 from (select phone, id, city from `user` where id > 12) as x limit 10",
"Table": "`user`"
}
]
@@ -336,8 +338,12 @@
"ResultColumns": 2,
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,2,3",
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":1 as val1",
+ "1 as 1",
+ ":2 as weight_string(val1)"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -350,9 +356,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1",
- "OrderBy": "(1|3) ASC",
- "Query": "select x.id, x.val1, 1, weight_string(x.val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit 2",
+ "FieldQuery": "select x.id, x.val1, weight_string(x.val1) from (select id, val1 from `user` where 1 != 1) as x where 1 != 1",
+ "OrderBy": "(1|2) ASC",
+ "Query": "select x.id, x.val1, weight_string(x.val1) from (select id, val1 from `user` where val2 < 4) as x order by x.val1 asc limit 2",
"Table": "`user`"
}
]
@@ -373,25 +379,42 @@
"QueryType": "SELECT",
"Original": "with s as (select id from user having count(*) = 1) select * from s",
"Instructions": {
- "OperatorType": "Filter",
- "Predicate": "count(*) = 1",
- "ResultColumns": 1,
+ "OperatorType": "SimpleProjection",
+ "ColumnNames": [
+ "0:id"
+ ],
+ "Columns": "0",
"Inputs": [
{
- "OperatorType": "Aggregate",
- "Variant": "Scalar",
- "Aggregates": "any_value(0) AS id, sum_count_star(1) AS count(*)",
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":0 as id",
+ "count(*) = 1 as count(*) = 1"
+ ],
"Inputs": [
{
- "OperatorType": "Route",
- "Variant": "Scatter",
- "Keyspace": {
- "Name": "user",
- "Sharded": true
- },
- "FieldQuery": "select id, count(*) from `user` where 1 != 1",
- "Query": "select id, count(*) from `user`",
- "Table": "`user`"
+ "OperatorType": "Filter",
+ "Predicate": "count(*) = 1",
+ "Inputs": [
+ {
+ "OperatorType": "Aggregate",
+ "Variant": "Scalar",
+ "Aggregates": "any_value(0) AS id, sum_count_star(1) AS count(*), any_value(2)",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id, count(*), 1 from `user` where 1 != 1",
+ "Query": "select id, count(*), 1 from `user`",
+ "Table": "`user`"
+ }
+ ]
+ }
+ ]
}
]
}
@@ -479,7 +502,6 @@
"Variant": "Ordered",
"Aggregates": "max(1|3) AS bazo",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -518,7 +540,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count(1) AS bazo",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -565,7 +586,6 @@
"OperatorType": "Aggregate",
"Variant": "Ordered",
"GroupBy": "0, (1|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "SimpleProjection",
@@ -574,7 +594,7 @@
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "any_value(0) AS id, sum_count_star(1) AS a, any_value(2)",
+ "Aggregates": "any_value(0|2) AS id, sum_count_star(1) AS a",
"Inputs": [
{
"OperatorType": "Route",
@@ -583,9 +603,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select id, count(*) as a, weight_string(id) from `user` where 1 != 1",
- "OrderBy": "1 ASC, (0|2) ASC",
- "Query": "select id, count(*) as a, weight_string(id) from `user` order by count(*) asc, id asc",
+ "FieldQuery": "select dt.c0 as id, dt.c1 as a, weight_string(dt.c0), weight_string(dt.c0) from (select id, count(*) as a from `user` where 1 != 1) as dt(c0, c1) where 1 != 1",
+ "OrderBy": "1 ASC, (0|3) ASC",
+ "Query": "select dt.c0 as id, dt.c1 as a, weight_string(dt.c0), weight_string(dt.c0) from (select id, count(*) as a from `user` order by count(*) asc, id asc) as dt(c0, c1)",
"Table": "`user`"
}
]
@@ -718,13 +738,15 @@
"Aggregates": "count_star(0) AS count(*)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "sum_count_star(0) AS count(*), any_value(1)",
+ "Aggregates": "sum_count_star(0) AS count(*)",
"Inputs": [
{
"OperatorType": "Route",
@@ -733,8 +755,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select count(*), 1 from `user` where 1 != 1",
- "Query": "select count(*), 1 from `user`",
+ "FieldQuery": "select count(*) from `user` where 1 != 1",
+ "Query": "select count(*) from `user`",
"Table": "`user`"
}
]
@@ -1999,8 +2021,11 @@
"Aggregates": "any_value(0), sum(1) AS sum(num)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,0",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1000 as 1000",
+ ":0 as num"
+ ],
"Inputs": [
{
"OperatorType": "Concatenate",
@@ -2012,11 +2037,13 @@
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "count_star(0) AS num, any_value(1)",
+ "Aggregates": "count_star(0) AS num",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,2",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -2029,8 +2056,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select t.id, 1, 1000 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1",
- "Query": "select t.id, 1, 1000 from (select `user`.id from `user` where `user`.textcol1 = 'open' and `user`.intcol = 1) as t limit :__upper_limit",
+ "FieldQuery": "select 1 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1",
+ "Query": "select 1 from (select `user`.id from `user` where `user`.textcol1 = 'open' and `user`.intcol = 1) as t limit :__upper_limit",
"Table": "`user`"
}
]
@@ -2048,11 +2075,13 @@
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "count_star(0) AS num, any_value(1)",
+ "Aggregates": "count_star(0) AS num",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,2",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Limit",
@@ -2065,8 +2094,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select t.id, 1, 1000 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1",
- "Query": "select t.id, 1, 1000 from (select `user`.id from `user` where `user`.textcol1 = 'closed' and `user`.intcol = 1) as t limit :__upper_limit",
+ "FieldQuery": "select 1 from (select `user`.id from `user` where 1 != 1) as t where 1 != 1",
+ "Query": "select 1 from (select `user`.id from `user` where `user`.textcol1 = 'closed' and `user`.intcol = 1) as t limit :__upper_limit",
"Table": "`user`"
}
]
@@ -2093,5 +2122,461 @@
"user.user"
]
}
+ },
+ {
+ "comment": "Recursive CTE that cannot be merged",
+ "query": "with recursive cte as (select name, id from user where manager_id is null union all select e.name, e.id from user e inner join cte on e.manager_id = cte.id) select name from cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "with recursive cte as (select name, id from user where manager_id is null union all select e.name, e.id from user e inner join cte on e.manager_id = cte.id) select name from cte",
+ "Instructions": {
+ "OperatorType": "SimpleProjection",
+ "ColumnNames": [
+ "0:name"
+ ],
+ "Columns": "0",
+ "Inputs": [
+ {
+ "OperatorType": "RecurseCTE",
+ "JoinVars": {
+ "cte_id": 1
+ },
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, id from `user` where 1 != 1",
+ "Query": "select `name`, id from `user` where manager_id is null",
+ "Table": "`user`"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select e.`name`, e.id from `user` as e where 1 != 1",
+ "Query": "select e.`name`, e.id from `user` as e where e.manager_id = :cte_id",
+ "Table": "`user`, dual"
+ }
+ ]
+ }
+ ]
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Recursive CTE that cannot be merged 2",
+ "query": "with recursive cte as (select name, id from user where manager_id is null union all select e.name, e.id from cte join user e on e.manager_id = cte.id) select name from cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "with recursive cte as (select name, id from user where manager_id is null union all select e.name, e.id from cte join user e on e.manager_id = cte.id) select name from cte",
+ "Instructions": {
+ "OperatorType": "SimpleProjection",
+ "ColumnNames": [
+ "0:name"
+ ],
+ "Columns": "0",
+ "Inputs": [
+ {
+ "OperatorType": "RecurseCTE",
+ "JoinVars": {
+ "cte_id": 1
+ },
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, id from `user` where 1 != 1",
+ "Query": "select `name`, id from `user` where manager_id is null",
+ "Table": "`user`"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select e.`name`, e.id from `user` as e where 1 != 1",
+ "Query": "select e.`name`, e.id from `user` as e where e.manager_id = :cte_id",
+ "Table": "`user`, dual"
+ }
+ ]
+ }
+ ]
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Merge into a single dual route",
+ "query": "WITH RECURSIVE cte AS (SELECT 1 as n UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT n FROM cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE cte AS (SELECT 1 as n UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT n FROM cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Reference",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": false
+ },
+ "FieldQuery": "with recursive cte as (select 1 as n from dual where 1 != 1 union all select n + 1 from cte where 1 != 1) select n from cte where 1 != 1",
+ "Query": "with recursive cte as (select 1 as n from dual union all select n + 1 from cte where n < 5) select n from cte",
+ "Table": "dual"
+ },
+ "TablesUsed": [
+ "main.dual"
+ ]
+ }
+ },
+ {
+ "comment": "Recursive CTE with star projection",
+ "query": "WITH RECURSIVE cte (n) AS (SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT * FROM cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE cte (n) AS (SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT * FROM cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Reference",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": false
+ },
+ "FieldQuery": "with recursive cte(n) as (select 1 from dual where 1 != 1 union all select n + 1 from cte where 1 != 1) select n from cte where 1 != 1",
+ "Query": "with recursive cte(n) as (select 1 from dual union all select n + 1 from cte where n < 5) select n from cte",
+ "Table": "dual"
+ },
+ "TablesUsed": [
+ "main.dual"
+ ]
+ }
+ },
+ {
+ "comment": "Recursive CTE calculations on the term side - merged",
+ "query": "WITH RECURSIVE emp_cte AS (SELECT id, 1 AS level FROM user WHERE manager_id IS NULL and id = 6 UNION ALL SELECT e.id, cte.level + 1 FROM user e JOIN emp_cte cte ON e.manager_id = cte.id and e.id = 6) SELECT * FROM emp_cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE emp_cte AS (SELECT id, 1 AS level FROM user WHERE manager_id IS NULL and id = 6 UNION ALL SELECT e.id, cte.level + 1 FROM user e JOIN emp_cte cte ON e.manager_id = cte.id and e.id = 6) SELECT * FROM emp_cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "with recursive emp_cte as (select id, 1 as `level` from `user` where 1 != 1 union all select e.id, cte.`level` + 1 from cte as cte, `user` as e where 1 != 1) select id, `level` from emp_cte where 1 != 1",
+ "Query": "with recursive emp_cte as (select id, 1 as `level` from `user` where manager_id is null and id = 6 union all select e.id, cte.`level` + 1 from cte as cte, `user` as e where e.id = 6 and e.manager_id = cte.id) select id, `level` from emp_cte",
+ "Table": "`user`, dual",
+ "Values": [
+ "6"
+ ],
+ "Vindex": "user_index"
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Recursive CTE calculations on the term side - unmerged",
+ "query": "WITH RECURSIVE emp_cte AS (SELECT id, 1 AS level FROM user WHERE manager_id IS NULL UNION ALL SELECT e.id, cte.level + 1 FROM user e JOIN emp_cte cte ON e.manager_id = cte.id) SELECT * FROM emp_cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE emp_cte AS (SELECT id, 1 AS level FROM user WHERE manager_id IS NULL UNION ALL SELECT e.id, cte.level + 1 FROM user e JOIN emp_cte cte ON e.manager_id = cte.id) SELECT * FROM emp_cte",
+ "Instructions": {
+ "OperatorType": "RecurseCTE",
+ "JoinVars": {
+ "cte_id": 0,
+ "cte_level": 1
+ },
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id, 1 as `level` from `user` where 1 != 1",
+ "Query": "select id, 1 as `level` from `user` where manager_id is null",
+ "Table": "`user`"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select e.id, :cte_level + 1 as `cte.``level`` + 1` from `user` as e where 1 != 1",
+ "Query": "select e.id, :cte_level + 1 as `cte.``level`` + 1` from `user` as e where e.manager_id = :cte_id",
+ "Table": "`user`, dual"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Outer join with recursive CTE",
+ "query": "WITH RECURSIVE literal_cte AS (SELECT 1 AS id, 100 AS value, 1 AS manager_id UNION ALL SELECT id + 1, value * 2, id FROM literal_cte WHERE id < 5) SELECT l.id, l.value, l.manager_id, e.name AS employee_name FROM literal_cte l LEFT JOIN user e ON l.id = e.id",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE literal_cte AS (SELECT 1 AS id, 100 AS value, 1 AS manager_id UNION ALL SELECT id + 1, value * 2, id FROM literal_cte WHERE id < 5) SELECT l.id, l.value, l.manager_id, e.name AS employee_name FROM literal_cte l LEFT JOIN user e ON l.id = e.id",
+ "Instructions": {
+ "OperatorType": "Join",
+ "Variant": "LeftJoin",
+ "JoinColumnIndexes": "L:0,L:1,L:2,R:0",
+ "JoinVars": {
+ "l_id": 0
+ },
+ "TableName": "dual_`user`",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Reference",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": false
+ },
+ "FieldQuery": "with recursive literal_cte as (select 1 as id, 100 as value, 1 as manager_id from dual where 1 != 1 union all select id + 1, value * 2, id from literal_cte where 1 != 1) select l.id, l.value, l.manager_id from literal_cte as l where 1 != 1",
+ "Query": "with recursive literal_cte as (select 1 as id, 100 as value, 1 as manager_id from dual union all select id + 1, value * 2, id from literal_cte where id < 5) select l.id, l.value, l.manager_id from literal_cte as l",
+ "Table": "dual"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select e.`name` as employee_name from `user` as e where 1 != 1",
+ "Query": "select e.`name` as employee_name from `user` as e where e.id = :l_id",
+ "Table": "`user`",
+ "Values": [
+ ":l_id"
+ ],
+ "Vindex": "user_index"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Aggregation on the output of a recursive CTE",
+ "query": "WITH RECURSIVE emp_cte AS (SELECT id, name, manager_id FROM user WHERE manager_id IS NULL UNION ALL SELECT e.id, e.name, e.manager_id FROM user e INNER JOIN emp_cte cte ON e.manager_id = cte.id) SELECT manager_id, COUNT(*) AS employee_count FROM emp_cte GROUP BY manager_id",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE emp_cte AS (SELECT id, name, manager_id FROM user WHERE manager_id IS NULL UNION ALL SELECT e.id, e.name, e.manager_id FROM user e INNER JOIN emp_cte cte ON e.manager_id = cte.id) SELECT manager_id, COUNT(*) AS employee_count FROM emp_cte GROUP BY manager_id",
+ "Instructions": {
+ "OperatorType": "Aggregate",
+ "Variant": "Ordered",
+ "Aggregates": "count_star(1) AS employee_count",
+ "GroupBy": "(0|2)",
+ "ResultColumns": 2,
+ "Inputs": [
+ {
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":2 as manager_id",
+ "1 as 1",
+ "weight_string(:2) as weight_string(manager_id)"
+ ],
+ "Inputs": [
+ {
+ "OperatorType": "Sort",
+ "Variant": "Memory",
+ "OrderBy": "(2|3) ASC",
+ "Inputs": [
+ {
+ "OperatorType": "RecurseCTE",
+ "JoinVars": {
+ "cte_id": 0
+ },
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select dt.c0 as id, dt.c1 as `name`, dt.c2 as manager_id, weight_string(dt.c2) from (select id, `name`, manager_id from `user` where 1 != 1) as dt(c0, c1, c2) where 1 != 1",
+ "Query": "select dt.c0 as id, dt.c1 as `name`, dt.c2 as manager_id, weight_string(dt.c2) from (select id, `name`, manager_id from `user` where manager_id is null) as dt(c0, c1, c2)",
+ "Table": "`user`"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select e.id, e.`name`, e.manager_id, weight_string(e.manager_id) from `user` as e where 1 != 1",
+ "Query": "select e.id, e.`name`, e.manager_id, weight_string(e.manager_id) from `user` as e where e.manager_id = :cte_id",
+ "Table": "`user`, dual"
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Query that can be merged, dual on the RHS of the UNION",
+ "query": "with recursive cte as (select id from user where id = 72 union all select id+1 from cte where id < 100) select * from cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "with recursive cte as (select id from user where id = 72 union all select id+1 from cte where id < 100) select * from cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "with recursive cte as (select id from `user` where 1 != 1 union all select id + 1 from cte where 1 != 1) select id from cte where 1 != 1",
+ "Query": "with recursive cte as (select id from `user` where id = 72 union all select id + 1 from cte where id < 100) select id from cte",
+ "Table": "`user`, dual",
+ "Values": [
+ "72"
+ ],
+ "Vindex": "user_index"
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Merge CTE with reference tables",
+ "query": "with recursive cte as (select ue.id, ue.foo from user u join user_extra ue on u.id = ue.user_id union all select sr.id, sr.foo from cte join main.source_of_ref sr on sr.foo = cte.foo join main.rerouted_ref rr on rr.bar = sr.bar) select * from cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "with recursive cte as (select ue.id, ue.foo from user u join user_extra ue on u.id = ue.user_id union all select sr.id, sr.foo from cte join main.source_of_ref sr on sr.foo = cte.foo join main.rerouted_ref rr on rr.bar = sr.bar) select * from cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "with recursive cte as (select ue.id, ue.foo from `user` as u, user_extra as ue where 1 != 1 union all select sr.id, sr.foo from ref_with_source as sr, ref as rr where 1 != 1) select id, foo from cte where 1 != 1",
+ "Query": "with recursive cte as (select ue.id, ue.foo from `user` as u, user_extra as ue where u.id = ue.user_id union all select sr.id, sr.foo from ref_with_source as sr, ref as rr where rr.bar = sr.bar and sr.foo = cte.foo) select id, foo from cte",
+ "Table": "`user`, ref, ref_with_source, user_extra"
+ },
+ "TablesUsed": [
+ "user.ref",
+ "user.ref_with_source",
+ "user.user",
+ "user.user_extra"
+ ]
+ }
+ },
+ {
+ "comment": "Merge CTE with reference tables 2",
+ "query": "with recursive cte as (select sr.id, sr.foo from main.source_of_ref sr join main.rerouted_ref rr on rr.bar = sr.bar union all select ue.id, ue.foo from cte join user_extra ue on cte.foo = ue.foo join user u on ue.user_id = u.id) select * from cte",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "with recursive cte as (select sr.id, sr.foo from main.source_of_ref sr join main.rerouted_ref rr on rr.bar = sr.bar union all select ue.id, ue.foo from cte join user_extra ue on cte.foo = ue.foo join user u on ue.user_id = u.id) select * from cte",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "with recursive cte as (select 1 from ref_with_source as sr, ref as rr where 1 != 1 union all select ue.id, ue.foo from cte, user_extra as ue, `user` as u where 1 != 1) select id, foo from cte where 1 != 1",
+ "Query": "with recursive cte as (select 1 from ref_with_source as sr, ref as rr where rr.bar = sr.bar union all select ue.id, ue.foo from cte, user_extra as ue, `user` as u where ue.user_id = u.id and cte.foo = ue.foo) select id, foo from cte",
+ "Table": "`user`, dual, ref, ref_with_source, user_extra"
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.ref",
+ "user.ref_with_source",
+ "user.user",
+ "user.user_extra"
+ ]
+ }
+ },
+ {
+ "comment": "Merged recursive CTE with DISTINCT",
+ "query": "WITH RECURSIVE hierarchy AS (SELECT id, name, manager_id FROM user UNION ALL SELECT id, name, manager_id FROM user UNION DISTINCT SELECT id*2, name, manager_id from hierarchy WHERE id < 10 ) SELECT * FROM hierarchy",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "WITH RECURSIVE hierarchy AS (SELECT id, name, manager_id FROM user UNION ALL SELECT id, name, manager_id FROM user UNION DISTINCT SELECT id*2, name, manager_id from hierarchy WHERE id < 10 ) SELECT * FROM hierarchy",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "with recursive hierarchy as (select id, `name`, manager_id from `user` where 1 != 1 union all select id, `name`, manager_id from `user` where 1 != 1 union select id * 2, `name`, manager_id from hierarchy where 1 != 1) select id, `name`, manager_id from hierarchy where 1 != 1",
+ "Query": "with recursive hierarchy as (select id, `name`, manager_id from `user` union all select id, `name`, manager_id from `user` union select id * 2, `name`, manager_id from hierarchy where id < 10) select id, `name`, manager_id from hierarchy",
+ "Table": "`user`, dual"
+ },
+ "TablesUsed": [
+ "main.dual",
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Query that caused planner to stack overflow",
+ "query": "SELECT * FROM (SELECT 1 UNION ALL SELECT 2) AS dt(a) WHERE EXISTS(WITH RECURSIVE qn AS (SELECT a * 0 AS b UNION ALL SELECT b + 1 FROM qn WHERE b = 0) SELECT * FROM qn WHERE b = a)",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "SELECT * FROM (SELECT 1 UNION ALL SELECT 2) AS dt(a) WHERE EXISTS(WITH RECURSIVE qn AS (SELECT a * 0 AS b UNION ALL SELECT b + 1 FROM qn WHERE b = 0) SELECT * FROM qn WHERE b = a)",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Reference",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": false
+ },
+ "FieldQuery": "select a from (select 1 from dual where 1 != 1 union all select 2 from dual where 1 != 1) as dt(a) where 1 != 1",
+ "Query": "select a from (select 1 from dual union all select 2 from dual) as dt(a) where exists (with recursive qn as (select a * 0 as b from dual union all select b + 1 from qn where b = 0) select 1 from qn where b = a)",
+ "Table": "dual"
+ },
+ "TablesUsed": [
+ "main.dual"
+ ]
+ }
}
]
diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases.json b/go/vt/vtgate/planbuilder/testdata/dml_cases.json
index b5d0fa8951f..f796b935605 100644
--- a/go/vt/vtgate/planbuilder/testdata/dml_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/dml_cases.json
@@ -2823,7 +2823,7 @@
"Query": "delete from `user` where `name` = _binary 'abc'",
"Table": "user",
"Values": [
- "'abc'"
+ "_binary'abc'"
],
"Vindex": "name_user_map"
},
diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json b/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json
new file mode 100644
index 00000000000..ff66967c2ce
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/testdata/dml_cases_with_user_as_default.json
@@ -0,0 +1,24 @@
+[
+ {
+ "comment": "Update reference table from sharded keyspace to unsharded keyspace",
+ "query": "update ambiguous_ref_with_source set done = true where id = 1;",
+ "plan": {
+ "QueryType": "UPDATE",
+ "Original": "update ambiguous_ref_with_source set done = true where id = 1;",
+ "Instructions": {
+ "OperatorType": "Update",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": false
+ },
+ "TargetTabletType": "PRIMARY",
+ "Query": "update ambiguous_ref_with_source set done = true where id = 1",
+ "Table": "ambiguous_ref_with_source"
+ },
+ "TablesUsed": [
+ "main.ambiguous_ref_with_source"
+ ]
+ }
+ }
+]
\ No newline at end of file
diff --git a/go/vt/vtgate/planbuilder/testdata/filter_cases.json b/go/vt/vtgate/planbuilder/testdata/filter_cases.json
index b60e8812dda..edce4ebd0cb 100644
--- a/go/vt/vtgate/planbuilder/testdata/filter_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/filter_cases.json
@@ -789,7 +789,7 @@
"Sharded": true
},
"FieldQuery": "select Id from `user` where 1 != 1",
- "Query": "select Id from `user` where 1 in ('aa', 'bb')",
+ "Query": "select Id from `user` where 0",
"Table": "`user`"
},
"TablesUsed": [
@@ -797,6 +797,108 @@
]
}
},
+ {
+ "comment": "Disjunction of conjunctions with 4 or more disjunctions",
+ "query": "select id from user where (col = 'aa' AND name = 'bb') OR (col = 'cc' AND name = 'dd') OR (col = 'ee' AND name = 'ff') OR (col = 'gg' AND name = 'hh')",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select id from user where (col = 'aa' AND name = 'bb') OR (col = 'cc' AND name = 'dd') OR (col = 'ee' AND name = 'ff') OR (col = 'gg' AND name = 'hh')",
+ "Instructions": {
+ "OperatorType": "VindexLookup",
+ "Variant": "MultiEqual",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "Values": [
+ "('bb', 'dd', 'ff', 'hh')"
+ ],
+ "Vindex": "name_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id from `user` where 1 != 1",
+ "Query": "select id from `user` where col = 'aa' and `name` = 'bb' or col = 'cc' and `name` = 'dd' or col = 'ee' and `name` = 'ff' or col = 'gg' and `name` = 'hh'",
+ "Table": "`user`"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "user.user"
+ ]
+ }
+ },
+ {
+ "comment": "Disjunction of conjunctions with 3 or less disjunctions",
+ "query": "select id from user where (col = 'aa' AND name = 'bb') OR (col = 'cc' AND name = 'dd') OR (col = 'ee' AND name = 'ff')",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select id from user where (col = 'aa' AND name = 'bb') OR (col = 'cc' AND name = 'dd') OR (col = 'ee' AND name = 'ff')",
+ "Instructions": {
+ "OperatorType": "VindexLookup",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "Values": [
+ "('bb', 'dd', 'ff')"
+ ],
+ "Vindex": "name_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select id from `user` where 1 != 1",
+ "Query": "select id from `user` where col in ('aa', 'cc', 'ee') and (col in ('aa', 'cc') or `name` = 'ff') and (col = 'aa' or `name` = 'dd' or col = 'ee') and (col = 'aa' or `name` = 'dd' or `name` = 'ff') and (`name` = 'bb' or col = 'cc' or col = 'ee') and (`name` = 'bb' or col = 'cc' or `name` = 'ff') and (`name` in ('bb', 'dd') or col = 'ee') and `name` in ::__vals",
+ "Table": "`user`"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "user.user"
+ ]
+ }
+ },
{
"comment": "Single table complex in clause",
"query": "select id from user where name in (col, 'bb')",
@@ -1251,7 +1353,7 @@
"Sharded": true
},
"FieldQuery": "select `user`.col from `user` where 1 != 1",
- "Query": "select `user`.col from `user` where 1 = 1",
+ "Query": "select `user`.col from `user`",
"Table": "`user`"
},
{
@@ -1262,7 +1364,7 @@
"Sharded": true
},
"FieldQuery": "select user_extra.id from user_extra where 1 != 1",
- "Query": "select user_extra.id from user_extra where user_extra.col = :user_col /* INT16 */ and 1 = 1",
+ "Query": "select user_extra.id from user_extra where user_extra.col = :user_col /* INT16 */",
"Table": "user_extra"
}
]
@@ -3302,19 +3404,44 @@
"QueryType": "SELECT",
"Original": "select * from multicolvin where column_b = 1",
"Instructions": {
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select * from multicolvin where 1 != 1",
- "Query": "select * from multicolvin where column_b = 1",
- "Table": "multicolvin",
"Values": [
"1"
],
- "Vindex": "colb_colc_map"
+ "Vindex": "colb_colc_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select colb, keyspace_id from colb_colc_map where 1 != 1",
+ "Query": "select colb, keyspace_id from colb_colc_map where colb in ::__vals",
+ "Table": "colb_colc_map",
+ "Values": [
+ "::colb"
+ ],
+ "Vindex": "hash"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select * from multicolvin where 1 != 1",
+ "Query": "select * from multicolvin where column_b = 1",
+ "Table": "multicolvin"
+ }
+ ]
},
"TablesUsed": [
"user.multicolvin"
@@ -3328,19 +3455,44 @@
"QueryType": "SELECT",
"Original": "select * from multicolvin where column_b = 1 and column_c = 2",
"Instructions": {
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select * from multicolvin where 1 != 1",
- "Query": "select * from multicolvin where column_b = 1 and column_c = 2",
- "Table": "multicolvin",
"Values": [
"1"
],
- "Vindex": "colb_colc_map"
+ "Vindex": "colb_colc_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select colb, keyspace_id from colb_colc_map where 1 != 1",
+ "Query": "select colb, keyspace_id from colb_colc_map where colb in ::__vals",
+ "Table": "colb_colc_map",
+ "Values": [
+ "::colb"
+ ],
+ "Vindex": "hash"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select * from multicolvin where 1 != 1",
+ "Query": "select * from multicolvin where column_b = 1 and column_c = 2",
+ "Table": "multicolvin"
+ }
+ ]
},
"TablesUsed": [
"user.multicolvin"
@@ -3354,19 +3506,44 @@
"QueryType": "SELECT",
"Original": "select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3",
"Instructions": {
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select * from multicolvin where 1 != 1",
- "Query": "select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3",
- "Table": "multicolvin",
"Values": [
"1"
],
- "Vindex": "colb_colc_map"
+ "Vindex": "colb_colc_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select colb, keyspace_id from colb_colc_map where 1 != 1",
+ "Query": "select colb, keyspace_id from colb_colc_map where colb in ::__vals",
+ "Table": "colb_colc_map",
+ "Values": [
+ "::colb"
+ ],
+ "Vindex": "hash"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select * from multicolvin where 1 != 1",
+ "Query": "select * from multicolvin where column_b = 1 and column_c = 2 and column_a = 3",
+ "Table": "multicolvin"
+ }
+ ]
},
"TablesUsed": [
"user.multicolvin"
@@ -3380,19 +3557,44 @@
"QueryType": "SELECT",
"Original": "select * from multicolvin where column_a = 3 and column_b = 1",
"Instructions": {
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select * from multicolvin where 1 != 1",
- "Query": "select * from multicolvin where column_a = 3 and column_b = 1",
- "Table": "multicolvin",
"Values": [
"1"
],
- "Vindex": "colb_colc_map"
+ "Vindex": "colb_colc_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select colb, keyspace_id from colb_colc_map where 1 != 1",
+ "Query": "select colb, keyspace_id from colb_colc_map where colb in ::__vals",
+ "Table": "colb_colc_map",
+ "Values": [
+ "::colb"
+ ],
+ "Vindex": "hash"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select * from multicolvin where 1 != 1",
+ "Query": "select * from multicolvin where column_a = 3 and column_b = 1",
+ "Table": "multicolvin"
+ }
+ ]
},
"TablesUsed": [
"user.multicolvin"
diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json
index 47f10cd273b..799c9bd4420 100644
--- a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json
@@ -1518,7 +1518,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where 1 != 1",
- "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and cast('foo' as CHAR) is not null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
+ "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
"Table": "u_tbl8, u_tbl9"
},
{
@@ -1594,7 +1594,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where 1 != 1",
- "Query": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where u_tbl3.col3 is null and cast('foo' as CHAR) is not null and not (u_tbl4.col4) <=> (cast('foo' as CHAR)) and (u_tbl4.col4) in ::fkc_vals limit 1 for share",
+ "Query": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where u_tbl3.col3 is null and not (u_tbl4.col4) <=> (cast('foo' as CHAR)) and (u_tbl4.col4) in ::fkc_vals limit 1 for share",
"Table": "u_tbl3, u_tbl4"
},
{
@@ -1606,7 +1606,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl4, u_tbl9 where 1 != 1",
- "Query": "select 1 from u_tbl4, u_tbl9 where u_tbl4.col4 = u_tbl9.col9 and (u_tbl4.col4) in ::fkc_vals and (cast('foo' as CHAR) is null or (u_tbl9.col9) not in ((cast('foo' as CHAR)))) limit 1 for share",
+ "Query": "select 1 from u_tbl4, u_tbl9 where u_tbl4.col4 = u_tbl9.col9 and (u_tbl4.col4) in ::fkc_vals and (u_tbl9.col9) not in ((cast('foo' as CHAR))) limit 1 for share",
"Table": "u_tbl4, u_tbl9"
},
{
@@ -2532,7 +2532,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_multicol_tbl2 left join u_multicol_tbl1 on u_multicol_tbl1.cola = 2 and u_multicol_tbl1.colb = u_multicol_tbl2.colc - 2 where 1 != 1",
- "Query": "select 1 from u_multicol_tbl2 left join u_multicol_tbl1 on u_multicol_tbl1.cola = 2 and u_multicol_tbl1.colb = u_multicol_tbl2.colc - 2 where u_multicol_tbl1.cola is null and 2 is not null and u_multicol_tbl1.colb is null and u_multicol_tbl2.colc - 2 is not null and not (u_multicol_tbl2.cola, u_multicol_tbl2.colb) <=> (2, u_multicol_tbl2.colc - 2) and u_multicol_tbl2.id = 7 limit 1 for share",
+ "Query": "select 1 from u_multicol_tbl2 left join u_multicol_tbl1 on u_multicol_tbl1.cola = 2 and u_multicol_tbl1.colb = u_multicol_tbl2.colc - 2 where u_multicol_tbl1.cola is null and u_multicol_tbl1.colb is null and u_multicol_tbl2.colc - 2 is not null and not (u_multicol_tbl2.cola, u_multicol_tbl2.colb) <=> (2, u_multicol_tbl2.colc - 2) and u_multicol_tbl2.id = 7 limit 1 for share",
"Table": "u_multicol_tbl1, u_multicol_tbl2"
},
{
@@ -4110,7 +4110,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where 1 != 1",
- "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and cast('foo' as CHAR) is not null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
+ "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
"Table": "u_tbl8, u_tbl9"
},
{
diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json
index 7b525b2dcc9..5464ccbd619 100644
--- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json
@@ -1595,7 +1595,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where 1 != 1",
- "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and cast('foo' as CHAR) is not null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
+ "Query": "select 1 from u_tbl8 left join u_tbl9 on u_tbl9.col9 = cast('foo' as CHAR) where u_tbl9.col9 is null and not (u_tbl8.col8) <=> (cast('foo' as CHAR)) and (u_tbl8.col8) in ::fkc_vals limit 1 for share nowait",
"Table": "u_tbl8, u_tbl9"
},
{
@@ -1671,7 +1671,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where 1 != 1",
- "Query": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where u_tbl3.col3 is null and cast('foo' as CHAR) is not null and not (u_tbl4.col4) <=> (cast('foo' as CHAR)) and (u_tbl4.col4) in ::fkc_vals limit 1 for share",
+ "Query": "select 1 from u_tbl4 left join u_tbl3 on u_tbl3.col3 = cast('foo' as CHAR) where u_tbl3.col3 is null and not (u_tbl4.col4) <=> (cast('foo' as CHAR)) and (u_tbl4.col4) in ::fkc_vals limit 1 for share",
"Table": "u_tbl3, u_tbl4"
},
{
@@ -1683,7 +1683,7 @@
"Sharded": false
},
"FieldQuery": "select 1 from u_tbl4, u_tbl9 where 1 != 1",
- "Query": "select 1 from u_tbl4, u_tbl9 where u_tbl4.col4 = u_tbl9.col9 and (u_tbl4.col4) in ::fkc_vals and (cast('foo' as CHAR) is null or (u_tbl9.col9) not in ((cast('foo' as CHAR)))) limit 1 for share",
+ "Query": "select 1 from u_tbl4, u_tbl9 where u_tbl4.col4 = u_tbl9.col9 and (u_tbl4.col4) in ::fkc_vals and (u_tbl9.col9) not in ((cast('foo' as CHAR))) limit 1 for share",
"Table": "u_tbl4, u_tbl9"
},
{
diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json
index ca94f4ee866..bec64fd7b1e 100644
--- a/go/vt/vtgate/planbuilder/testdata/from_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json
@@ -788,7 +788,6 @@
"OperatorType": "Aggregate",
"Variant": "Ordered",
"GroupBy": "(0|1)",
- "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "SimpleProjection",
@@ -799,7 +798,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS count",
"GroupBy": "(0|2)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -4594,20 +4592,21 @@
"Aggregates": "count_star(0) AS count(*)",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Distinct",
"Collations": [
- "(0:2)",
- "1"
+ "(0:1)"
],
"Inputs": [
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "R:0,L:1,R:1",
+ "JoinColumnIndexes": "R:0,R:1",
"JoinVars": {
"m_id": 0
},
@@ -4620,8 +4619,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select subquery_for_count.`m.id`, 1 from (select m.id as `m.id` from music as m where 1 != 1) as subquery_for_count where 1 != 1",
- "Query": "select distinct subquery_for_count.`m.id`, 1 from (select m.id as `m.id` from music as m) as subquery_for_count",
+ "FieldQuery": "select subquery_for_count.`m.id` from (select m.id as `m.id` from music as m where 1 != 1) as subquery_for_count where 1 != 1",
+ "Query": "select distinct subquery_for_count.`m.id` from (select m.id as `m.id` from music as m) as subquery_for_count",
"Table": "music"
},
{
@@ -4710,19 +4709,44 @@
]
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select 1 from music as m where 1 != 1",
- "Query": "select 1 from music as m where m.user_id = 5 and m.id = 20 and m.col = :u_col /* INT16 */",
- "Table": "music",
"Values": [
"20"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select 1 from music as m where 1 != 1",
+ "Query": "select 1 from music as m where m.user_id = 5 and m.id = 20 and m.col = :u_col /* INT16 */",
+ "Table": "music"
+ }
+ ]
}
]
},
diff --git a/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json b/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json
index f26b160ef69..a35949cd4c1 100644
--- a/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/memory_sort_cases.json
@@ -14,9 +14,8 @@
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "any_value(1) AS b, sum_count_star(2) AS count(*), any_value(4)",
+ "Aggregates": "any_value(1|4) AS b, sum_count_star(2) AS count(*)",
"GroupBy": "(0|3)",
- "ResultColumns": 5,
"Inputs": [
{
"OperatorType": "Route",
@@ -25,9 +24,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select a, b, count(*), weight_string(a), weight_string(`user`.b) from `user` where 1 != 1 group by a, weight_string(a)",
+ "FieldQuery": "select dt.c0 as a, dt.c1 as b, dt.c2 as `count(*)`, dt.c3 as `weight_string(a)`, weight_string(dt.c1) from (select a, b, count(*), weight_string(a) from `user` where 1 != 1 group by a, weight_string(a)) as dt(c0, c1, c2, c3) where 1 != 1",
"OrderBy": "(0|3) ASC",
- "Query": "select a, b, count(*), weight_string(a), weight_string(`user`.b) from `user` group by a, weight_string(a) order by a asc",
+ "Query": "select dt.c0 as a, dt.c1 as b, dt.c2 as `count(*)`, dt.c3 as `weight_string(a)`, weight_string(dt.c1) from (select a, b, count(*), weight_string(a) from `user` group by a, weight_string(a) order by a asc) as dt(c0, c1, c2, c3)",
"Table": "`user`"
}
]
@@ -49,13 +48,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "2 ASC",
+ "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "any_value(1) AS b, sum_count_star(2) AS k",
"GroupBy": "(0|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -93,9 +92,8 @@
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "any_value(1) AS b, sum_count_star(2) AS k, any_value(4)",
+ "Aggregates": "any_value(1|4) AS b, sum_count_star(2) AS k",
"GroupBy": "(0|3)",
- "ResultColumns": 5,
"Inputs": [
{
"OperatorType": "Route",
@@ -104,9 +102,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select a, b, count(*) as k, weight_string(a), weight_string(`user`.b) from `user` where 1 != 1 group by a, weight_string(a)",
+ "FieldQuery": "select dt.c0 as a, dt.c1 as b, dt.c2 as k, dt.c3 as `weight_string(a)`, weight_string(dt.c1) from (select a, b, count(*) as k, weight_string(a) from `user` where 1 != 1 group by a, weight_string(a)) as dt(c0, c1, c2, c3) where 1 != 1",
"OrderBy": "(0|3) ASC",
- "Query": "select a, b, count(*) as k, weight_string(a), weight_string(`user`.b) from `user` group by a, weight_string(a) order by a asc",
+ "Query": "select dt.c0 as a, dt.c1 as b, dt.c2 as k, dt.c3 as `weight_string(a)`, weight_string(dt.c1) from (select a, b, count(*) as k, weight_string(a) from `user` group by a, weight_string(a) order by a asc) as dt(c0, c1, c2, c3)",
"Table": "`user`"
}
]
@@ -132,13 +130,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "2 DESC",
+ "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "any_value(1) AS b, sum_count_star(2) AS k",
"GroupBy": "(0|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -180,7 +178,6 @@
"Variant": "Ordered",
"Aggregates": "any_value(1) AS b, sum_count_star(2) AS k",
"GroupBy": "(0|3)",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Route",
@@ -321,19 +318,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3 as c, weight_string(music.col3) from music where 1 != 1",
- "Query": "select music.col3 as c, weight_string(music.col3) from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3 as c, weight_string(music.col3) from music where 1 != 1",
+ "Query": "select music.col3 as c, weight_string(music.col3) from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
}
@@ -360,9 +382,9 @@
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "L:0,L:1,R:0,L:2,R:1,L:3",
+ "JoinColumnIndexes": "L:0,L:1,R:0,L:3,R:1,L:4",
"JoinVars": {
- "user_id": 4
+ "user_id": 2
},
"TableName": "`user`_music",
"Inputs": [
@@ -373,8 +395,8 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select `user`.col1 as a, `user`.col2, weight_string(`user`.col1), weight_string(`user`.col2), `user`.id from `user` where 1 != 1",
- "Query": "select `user`.col1 as a, `user`.col2, weight_string(`user`.col1), weight_string(`user`.col2), `user`.id from `user` where `user`.id = 1",
+ "FieldQuery": "select `user`.col1 as a, `user`.col2, `user`.id, weight_string(`user`.col1), weight_string(`user`.col2) from `user` where 1 != 1",
+ "Query": "select `user`.col1 as a, `user`.col2, `user`.id, weight_string(`user`.col1), weight_string(`user`.col2) from `user` where `user`.id = 1",
"Table": "`user`",
"Values": [
"1"
@@ -382,19 +404,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3, weight_string(music.col3) from music where 1 != 1",
- "Query": "select music.col3, weight_string(music.col3) from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3, weight_string(music.col3) from music where 1 != 1",
+ "Query": "select music.col3, weight_string(music.col3) from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
}
diff --git a/go/vt/vtgate/planbuilder/testdata/mirror_cases.json b/go/vt/vtgate/planbuilder/testdata/mirror_cases.json
new file mode 100644
index 00000000000..2466b3dca12
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/testdata/mirror_cases.json
@@ -0,0 +1,397 @@
+[
+ {
+ "comment": "select unsharded, qualified, table mirrored to unsharded table",
+ "query": "select t1.id from unsharded_src1.t1 where t1.id = 1",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t1.id from unsharded_src1.t1 where t1.id = 1",
+ "Instructions": {
+ "OperatorType": "Mirror",
+ "Variant": "PercentBased",
+ "Percent": 1,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1",
+ "Query": "select t1.id from t1 where t1.id = 1",
+ "Table": "t1"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_dst1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1",
+ "Query": "select t1.id from t1 where t1.id = 1",
+ "Table": "t1"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "unsharded_dst1.t1",
+ "unsharded_src1.t1"
+ ]
+ }
+ },
+ {
+ "comment": "select unsharded, qualified, table mirrored to unsharded table with zero percentage",
+ "query": "select t3.id from unsharded_src1.t3 where t3.id = 1",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t3.id from unsharded_src1.t3 where t3.id = 1",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t3.id from t3 where 1 != 1",
+ "Query": "select t3.id from t3 where t3.id = 1",
+ "Table": "t3"
+ },
+ "TablesUsed": [
+ "unsharded_src1.t3"
+ ]
+ }
+ },
+ {
+ "comment": "select unsharded, qualified, table mirrored to sharded table",
+ "query": "select t2.id from unsharded_src1.t2 where t2.id = 1",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t2.id from unsharded_src1.t2 where t2.id = 1",
+ "Instructions": {
+ "OperatorType": "Mirror",
+ "Variant": "PercentBased",
+ "Percent": 2,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t2.id from t2 where 1 != 1",
+ "Query": "select t2.id from t2 where t2.id = 1",
+ "Table": "t2"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "sharded_dst1",
+ "Sharded": true
+ },
+ "FieldQuery": "select t2.id from t1 as t2 where 1 != 1",
+ "Query": "select t2.id from t1 as t2 where t2.id = 1",
+ "Table": "t1",
+ "Values": [
+ "1"
+ ],
+ "Vindex": "xxhash"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "sharded_dst1.t1",
+ "unsharded_src1.t2"
+ ]
+ }
+ },
+ {
+ "comment": "select two unsharded, qualified, tables, one mirrored to unsharded table, other to sharded table",
+ "query": "select t1.id, t2.id from unsharded_src1.t1, unsharded_src1.t2 where t1.id = t2.id",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t1.id, t2.id from unsharded_src1.t1, unsharded_src1.t2 where t1.id = t2.id",
+ "Instructions": {
+ "OperatorType": "Mirror",
+ "Variant": "PercentBased",
+ "Percent": 1,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id, t2.id from t1, t2 where 1 != 1",
+ "Query": "select t1.id, t2.id from t1, t2 where t1.id = t2.id",
+ "Table": "t1, t2"
+ },
+ {
+ "OperatorType": "Join",
+ "Variant": "Join",
+ "JoinColumnIndexes": "L:0,R:0",
+ "JoinVars": {
+ "t1_id1": 0
+ },
+ "TableName": "t1_t1",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_dst1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1",
+ "Query": "select t1.id from t1",
+ "Table": "t1"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "sharded_dst1",
+ "Sharded": true
+ },
+ "FieldQuery": "select t2.id from t1 as t2 where 1 != 1",
+ "Query": "select t2.id from t1 as t2 where t2.id = :t1_id1",
+ "Table": "t1",
+ "Values": [
+ ":t1_id1"
+ ],
+ "Vindex": "xxhash"
+ }
+ ]
+ }
+ ]
+ },
+ "TablesUsed": [
+ "sharded_dst1.t1",
+ "unsharded_dst1.t1",
+ "unsharded_src1.t1",
+ "unsharded_src1.t2"
+ ]
+ }
+ },
+ {
+ "comment": "union of selects from unsharded, qualified, tables, one mirrored to unsharded table, other to sharded table",
+ "query": "select t1.id from unsharded_src1.t1 union select t2.id from unsharded_src1.t2",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t1.id from unsharded_src1.t1 union select t2.id from unsharded_src1.t2",
+ "Instructions": {
+ "OperatorType": "Mirror",
+ "Variant": "PercentBased",
+ "Percent": 1,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1 union select t2.id from t2 where 1 != 1",
+ "Query": "select t1.id from t1 union select t2.id from t2",
+ "Table": "t1, t2"
+ },
+ {
+ "OperatorType": "Distinct",
+ "Collations": [
+ "(0:1)"
+ ],
+ "Inputs": [
+ {
+ "OperatorType": "Concatenate",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_dst1",
+ "Sharded": false
+ },
+ "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select t1.id from t1 where 1 != 1) as dt(c0) where 1 != 1",
+ "Query": "select dt.c0 as id, weight_string(dt.c0) from (select distinct t1.id from t1) as dt(c0)",
+ "Table": "t1"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "sharded_dst1",
+ "Sharded": true
+ },
+ "FieldQuery": "select dt.c0 as id, weight_string(dt.c0) from (select t2.id from t1 as t2 where 1 != 1) as dt(c0) where 1 != 1",
+ "Query": "select dt.c0 as id, weight_string(dt.c0) from (select distinct t2.id from t1 as t2) as dt(c0)",
+ "Table": "t1"
+ }
+ ]
+ }
+ ]
+ }
+ ]
+ },
+ "TablesUsed": [
+ "sharded_dst1.t1",
+ "unsharded_dst1.t1",
+ "unsharded_src1.t1",
+ "unsharded_src1.t2"
+ ]
+ }
+ },
+ {
+ "comment": "inserts are not mirrored",
+ "query": "insert into unsharded_src1.t1 (id) values(1)",
+ "plan": {
+ "QueryType": "INSERT",
+ "Original": "insert into unsharded_src1.t1 (id) values(1)",
+ "Instructions": {
+ "OperatorType": "Insert",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "TargetTabletType": "PRIMARY",
+ "Query": "insert into t1(id) values (1)",
+ "TableName": "t1"
+ },
+ "TablesUsed": [
+ "unsharded_src1.t1"
+ ]
+ }
+ },
+ {
+ "comment": "updates are not mirrored",
+ "query": "update unsharded_src1.t1 set data = 'a' where id = 1",
+ "plan": {
+ "QueryType": "UPDATE",
+ "Original": "update unsharded_src1.t1 set data = 'a' where id = 1",
+ "Instructions": {
+ "OperatorType": "Update",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "TargetTabletType": "PRIMARY",
+ "Query": "update t1 set `data` = 'a' where id = 1",
+ "Table": "t1"
+ },
+ "TablesUsed": [
+ "unsharded_src1.t1"
+ ]
+ }
+ },
+ {
+ "comment": "deletes are not mirrored",
+ "query": "delete from unsharded_src1.t1 where id = 1",
+ "plan": {
+ "QueryType": "DELETE",
+ "Original": "delete from unsharded_src1.t1 where id = 1",
+ "Instructions": {
+ "OperatorType": "Delete",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src1",
+ "Sharded": false
+ },
+ "TargetTabletType": "PRIMARY",
+ "Query": "delete from t1 where id = 1",
+ "Table": "t1"
+ },
+ "TablesUsed": [
+ "unsharded_src1.t1"
+ ]
+ }
+ },
+ {
+ "comment": "self-mirror is not allowed",
+ "query": "select t1.id from unsharded_src2.t1",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t1.id from unsharded_src2.t1",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src2",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1",
+ "Query": "select t1.id from t1",
+ "Table": "t1"
+ },
+ "TablesUsed": [
+ "unsharded_src2.t1"
+ ]
+ }
+ },
+ {
+ "comment": "chained mirror is not allowed",
+ "query": "select t2.id from unsharded_src2.t2",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t2.id from unsharded_src2.t2",
+ "Instructions": {
+ "OperatorType": "Mirror",
+ "Variant": "PercentBased",
+ "Percent": 4,
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src2",
+ "Sharded": false
+ },
+ "FieldQuery": "select t2.id from t2 where 1 != 1",
+ "Query": "select t2.id from t2",
+ "Table": "t2"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_dst2",
+ "Sharded": false
+ },
+ "FieldQuery": "select t2.id from t2 where 1 != 1",
+ "Query": "select t2.id from t2",
+ "Table": "t2"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "unsharded_dst2.t2",
+ "unsharded_src2.t2"
+ ]
+ }
+ },
+ {
+ "comment": "circular mirror is not allowed",
+ "query": "select t1.id from unsharded_src3.t1",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select t1.id from unsharded_src3.t1",
+ "Instructions": {
+ "OperatorType": "Route",
+ "Variant": "Unsharded",
+ "Keyspace": {
+ "Name": "unsharded_src3",
+ "Sharded": false
+ },
+ "FieldQuery": "select t1.id from t1 where 1 != 1",
+ "Query": "select t1.id from t1",
+ "Table": "t1"
+ },
+ "TablesUsed": [
+ "unsharded_src3.t1"
+ ]
+ }
+ }
+]
diff --git a/go/vt/vtgate/planbuilder/testdata/onecase.json b/go/vt/vtgate/planbuilder/testdata/onecase.json
index da7543f706a..9d653b2f6e9 100644
--- a/go/vt/vtgate/planbuilder/testdata/onecase.json
+++ b/go/vt/vtgate/planbuilder/testdata/onecase.json
@@ -3,7 +3,6 @@
"comment": "Add your test case here for debugging and run go test -run=One.",
"query": "",
"plan": {
-
}
}
]
\ No newline at end of file
diff --git a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json
index 74e5229016a..6a8e94c0241 100644
--- a/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/postprocess_cases.json
@@ -544,19 +544,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3 from music where 1 != 1",
- "Query": "select music.col3 from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3 from music where 1 != 1",
+ "Query": "select music.col3 from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -597,19 +622,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3 from music where 1 != 1",
- "Query": "select music.col3 from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3 from music where 1 != 1",
+ "Query": "select music.col3 from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -650,19 +700,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3 from music where 1 != 1",
- "Query": "select music.col3 from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3 from music where 1 != 1",
+ "Query": "select music.col3 from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -770,19 +845,44 @@
"Vindex": "user_index"
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.col3 from music where 1 != 1",
- "Query": "select music.col3 from music where music.id = :user_id",
- "Table": "music",
"Values": [
":user_id"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.col3 from music where 1 != 1",
+ "Query": "select music.col3 from music where music.id = :user_id",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -1554,7 +1654,7 @@
{
"OperatorType": "Aggregate",
"Variant": "Scalar",
- "Aggregates": "sum_count(0) AS count(id), any_value(1) AS num, any_value(2)",
+ "Aggregates": "sum_count(0) AS count(id), any_value(1|2) AS num",
"Inputs": [
{
"OperatorType": "Route",
@@ -1618,13 +1718,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "0 ASC",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum_count(0) AS count(id)",
"GroupBy": "(1|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Route",
@@ -1729,7 +1829,7 @@
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "sum_count_star(1) AS count(*), any_value(2) AS c1, any_value(3)",
+ "Aggregates": "sum_count_star(1) AS count(*), any_value(2|3) AS c1",
"GroupBy": "0",
"Inputs": [
{
@@ -1739,9 +1839,9 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select col, count(*), c1, weight_string(c1) from `user` where 1 != 1 group by col",
+ "FieldQuery": "select dt.c0 as col, dt.c1 as `count(*)`, dt.c2 as c1, weight_string(dt.c2) from (select col, count(*), c1 from `user` where 1 != 1 group by col) as dt(c0, c1, c2) where 1 != 1",
"OrderBy": "0 ASC",
- "Query": "select col, count(*), c1, weight_string(c1) from `user` group by col order by col asc",
+ "Query": "select dt.c0 as col, dt.c1 as `count(*)`, dt.c2 as c1, weight_string(dt.c2) from (select col, count(*), c1 from `user` group by col order by col asc) as dt(c0, c1, c2)",
"Table": "`user`"
}
]
@@ -2090,7 +2190,6 @@
"Variant": "Ordered",
"Aggregates": "min(1|3) AS min(a.id)",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Join",
diff --git a/go/vt/vtgate/planbuilder/testdata/schemas/main.sql b/go/vt/vtgate/planbuilder/testdata/schemas/main.sql
new file mode 100644
index 00000000000..8c15b99218c
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/testdata/schemas/main.sql
@@ -0,0 +1,12 @@
+CREATE TABLE `unsharded` (
+ `id` INT NOT NULL PRIMARY KEY,
+ `col1` VARCHAR(255) DEFAULT NULL,
+ `col2` VARCHAR(255) DEFAULT NULL,
+ `name` VARCHAR(255) DEFAULT NULL
+);
+
+CREATE TABLE `unsharded_auto` (
+ `id` INT NOT NULL PRIMARY KEY,
+ `col1` VARCHAR(255) DEFAULT NULL,
+ `col2` VARCHAR(255) DEFAULT NULL
+);
\ No newline at end of file
diff --git a/go/vt/vtgate/planbuilder/testdata/schemas/user.sql b/go/vt/vtgate/planbuilder/testdata/schemas/user.sql
new file mode 100644
index 00000000000..55f4078557a
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/testdata/schemas/user.sql
@@ -0,0 +1,100 @@
+CREATE TABLE user
+(
+ id INT PRIMARY KEY,
+ col BIGINT,
+ predef1 VARCHAR(255),
+ predef2 VARCHAR(255),
+ textcol1 VARCHAR(255),
+ intcol BIGINT,
+ textcol2 VARCHAR(255)
+);
+
+CREATE TABLE user_metadata
+(
+ user_id INT,
+ email VARCHAR(255),
+ address VARCHAR(255),
+ md5 VARCHAR(255),
+ non_planable VARCHAR(255),
+ PRIMARY KEY (user_id)
+);
+
+CREATE TABLE music
+(
+ user_id INT,
+ id INT,
+ PRIMARY KEY (user_id)
+);
+
+CREATE TABLE samecolvin
+(
+ col VARCHAR(255),
+ PRIMARY KEY (col)
+);
+
+CREATE TABLE multicolvin
+(
+ kid INT,
+ column_a VARCHAR(255),
+ column_b VARCHAR(255),
+ column_c VARCHAR(255),
+ PRIMARY KEY (kid)
+);
+
+CREATE TABLE customer
+(
+ id INT,
+ email VARCHAR(255),
+ phone VARCHAR(255),
+ PRIMARY KEY (id)
+);
+
+CREATE TABLE multicol_tbl
+(
+ cola VARCHAR(255),
+ colb VARCHAR(255),
+ colc VARCHAR(255),
+ name VARCHAR(255),
+ PRIMARY KEY (cola, colb)
+);
+
+CREATE TABLE mixed_tbl
+(
+ shard_key VARCHAR(255),
+ lkp_key VARCHAR(255),
+ PRIMARY KEY (shard_key)
+);
+
+CREATE TABLE pin_test
+(
+ id INT PRIMARY KEY
+);
+
+CREATE TABLE cfc_vindex_col
+(
+ c1 VARCHAR(255),
+ c2 VARCHAR(255),
+ PRIMARY KEY (c1)
+);
+
+CREATE TABLE unq_lkp_idx
+(
+ unq_key INT PRIMARY KEY,
+ keyspace_id VARCHAR(255)
+);
+
+CREATE TABLE t1
+(
+ c1 INT,
+ c2 INT,
+ c3 INT,
+ PRIMARY KEY (c1)
+);
+
+CREATE TABLE authoritative
+(
+ user_id bigint NOT NULL,
+ col1 VARCHAR(255),
+ col2 bigint,
+ PRIMARY KEY (user_id)
+) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.json b/go/vt/vtgate/planbuilder/testdata/select_cases.json
index 1099f62175f..eac13216380 100644
--- a/go/vt/vtgate/planbuilder/testdata/select_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/select_cases.json
@@ -65,6 +65,36 @@
]
}
},
+ {
+ "comment": "join on sharding column with limit - should be a simple scatter query and limit on top with non resolved columns",
+ "query": "select * from user u join user_metadata um on u.id = um.user_id where foo=41 limit 20",
+ "plan": {
+ "QueryType": "SELECT",
+ "Original": "select * from user u join user_metadata um on u.id = um.user_id where foo=41 limit 20",
+ "Instructions": {
+ "OperatorType": "Limit",
+ "Count": "20",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select * from `user` as u, user_metadata as um where 1 != 1",
+ "Query": "select * from `user` as u, user_metadata as um where foo = 41 and u.id = um.user_id limit 20",
+ "Table": "`user`, user_metadata"
+ }
+ ]
+ },
+ "TablesUsed": [
+ "user.user",
+ "user.user_metadata"
+ ]
+ },
+ "skip_e2e": true
+ },
{
"comment": "select with timeout directive sets QueryTimeout in the route",
"query": "select /*vt+ QUERY_TIMEOUT_MS=1000 */ * from user",
@@ -168,7 +198,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select with partial scatter directive",
@@ -373,7 +404,8 @@
{
"comment": "test table lookup failure for authoritative code path",
"query": "select a.* from authoritative",
- "plan": "Unknown table 'a'"
+ "plan": "Unknown table 'a'",
+ "skip_e2e": true
},
{
"comment": "select * from qualified authoritative table",
@@ -441,7 +473,8 @@
"user.authoritative",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "auto-resolve anonymous columns for simple route",
@@ -464,7 +497,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "json_arrayagg in single sharded query",
@@ -490,7 +524,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "json_objectagg in single sharded query",
@@ -516,17 +551,20 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "unsupported json aggregation expressions in scatter query",
"query": "select count(1) from user where cola = 'abc' group by n_id having json_arrayagg(a_id) = '[]'",
- "plan": "VT12001: unsupported: in scatter query: aggregation function 'json_arrayagg(a_id)'"
+ "plan": "VT12001: unsupported: in scatter query: aggregation function 'json_arrayagg(a_id)'",
+ "skip_e2e": true
},
{
"comment": "Cannot auto-resolve for cross-shard joins",
"query": "select col from user join user_extra",
- "plan": "Column 'col' in field list is ambiguous"
+ "plan": "Column 'col' in field list is ambiguous",
+ "skip_e2e": true
},
{
"comment": "Auto-resolve should work if unique vindex columns are referenced",
@@ -568,7 +606,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "database calls should be substituted",
@@ -590,7 +629,8 @@
"TablesUsed": [
"main.dual"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "last_insert_id for unsharded route",
@@ -612,7 +652,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select from dual on unqualified keyspace",
@@ -665,7 +706,8 @@
{
"comment": "prefixing dual with a keyspace should not work",
"query": "select 1 from user.dual",
- "plan": "table dual not found"
+ "plan": "table dual not found",
+ "skip_e2e": true
},
{
"comment": "RHS route referenced",
@@ -707,7 +749,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Both routes referenced",
@@ -749,7 +792,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Expression with single-route reference",
@@ -791,7 +835,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "subquery with an aggregation in order by that can be merged into a single route",
@@ -818,7 +863,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Jumbled references",
@@ -860,7 +906,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Comments",
@@ -902,7 +949,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "for update",
@@ -944,7 +992,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Field query should work for joins select bind vars",
@@ -989,7 +1038,8 @@
"main.unsharded",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Case preservation",
@@ -1031,12 +1081,14 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "syntax error",
"query": "the quick brown fox",
- "plan": "syntax error at position 4 near 'the'"
+ "plan": "syntax error at position 4 near 'the'",
+ "skip_e2e": true
},
{
"comment": "Hex number is not treated as a simple value",
@@ -1055,7 +1107,7 @@
"Query": "select * from `user` where id = 0x04",
"Table": "`user`",
"Values": [
- "'\u0004'"
+ "_binary'\u0004'"
],
"Vindex": "user_index"
},
@@ -1084,7 +1136,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Selection but make the planner explicitly use a vindex",
@@ -1135,12 +1188,14 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Vindex hint on a non-existing vindex",
"query": "select * from user use vindex (does_not_exist) where id = 1",
- "plan": "VT09021: Vindex 'does_not_exist' does not exist in table 'user.user'"
+ "plan": "VT09021: Vindex 'does_not_exist' does not exist in table 'user.user'",
+ "skip_e2e": true
},
{
"comment": "sharded limit offset",
@@ -1202,7 +1257,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Sharding Key Condition in Parenthesis",
@@ -1228,7 +1284,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Multiple parenthesized expressions",
@@ -1254,7 +1311,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Multiple parenthesized expressions",
@@ -1280,7 +1338,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Column Aliasing with Table.Column",
@@ -1358,7 +1417,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Column as boolean-ish",
@@ -1384,7 +1444,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "PK as fake boolean, and column as boolean-ish",
@@ -1410,7 +1471,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "top level subquery in select",
@@ -1455,7 +1517,8 @@
"main.unsharded",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "sub-expression subquery in select",
@@ -1500,7 +1563,8 @@
"main.unsharded",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select * from derived table expands specific columns",
@@ -1542,17 +1606,20 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "duplicate columns not allowed in derived table",
"query": "select * from (select user.id, user_extra.id from user join user_extra) as t",
- "plan": "Duplicate column name 'id'"
+ "plan": "Duplicate column name 'id'",
+ "skip_e2e": true
},
{
"comment": "non-existent symbol in cross-shard derived table",
"query": "select t.col from (select user.id from user join user_extra) as t",
- "plan": "column 't.col' not found"
+ "plan": "column 't.col' not found",
+ "skip_e2e": true
},
{
"comment": "union with the same target shard",
@@ -1579,7 +1646,8 @@
"user.music",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "union with the same target shard last_insert_id",
@@ -1606,7 +1674,8 @@
"user.music",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "unsharded union in derived table",
@@ -1764,7 +1833,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "routing table on music",
@@ -1786,7 +1856,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "testing SingleRow Projection",
@@ -1836,7 +1907,6 @@
"Variant": "Ordered",
"Aggregates": "sum(0) AS avg_col, sum_count(3) AS count(intcol)",
"GroupBy": "1 COLLATE latin1_swedish_ci, (2|4) COLLATE ",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Route",
@@ -1875,7 +1945,7 @@
"Sharded": false
},
"FieldQuery": "select 42 from dual where 1 != 1",
- "Query": "select 42 from dual where false",
+ "Query": "select 42 from dual where 0",
"Table": "dual"
},
"TablesUsed": [
@@ -1934,7 +2004,8 @@
"main.unsharded_a",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Complex expression in a subquery used in NOT IN clause of an aggregate query",
@@ -1987,7 +2058,8 @@
"main.unsharded_a",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "testing SingleRow Projection with arithmetics",
@@ -2190,12 +2262,14 @@
{
"comment": "sql_calc_found_rows in sub queries",
"query": "select * from music where user_id IN (select sql_calc_found_rows * from music limit 10)",
- "plan": "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'"
+ "plan": "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'",
+ "skip_e2e": true
},
{
"comment": "sql_calc_found_rows in derived table",
"query": "select sql_calc_found_rows * from (select sql_calc_found_rows * from music limit 10) t limit 1",
- "plan": "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'"
+ "plan": "Incorrect usage/placement of 'SQL_CALC_FOUND_ROWS'",
+ "skip_e2e": true
},
{
"comment": "select from unsharded keyspace into dumpfile",
@@ -2217,7 +2291,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select from unsharded keyspace into outfile",
@@ -2239,7 +2314,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select from unsharded keyspace into outfile s3",
@@ -2261,7 +2337,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "left join with a dual table on left - merge-able",
@@ -2472,17 +2549,20 @@
{
"comment": "Union after into outfile is incorrect",
"query": "select id from user into outfile 'out_file_name' union all select id from music",
- "plan": "syntax error at position 55 near 'union'"
+ "plan": "syntax error at position 55 near 'union'",
+ "skip_e2e": true
},
{
"comment": "Into outfile s3 in derived table is incorrect",
"query": "select id from (select id from user into outfile s3 'inner_outfile') as t2",
- "plan": "syntax error at position 41 near 'into'"
+ "plan": "syntax error at position 41 near 'into'",
+ "skip_e2e": true
},
{
"comment": "Into outfile s3 in derived table with union incorrect",
"query": "select id from (select id from user into outfile s3 'inner_outfile' union select 1) as t2",
- "plan": "syntax error at position 41 near 'into'"
+ "plan": "syntax error at position 41 near 'into'",
+ "skip_e2e": true
},
{
"comment": "select (select u.id from user as u where u.id = 1), a.id from user as a where a.id = 1",
@@ -2551,7 +2631,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "((((select 1))))",
@@ -2596,7 +2677,8 @@
"main.dual",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "subquery in select expression of derived table",
@@ -2666,7 +2748,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "ORDER BY subquery",
@@ -2736,7 +2819,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "plan test for a natural character set string",
@@ -2803,7 +2887,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Straight Join ensures specific ordering of joins",
@@ -2848,7 +2933,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Dual query should be handled on the vtgate even with a LIMIT",
@@ -2922,7 +3008,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Straight Join preserved in MySQL query",
@@ -2945,7 +3032,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "correlated subquery in exists clause",
@@ -3003,7 +3091,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "correlated subquery in exists clause with an order by",
@@ -3062,7 +3151,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "correlated subquery having dependencies on two tables",
@@ -3135,7 +3225,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "correlated subquery using a column twice",
@@ -3192,7 +3283,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "correlated subquery that is dependent on one side of a join, fully mergeable",
@@ -3243,7 +3335,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "union as a derived table",
@@ -3332,7 +3425,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "mergeable derived table with order by and limit",
@@ -3354,7 +3448,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "mergeable derived table with group by and limit",
@@ -3376,7 +3471,8 @@
"TablesUsed": [
"main.unsharded"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select user.id, trim(leading 'x' from user.name) from user",
@@ -3398,7 +3494,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "json utility functions",
@@ -3420,7 +3517,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "dual query with exists clause",
@@ -3518,7 +3616,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "yeah, it does not make sense, but it's valid",
@@ -3604,14 +3703,15 @@
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select a -> '$[4]', a ->> '$[3]' from `user` where 1 != 1",
- "Query": "select a -> '$[4]', a ->> '$[3]' from `user`",
+ "FieldQuery": "select json_extract(a, '$[4]'), json_unquote(json_extract(a, '$[3]')) from `user` where 1 != 1",
+ "Query": "select json_extract(a, '$[4]'), json_unquote(json_extract(a, '$[3]')) from `user`",
"Table": "`user`"
},
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "groupe by with non aggregated columns and table alias",
@@ -3633,7 +3733,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Functions that return JSON value attributes",
@@ -3838,7 +3939,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "gtid functions",
@@ -3906,7 +4008,8 @@
"user.user_extra",
"user.user_metadata"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Join across multiple tables, with conditions on different vindexes, but mergeable through join predicates",
@@ -3934,7 +4037,8 @@
"user.music_extra",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "SQL_CALC_FOUND_ROWS with vindex lookup",
@@ -4045,7 +4149,8 @@
"TablesUsed": [
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "`None` route being merged with another route via join predicate on Vindex columns",
@@ -4094,7 +4199,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Subquery with `IN` condition using columns with matching lookup vindexes",
@@ -4172,7 +4278,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Subquery with `IN` condition using columns with matching lookup vindexes",
@@ -4286,7 +4393,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Mergeable scatter subquery with `GROUP BY` on unique vindex column",
@@ -4308,7 +4416,8 @@
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Unmergeable scatter subquery with `GROUP BY` on-non vindex column",
@@ -4330,7 +4439,6 @@
"Variant": "Ordered",
"Aggregates": "any_value(0) AS id",
"GroupBy": "(1|2)",
- "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "Route",
@@ -4348,26 +4456,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Unmergeable scatter subquery with LIMIT",
@@ -4403,26 +4537,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Mergeable subquery with `MAX` aggregate and grouped by unique vindex",
@@ -4456,26 +4616,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Unmergeable subquery with `MAX` aggregate",
@@ -4496,7 +4682,6 @@
"OperatorType": "Aggregate",
"Variant": "Scalar",
"Aggregates": "max(0|1) AS max(music.id)",
- "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "Route",
@@ -4517,19 +4702,44 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -4570,19 +4780,44 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
@@ -4623,26 +4858,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Mergeable subquery with multiple levels of derived statements",
@@ -4734,26 +4995,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Unmergeable subquery with multiple levels of derived statements",
@@ -4789,26 +5076,52 @@
},
{
"InputName": "Outer",
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "IN",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select music.id from music where 1 != 1",
- "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
- "Table": "music",
"Values": [
"::__sq1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select music.id from music where 1 != 1",
+ "Query": "select music.id from music where :__sq_has_values and music.id in ::__vals",
+ "Table": "music"
+ }
+ ]
}
]
},
"TablesUsed": [
"user.music"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "`None` subquery as top level predicate - outer query changes from `Scatter` to `None` on merge",
@@ -5007,7 +5320,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "select user.a, t.b from user join (select id, count(*) b, req from user_extra group by req, id) as t on user.id = t.id",
@@ -5033,7 +5347,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS b",
"GroupBy": "(2|3), (0|4)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Route",
@@ -5072,7 +5385,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "SELECT music.id FROM (SELECT MAX(id) as maxt FROM music WHERE music.user_id = 5) other JOIN music ON other.maxt = music.id",
@@ -5195,7 +5509,8 @@
"main.dual",
"main.unsharded_a"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "subquery having join table on clause, using column reference of outer select table",
@@ -5244,7 +5559,8 @@
"main.unsharded",
"main.unsharded_a"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "ALL modifier on unsharded table works well",
@@ -5267,7 +5583,8 @@
"main.unsharded",
"main.unsharded_a"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "allow last_insert_id with argument",
@@ -5312,7 +5629,8 @@
"user.music_extra",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Query with non-plannable lookup vindex",
@@ -5338,7 +5656,8 @@
"TablesUsed": [
"user.user_metadata"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "join query with lookup and join on different vindex column",
@@ -5390,7 +5709,8 @@
"user.user",
"user.user_metadata"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "pick email as vindex lookup",
@@ -5400,7 +5720,7 @@
"Original": "select * from customer where email = 'a@mail.com'",
"Instructions": {
"OperatorType": "VindexLookup",
- "Variant": "Equal",
+ "Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
@@ -5485,7 +5805,8 @@
"TablesUsed": [
"user.customer"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "email vindex is costly than phone vindex - but phone vindex is backfiling hence ignored",
@@ -5495,7 +5816,7 @@
"Original": "select * from customer where email = 'a@mail.com' and phone = 123456",
"Instructions": {
"OperatorType": "VindexLookup",
- "Variant": "Equal",
+ "Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
@@ -5546,7 +5867,7 @@
"Original": "select * from customer where phone = 123456 and email = 'a@mail.com'",
"Instructions": {
"OperatorType": "VindexLookup",
- "Variant": "Equal",
+ "Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
@@ -5609,7 +5930,8 @@
"TablesUsed": [
"user.samecolvin"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "column with qualifier is correctly used",
@@ -5652,7 +5974,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Derived tables going to a single shard still need to expand derived table columns",
@@ -5697,7 +6020,8 @@
"main.unsharded",
"user.user"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "column name aliases in outer join queries",
@@ -5752,7 +6076,8 @@
"user.user",
"user.user_extra"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "Over clause works for unsharded tables",
@@ -5774,7 +6099,8 @@
"TablesUsed": [
"main.unsharded_a"
]
- }
+ },
+ "skip_e2e": true
},
{
"comment": "join with derived table with alias and join condition - merge into route",
diff --git a/go/vt/vtgate/planbuilder/testdata/show_cases.json b/go/vt/vtgate/planbuilder/testdata/show_cases.json
index f7fbce608bc..00bd37fe8f2 100644
--- a/go/vt/vtgate/planbuilder/testdata/show_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/show_cases.json
@@ -850,5 +850,28 @@
"TransactionID": "ks:-80:v24s7843sf78934l3"
}
}
+ },
+ {
+ "comment": "show unresolved transactions for default keyspace",
+ "query": "show unresolved transactions",
+ "plan": {
+ "QueryType": "SHOW",
+ "Original": "show unresolved transactions",
+ "Instructions": {
+ "OperatorType": "TransactionStatus"
+ }
+ }
+ },
+ {
+ "comment": "show unresolved transactions for explicitly provided keyspace",
+ "query": "show unresolved transactions for ks",
+ "plan": {
+ "QueryType": "SHOW",
+ "Original": "show unresolved transactions for ks",
+ "Instructions": {
+ "OperatorType": "TransactionStatus",
+ "Keyspace": "ks"
+ }
+ }
}
]
diff --git a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json
index 442f4d6b8b6..6b3f84d01d6 100644
--- a/go/vt/vtgate/planbuilder/testdata/tpch_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/tpch_cases.json
@@ -25,7 +25,6 @@
"Variant": "Ordered",
"Aggregates": "sum(2) AS sum_qty, sum(3) AS sum_base_price, sum(4) AS sum_disc_price, sum(5) AS sum_charge, sum(6) AS avg_qty, sum(7) AS avg_price, sum(8) AS avg_disc, sum_count_star(9) AS count_order, sum_count(10) AS count(l_quantity), sum_count(11) AS count(l_extendedprice), sum_count(12) AS count(l_discount)",
"GroupBy": "(0|13), (1|14)",
- "ResultColumns": 13,
"Inputs": [
{
"OperatorType": "Route",
@@ -74,7 +73,6 @@
"Variant": "Ordered",
"Aggregates": "sum(1) AS revenue",
"GroupBy": "(0|4), (2|5), (3|6)",
- "ResultColumns": 6,
"Inputs": [
{
"OperatorType": "Projection",
@@ -279,13 +277,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "1 DESC COLLATE utf8mb4_0900_ai_ci",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum(1) AS revenue",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Projection",
@@ -802,7 +800,6 @@
"Variant": "Ordered",
"Aggregates": "sum(1) AS sum(case when nation = 'BRAZIL' then volume else 0 end), sum(2) AS sum(volume)",
"GroupBy": "(0|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Projection",
@@ -832,7 +829,6 @@
"Variant": "Ordered",
"Aggregates": "sum(0) AS sum(case when nation = 'BRAZIL' then volume else 0 end), sum(1) AS sum(volume)",
"GroupBy": "(2|3)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Join",
@@ -1112,12 +1108,11 @@
"Variant": "Ordered",
"Aggregates": "sum(0) AS sum_profit",
"GroupBy": "(1|3), (2|4)",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "R:0,L:0,L:4,L:6,L:7",
+ "JoinColumnIndexes": "R:0,L:0,L:4,L:6,L:8",
"JoinVars": {
"l_discount": 2,
"l_extendedprice": 1,
@@ -1130,7 +1125,7 @@
{
"OperatorType": "Sort",
"Variant": "Memory",
- "OrderBy": "(0|6) ASC, (4|7) ASC",
+ "OrderBy": "(0|6) ASC, (4|8) ASC",
"Inputs": [
{
"OperatorType": "Join",
@@ -1349,13 +1344,13 @@
"OperatorType": "Sort",
"Variant": "Memory",
"OrderBy": "2 DESC COLLATE utf8mb4_0900_ai_ci",
+ "ResultColumns": 8,
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
"Aggregates": "sum(2) AS revenue",
"GroupBy": "(0|8), (1|9), (3|10), (6|11), (4|12), (5|13), (7|14)",
- "ResultColumns": 8,
"Inputs": [
{
"OperatorType": "Projection",
@@ -1660,6 +1655,7 @@
"InputName": "Outer",
"OperatorType": "Filter",
"Predicate": "sum(ps_supplycost * ps_availqty) > :__sq1",
+ "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Sort",
@@ -1671,7 +1667,6 @@
"Variant": "Ordered",
"Aggregates": "sum(1) AS value",
"GroupBy": "(0|2)",
- "ResultColumns": 2,
"Inputs": [
{
"OperatorType": "Projection",
@@ -1892,23 +1887,24 @@
"GroupBy": "0",
"Inputs": [
{
- "OperatorType": "SimpleProjection",
- "Columns": "1,3",
+ "OperatorType": "Projection",
+ "Expressions": [
+ ":1 as c_count",
+ "1 as 1"
+ ],
"Inputs": [
{
"OperatorType": "Aggregate",
"Variant": "Ordered",
- "Aggregates": "sum_count(1) AS count(o_orderkey), any_value(3)",
+ "Aggregates": "sum_count(1) AS count(o_orderkey)",
"GroupBy": "(0|2)",
- "ResultColumns": 4,
"Inputs": [
{
"OperatorType": "Projection",
"Expressions": [
":2 as c_custkey",
"count(*) * count(o_orderkey) as count(o_orderkey)",
- ":3 as weight_string(c_custkey)",
- ":4 as 1"
+ ":3 as weight_string(c_custkey)"
],
"Inputs": [
{
@@ -1919,7 +1915,7 @@
{
"OperatorType": "Join",
"Variant": "LeftJoin",
- "JoinColumnIndexes": "R:0,L:0,L:1,L:2,L:3",
+ "JoinColumnIndexes": "R:0,L:0,L:1,L:2",
"JoinVars": {
"c_custkey": 1
},
@@ -1932,9 +1928,9 @@
"Name": "main",
"Sharded": true
},
- "FieldQuery": "select count(*), c_custkey, weight_string(c_custkey), 1 from customer where 1 != 1 group by c_custkey, weight_string(c_custkey)",
+ "FieldQuery": "select count(*), c_custkey, weight_string(c_custkey) from customer where 1 != 1 group by c_custkey, weight_string(c_custkey)",
"OrderBy": "(1|2) ASC",
- "Query": "select count(*), c_custkey, weight_string(c_custkey), 1 from customer group by c_custkey, weight_string(c_custkey) order by c_custkey asc",
+ "Query": "select count(*), c_custkey, weight_string(c_custkey) from customer group by c_custkey, weight_string(c_custkey) order by c_custkey asc",
"Table": "customer"
},
{
@@ -1986,41 +1982,51 @@
"Aggregates": "any_value(0), sum(1) AS sum(case when p_type like 'PROMO%' then l_extendedprice * (1 - l_discount) else 0 end), sum(2) AS sum(l_extendedprice * (1 - l_discount))",
"Inputs": [
{
- "OperatorType": "Join",
- "Variant": "Join",
- "JoinColumnIndexes": "L:0,R:0,L:3",
- "JoinVars": {
- "l_discount": 2,
- "l_extendedprice": 1,
- "l_partkey": 4
- },
- "TableName": "lineitem_part",
+ "OperatorType": "Projection",
+ "Expressions": [
+ "100.00 as 100.00",
+ ":0 as case when p_type like 'PROMO%' then l_extendedprice * (1 - l_discount) else 0 end",
+ ":1 as l_extendedprice * (1 - l_discount)"
+ ],
"Inputs": [
{
- "OperatorType": "Route",
- "Variant": "Scatter",
- "Keyspace": {
- "Name": "main",
- "Sharded": true
- },
- "FieldQuery": "select 100.00, l_extendedprice, l_discount, l_extendedprice * (1 - l_discount), l_partkey from lineitem where 1 != 1",
- "Query": "select 100.00, l_extendedprice, l_discount, l_extendedprice * (1 - l_discount), l_partkey from lineitem where l_shipdate >= date('1995-09-01') and l_shipdate < date('1995-09-01') + interval '1' month",
- "Table": "lineitem"
- },
- {
- "OperatorType": "Route",
- "Variant": "EqualUnique",
- "Keyspace": {
- "Name": "main",
- "Sharded": true
+ "OperatorType": "Join",
+ "Variant": "Join",
+ "JoinColumnIndexes": "R:0,L:2",
+ "JoinVars": {
+ "l_discount": 1,
+ "l_extendedprice": 0,
+ "l_partkey": 3
},
- "FieldQuery": "select case when p_type like 'PROMO%' then :l_extendedprice * (1 - :l_discount) else 0 end from part where 1 != 1",
- "Query": "select case when p_type like 'PROMO%' then :l_extendedprice * (1 - :l_discount) else 0 end from part where p_partkey = :l_partkey",
- "Table": "part",
- "Values": [
- ":l_partkey"
- ],
- "Vindex": "hash"
+ "TableName": "lineitem_part",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "Scatter",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": true
+ },
+ "FieldQuery": "select l_extendedprice, l_discount, l_extendedprice * (1 - l_discount), l_partkey from lineitem where 1 != 1",
+ "Query": "select l_extendedprice, l_discount, l_extendedprice * (1 - l_discount), l_partkey from lineitem where l_shipdate >= date('1995-09-01') and l_shipdate < date('1995-09-01') + interval '1' month",
+ "Table": "lineitem"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "EqualUnique",
+ "Keyspace": {
+ "Name": "main",
+ "Sharded": true
+ },
+ "FieldQuery": "select case when p_type like 'PROMO%' then :l_extendedprice * (1 - :l_discount) else 0 end from part where 1 != 1",
+ "Query": "select case when p_type like 'PROMO%' then :l_extendedprice * (1 - :l_discount) else 0 end from part where p_partkey = :l_partkey",
+ "Table": "part",
+ "Values": [
+ ":l_partkey"
+ ],
+ "Vindex": "hash"
+ }
+ ]
}
]
}
@@ -2052,7 +2058,6 @@
"OperatorType": "Aggregate",
"Variant": "Scalar",
"Aggregates": "max(0|1) AS max(total_revenue)",
- "ResultColumns": 1,
"Inputs": [
{
"OperatorType": "Route",
@@ -2106,7 +2111,6 @@
"Variant": "Ordered",
"Aggregates": "count_distinct(3|7) AS supplier_cnt",
"GroupBy": "(0|4), (1|5), (2|6)",
- "ResultColumns": 7,
"Inputs": [
{
"OperatorType": "Sort",
@@ -2116,9 +2120,9 @@
{
"OperatorType": "Join",
"Variant": "Join",
- "JoinColumnIndexes": "R:0,R:1,R:2,L:0,R:3,R:4,R:5,L:1",
+ "JoinColumnIndexes": "R:0,R:1,R:2,L:0,R:3,R:4,R:5,L:2",
"JoinVars": {
- "ps_partkey": 2,
+ "ps_partkey": 1,
"ps_suppkey": 0
},
"TableName": "partsupp_part",
@@ -2151,8 +2155,8 @@
"Name": "main",
"Sharded": true
},
- "FieldQuery": "select ps_suppkey, weight_string(ps_suppkey), ps_partkey from partsupp where 1 != 1",
- "Query": "select ps_suppkey, weight_string(ps_suppkey), ps_partkey from partsupp where not :__sq_has_values or ps_suppkey not in ::__sq1",
+ "FieldQuery": "select ps_suppkey, ps_partkey, weight_string(ps_suppkey) from partsupp where 1 != 1",
+ "Query": "select ps_suppkey, ps_partkey, weight_string(ps_suppkey) from partsupp where not :__sq_has_values or ps_suppkey not in ::__sq1",
"Table": "partsupp"
}
]
@@ -2444,7 +2448,6 @@
"Variant": "Ordered",
"Aggregates": "sum_count_star(1) AS numwait",
"GroupBy": "(0|2)",
- "ResultColumns": 3,
"Inputs": [
{
"OperatorType": "Projection",
diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.json b/go/vt/vtgate/planbuilder/testdata/union_cases.json
index 7feabb0a698..2927c1c6093 100644
--- a/go/vt/vtgate/planbuilder/testdata/union_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/union_cases.json
@@ -447,34 +447,84 @@
"OperatorType": "Concatenate",
"Inputs": [
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select 1 from music where 1 != 1",
- "Query": "select distinct 1 from music where id = 1",
- "Table": "music",
"Values": [
"1"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select 1 from music where 1 != 1",
+ "Query": "select distinct 1 from music where id = 1",
+ "Table": "music"
+ }
+ ]
},
{
- "OperatorType": "Route",
+ "OperatorType": "VindexLookup",
"Variant": "EqualUnique",
"Keyspace": {
"Name": "user",
"Sharded": true
},
- "FieldQuery": "select 1 from music where 1 != 1",
- "Query": "select distinct 1 from music where id = 2",
- "Table": "music",
"Values": [
"2"
],
- "Vindex": "music_user_map"
+ "Vindex": "music_user_map",
+ "Inputs": [
+ {
+ "OperatorType": "Route",
+ "Variant": "IN",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select `name`, keyspace_id from name_user_vdx where 1 != 1",
+ "Query": "select `name`, keyspace_id from name_user_vdx where `name` in ::__vals",
+ "Table": "name_user_vdx",
+ "Values": [
+ "::name"
+ ],
+ "Vindex": "user_index"
+ },
+ {
+ "OperatorType": "Route",
+ "Variant": "ByDestination",
+ "Keyspace": {
+ "Name": "user",
+ "Sharded": true
+ },
+ "FieldQuery": "select 1 from music where 1 != 1",
+ "Query": "select distinct 1 from music where id = 2",
+ "Table": "music"
+ }
+ ]
}
]
}
diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json
index 0e230b3e44d..9241cec595c 100644
--- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json
@@ -329,11 +329,6 @@
"query": "with user as (select aa from user where user.id=1) select ref.col from ref join user",
"plan": "VT12001: unsupported: do not support CTE that use the CTE alias inside the CTE query"
},
- {
- "comment": "Recursive WITH",
- "query": "WITH RECURSIVE cte (n) AS (SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT * FROM cte",
- "plan": "VT12001: unsupported: recursive common table expression"
- },
{
"comment": "Alias cannot clash with base tables",
"query": "WITH user AS (SELECT col FROM user) SELECT * FROM user",
diff --git a/go/vt/vtgate/planbuilder/testdata/vexplain_cases.json b/go/vt/vtgate/planbuilder/testdata/vexplain_cases.json
index 630e59f3526..cafdc2851af 100644
--- a/go/vt/vtgate/planbuilder/testdata/vexplain_cases.json
+++ b/go/vt/vtgate/planbuilder/testdata/vexplain_cases.json
@@ -69,5 +69,20 @@
"user.user"
]
}
+ },
+ {
+ "comment": "vexplain keys",
+ "query": "vexplain keys select * from user",
+ "plan": {
+ "QueryType": "EXPLAIN",
+ "Original": "vexplain keys select * from user",
+ "Instructions": {
+ "OperatorType": "Rows",
+ "Fields": {
+ "ColumnUsage": "VARCHAR"
+ },
+ "RowCount": 1
+ }
+ }
}
]
diff --git a/go/vt/vtgate/planbuilder/testdata/vschemas/mirror_schema.json b/go/vt/vtgate/planbuilder/testdata/vschemas/mirror_schema.json
new file mode 100644
index 00000000000..4feaa09c126
--- /dev/null
+++ b/go/vt/vtgate/planbuilder/testdata/vschemas/mirror_schema.json
@@ -0,0 +1,103 @@
+{
+ "mirror_rules": {
+ "rules": [
+ {
+ "from_table": "unsharded_src1.t1",
+ "to_table": "unsharded_dst1.t1",
+ "percent": 1
+ },
+ {
+ "from_table": "unsharded_src1.t2",
+ "to_table": "sharded_dst1.t1",
+ "percent": 2
+ },
+ {
+ "from_table": "unsharded_src2.t1",
+ "to_table": "unsharded_src2.t1",
+ "percent": 3
+ },
+ {
+ "from_table": "unsharded_src2.t2",
+ "to_table": "unsharded_dst2.t2",
+ "percent": 4
+ },
+ {
+ "from_table": "unsharded_dst2.t2",
+ "to_table": "unsharded_dst3.t2",
+ "percent": 5
+ },
+ {
+ "from_table": "unsharded_src3.t1",
+ "to_table": "unsharded_dst4.t1",
+ "percent": 6
+ },
+ {
+ "from_table": "unsharded_dst4.t2",
+ "to_table": "unsharded_src3.t2",
+ "percent": 7
+ },
+ {
+ "from_table": "sharded_src1.t1",
+ "to_table": "sharded_dst1.t1",
+ "percent": 8
+ },
+ {
+ "from_table": "unsharded_src1.t3",
+ "to_table": "unsharded_dst1.t2",
+ "percent": 0
+ }
+ ]
+ },
+ "keyspaces": {
+ "main": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_src1": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_src2": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_src3": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_dst1": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_dst2": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_dst3": {
+ "sharded": false,
+ "tables": {}
+ },
+ "unsharded_dst4": {
+ "sharded": false,
+ "tables": {}
+ },
+ "sharded_dst1": {
+ "sharded": true,
+ "vindexes": {
+ "xxhash": {
+ "type": "xxhash"
+ }
+ },
+ "tables": {
+ "t1": {
+ "columnVindexes": [
+ {
+ "column": "id",
+ "name": "xxhash"
+ }
+ ]
+ }
+ }
+ }
+ }
+}
diff --git a/go/vt/vtgate/planbuilder/testdata/vschemas/schema.json b/go/vt/vtgate/planbuilder/testdata/vschemas/schema.json
index 4fe275f2398..a5de9d3697e 100644
--- a/go/vt/vtgate/planbuilder/testdata/vschemas/schema.json
+++ b/go/vt/vtgate/planbuilder/testdata/vschemas/schema.json
@@ -58,34 +58,52 @@
"sharded": true,
"vindexes": {
"user_index": {
- "type": "hash_test",
+ "type": "hash",
"owner": "user"
},
"kid_index": {
- "type": "hash_test",
+ "type": "hash",
"owner": "multicolvin"
},
+ "hash": {
+ "type": "hash"
+ },
"user_md5_index": {
"type": "unicode_loose_md5"
},
"music_user_map": {
- "type": "lookup_test",
- "owner": "music"
+ "type": "lookup_unique",
+ "owner": "music",
+ "params": {
+ "table": "name_user_vdx",
+ "from": "name",
+ "to": "keyspace_id"
+ }
},
"cola_map": {
- "type": "lookup_test",
- "owner": "multicolvin"
+ "type": "lookup_unique",
+ "owner": "multicolvin",
+ "params": {
+ "table": "cola_map",
+ "from": "cola",
+ "to": "keyspace_id"
+ }
},
"colb_colc_map": {
- "type": "lookup_test",
- "owner": "multicolvin"
+ "type": "lookup_unique",
+ "owner": "multicolvin",
+ "params": {
+ "table": "colb_colc_map",
+ "from": "colb,colc",
+ "to": "keyspace_id"
+ }
},
"cola_kid_map": {
- "type": "lookup_test",
+ "type": "lookup_unique",
"owner": "overlap_vindex"
},
"name_user_map": {
- "type": "name_lkp_test",
+ "type": "lookup",
"owner": "user",
"params": {
"table": "name_user_vdx",
@@ -94,42 +112,56 @@
}
},
"email_user_map": {
- "type": "lookup_test",
+ "type": "lookup_unique",
"owner": "user_metadata"
},
"address_user_map": {
- "type": "lookup_test",
+ "type": "lookup_unique",
"owner": "user_metadata"
},
"costly_map": {
- "type": "costly",
- "owner": "user"
+ "type": "lookup_cost",
+ "owner": "user",
+ "params": {
+ "table": "costly_map",
+ "from": "costly",
+ "to": "keyspace_id",
+ "cost": "100"
+ }
},
"hash_dup": {
- "type": "hash_test",
+ "type": "hash",
"owner": "user"
},
"vindex1": {
- "type": "hash_test",
+ "type": "hash",
"owner": "samecolvin"
},
"vindex2": {
- "type": "lookup_test",
+ "type": "lookup_unique",
"owner": "samecolvin"
},
"cfc": {
"type": "cfc"
},
"multicolIdx": {
- "type": "multiCol_test"
+ "type": "multicol",
+ "params": {
+ "column_count": "2"
+ }
},
"colc_map": {
- "type": "lookup_test",
+ "type": "lookup_unique",
"owner": "multicol_tbl"
},
"name_muticoltbl_map": {
- "type": "name_lkp_test",
- "owner": "multicol_tbl"
+ "type": "lookup",
+ "owner": "multicol_tbl",
+ "params": {
+ "table": "name_user_vdx",
+ "from": "name",
+ "to": "keyspace_id"
+ }
},
"non_planable_user_map": {
"type": "lookup_unicodeloosemd5_hash",
@@ -141,7 +173,7 @@
"owner": "user_metadata"
},
"lkp_shard_map": {
- "type": "name_lkp_test",
+ "type": "lookup_unique",
"owner": "mixed_tbl",
"params": {
"table": "lkp_shard_vdx",
@@ -153,18 +185,18 @@
"type": "xxhash"
},
"unq_lkp_bf_vdx": {
- "type": "unq_lkp_test",
+ "type": "lookup_unique",
"owner": "customer",
"params": {
"table": "unq_lkp_idx",
- "from": " ",
+ "from": "unq_key",
"to": "keyspace_id",
"cost": "100",
"write_only": "true"
}
},
"unq_lkp_vdx": {
- "type": "unq_lkp_test",
+ "type": "lookup_unique",
"owner": "customer",
"params": {
"table": "unq_lkp_idx",
@@ -174,11 +206,11 @@
}
},
"lkp_bf_vdx": {
- "type": "name_lkp_test",
+ "type": "lookup_unique",
"owner": "customer",
"params": {
"table": "lkp_shard_vdx",
- "from": " ",
+ "from": "unq_key",
"to": "keyspace_id",
"write_only": "true"
}
@@ -352,6 +384,22 @@
}
]
},
+ "cola_map": {
+ "column_vindexes": [
+ {
+ "column": "cola",
+ "name": "hash"
+ }
+ ]
+ },
+ "colb_colc_map": {
+ "column_vindexes": [
+ {
+ "column": "colb",
+ "name": "hash"
+ }
+ ]
+ },
"overlap_vindex": {
"column_vindexes": [
{
@@ -462,6 +510,14 @@
}
]
},
+ "costly_map": {
+ "column_vindexes": [
+ {
+ "column": "name",
+ "name": "user_md5_index"
+ }
+ ]
+ },
"mixed_tbl": {
"column_vindexes": [
{
@@ -641,7 +697,10 @@
"type": "hash_test"
},
"multicolIdx": {
- "type": "multiCol_test"
+ "type": "multicol",
+ "params": {
+ "column_count": "3"
+ }
}
},
"tables": {
diff --git a/go/vt/vtgate/planbuilder/vexplain.go b/go/vt/vtgate/planbuilder/vexplain.go
index 21a35f02967..7aed1e48884 100644
--- a/go/vt/vtgate/planbuilder/vexplain.go
+++ b/go/vt/vtgate/planbuilder/vexplain.go
@@ -26,18 +26,29 @@ import (
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/dynamicconfig"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/planbuilder/operators"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
-func buildVExplainPlan(ctx context.Context, vexplainStmt *sqlparser.VExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
+func buildVExplainPlan(
+ ctx context.Context,
+ vexplainStmt *sqlparser.VExplainStmt,
+ reservedVars *sqlparser.ReservedVars,
+ vschema plancontext.VSchema,
+ cfg dynamicconfig.DDL,
+) (*planResult, error) {
switch vexplainStmt.Type {
case sqlparser.QueriesVExplainType, sqlparser.AllVExplainType:
- return buildVExplainLoggingPlan(ctx, vexplainStmt, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ return buildVExplainLoggingPlan(ctx, vexplainStmt, reservedVars, vschema, cfg)
case sqlparser.PlanVExplainType:
- return buildVExplainVtgatePlan(ctx, vexplainStmt.Statement, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+ return buildVExplainVtgatePlan(ctx, vexplainStmt.Statement, reservedVars, vschema, cfg)
+ case sqlparser.TraceVExplainType:
+ return buildVExplainTracePlan(ctx, vexplainStmt.Statement, reservedVars, vschema, cfg)
+ case sqlparser.KeysVExplainType:
+ return buildVExplainKeysPlan(vexplainStmt.Statement, vschema)
}
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unexpected vtexplain type: %s", vexplainStmt.Type.ToString())
}
@@ -82,29 +93,40 @@ func explainTabPlan(explain *sqlparser.ExplainTab, vschema plancontext.VSchema)
}, singleTable(keyspace.Name, explain.Table.Name.String())), nil
}
-func buildVExplainVtgatePlan(ctx context.Context, explainStatement sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
- innerInstruction, err := createInstructionFor(ctx, sqlparser.String(explainStatement), explainStatement, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+func buildVExplainVtgatePlan(ctx context.Context, explainStatement sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ innerInstruction, err := createInstructionFor(ctx, sqlparser.String(explainStatement), explainStatement, reservedVars, vschema, cfg)
if err != nil {
return nil, err
}
- description := engine.PrimitiveToPlanDescription(innerInstruction.primitive)
- output, err := json.MarshalIndent(description, "", "\t")
+
+ return getJsonResultPlan(
+ engine.PrimitiveToPlanDescription(innerInstruction.primitive, nil),
+ "JSON",
+ )
+}
+
+// getJsonResultPlan marshals the given struct into a JSON string and returns it as a planResult.
+func getJsonResultPlan(v any, colName string) (*planResult, error) {
+ output, err := json.MarshalIndent(v, "", "\t")
if err != nil {
return nil, err
}
- fields := []*querypb.Field{
- {Name: "JSON", Type: querypb.Type_VARCHAR},
- }
- rows := []sqltypes.Row{
- {
- sqltypes.NewVarChar(string(output)),
- },
- }
+ fields := []*querypb.Field{{Name: colName, Type: querypb.Type_VARCHAR}}
+ rows := []sqltypes.Row{{sqltypes.NewVarChar(string(output))}}
return newPlanResult(engine.NewRowsPrimitive(rows, fields)), nil
}
-func buildVExplainLoggingPlan(ctx context.Context, explain *sqlparser.VExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, enableOnlineDDL, enableDirectDDL bool) (*planResult, error) {
- input, err := createInstructionFor(ctx, sqlparser.String(explain.Statement), explain.Statement, reservedVars, vschema, enableOnlineDDL, enableDirectDDL)
+func buildVExplainKeysPlan(statement sqlparser.Statement, vschema plancontext.VSchema) (*planResult, error) {
+ ctx, err := plancontext.CreatePlanningContext(statement, sqlparser.NewReservedVars("", sqlparser.BindVars{}), vschema, querypb.ExecuteOptions_Gen4)
+ if err != nil {
+ return nil, err
+ }
+ result := operators.GetVExplainKeys(ctx, statement)
+ return getJsonResultPlan(result, "ColumnUsage")
+}
+
+func buildVExplainLoggingPlan(ctx context.Context, explain *sqlparser.VExplainStmt, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ input, err := createInstructionFor(ctx, sqlparser.String(explain.Statement), explain.Statement, reservedVars, vschema, cfg)
if err != nil {
return nil, err
}
@@ -166,3 +188,17 @@ func explainPlan(explain *sqlparser.ExplainStmt, reservedVars *sqlparser.Reserve
SingleShardOnly: true,
}, tables...), nil
}
+
+func buildVExplainTracePlan(ctx context.Context, explainStatement sqlparser.Statement, reservedVars *sqlparser.ReservedVars, vschema plancontext.VSchema, cfg dynamicconfig.DDL) (*planResult, error) {
+ innerInstruction, err := createInstructionFor(ctx, sqlparser.String(explainStatement), explainStatement, reservedVars, vschema, cfg)
+ if err != nil {
+ return nil, err
+ }
+
+ // We'll set the trace engine as the root primitive
+ innerInstruction.primitive = &engine.VExplain{
+ Input: innerInstruction.primitive,
+ Type: sqlparser.TraceVExplainType,
+ }
+ return innerInstruction, nil
+}
diff --git a/go/vt/vtgate/plugin_mysql_server.go b/go/vt/vtgate/plugin_mysql_server.go
index 4004ae24566..dec7794c9bd 100644
--- a/go/vt/vtgate/plugin_mysql_server.go
+++ b/go/vt/vtgate/plugin_mysql_server.go
@@ -399,7 +399,7 @@ func (vh *vtgateHandler) KillConnection(ctx context.Context, connectionID uint32
c, exists := vh.connections[connectionID]
if !exists {
- return sqlerror.NewSQLError(sqlerror.ERNoSuchThread, sqlerror.SSUnknownSQLState, "Unknown thread id: %d", connectionID)
+ return sqlerror.NewSQLErrorf(sqlerror.ERNoSuchThread, sqlerror.SSUnknownSQLState, "Unknown thread id: %d", connectionID)
}
// First, we mark the connection for close, so that even when the context is cancelled, while returning the response back to client,
@@ -417,7 +417,7 @@ func (vh *vtgateHandler) KillQuery(connectionID uint32) error {
defer vh.mu.Unlock()
c, exists := vh.connections[connectionID]
if !exists {
- return sqlerror.NewSQLError(sqlerror.ERNoSuchThread, sqlerror.SSUnknownSQLState, "Unknown thread id: %d", connectionID)
+ return sqlerror.NewSQLErrorf(sqlerror.ERNoSuchThread, sqlerror.SSUnknownSQLState, "Unknown thread id: %d", connectionID)
}
c.CancelCtx()
return nil
diff --git a/go/vt/vtgate/querylogz.go b/go/vt/vtgate/querylogz.go
index 7c72e950d4a..05d301f28be 100644
--- a/go/vt/vtgate/querylogz.go
+++ b/go/vt/vtgate/querylogz.go
@@ -20,15 +20,15 @@ import (
"net/http"
"strconv"
"strings"
- "text/template"
"time"
- "vitess.io/vitess/go/vt/vtgate/logstats"
+ "github.com/google/safehtml/template"
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logz"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtgate/logstats"
)
var (
diff --git a/go/vt/vtgate/querylogz_test.go b/go/vt/vtgate/querylogz_test.go
index 3cecb983b3f..9236b2ac840 100644
--- a/go/vt/vtgate/querylogz_test.go
+++ b/go/vt/vtgate/querylogz_test.go
@@ -35,7 +35,7 @@ import (
func TestQuerylogzHandlerFormatting(t *testing.T) {
req, _ := http.NewRequest("GET", "/querylogz?timeout=10&limit=1", nil)
- logStats := logstats.NewLogStats(context.Background(), "Execute", "select name from test_table limit 1000", "suuid", nil)
+ logStats := logstats.NewLogStats(context.Background(), "Execute", "select name, 'inject ' from test_table limit 1000", "suuid", nil)
logStats.StmtType = "select"
logStats.RowsAffected = 1000
logStats.ShardQueries = 1
@@ -64,7 +64,7 @@ func TestQuerylogzHandlerFormatting(t *testing.T) {
`0.002 `,
`0.003 `,
`select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`1000 `,
` `,
@@ -94,7 +94,7 @@ func TestQuerylogzHandlerFormatting(t *testing.T) {
`0.002 `,
`0.003 `,
`select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`1000 `,
` `,
@@ -124,7 +124,7 @@ func TestQuerylogzHandlerFormatting(t *testing.T) {
`0.002 `,
`0.003 `,
`select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`1000 `,
` `,
diff --git a/go/vt/vtgate/safe_session_test.go b/go/vt/vtgate/safe_session_test.go
deleted file mode 100644
index 21bb2d6697a..00000000000
--- a/go/vt/vtgate/safe_session_test.go
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
-Copyright 2020 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 vtgate
-
-import (
- "reflect"
- "testing"
- "time"
-
- "github.com/stretchr/testify/assert"
- "github.com/stretchr/testify/require"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
-)
-
-func TestFailToMultiShardWhenSetToSingleDb(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{
- InTransaction: true, TransactionMode: vtgatepb.TransactionMode_SINGLE,
- })
-
- sess0 := &vtgatepb.Session_ShardSession{
- Target: &querypb.Target{Keyspace: "keyspace", Shard: "0"},
- TabletAlias: &topodatapb.TabletAlias{Cell: "cell", Uid: 0},
- TransactionId: 1,
- }
- sess1 := &vtgatepb.Session_ShardSession{
- Target: &querypb.Target{Keyspace: "keyspace", Shard: "1"},
- TabletAlias: &topodatapb.TabletAlias{Cell: "cell", Uid: 1},
- TransactionId: 1,
- }
-
- err := session.AppendOrUpdate(sess0, vtgatepb.TransactionMode_SINGLE)
- require.NoError(t, err)
- err = session.AppendOrUpdate(sess1, vtgatepb.TransactionMode_SINGLE)
- require.Error(t, err)
-}
-
-func TestPrequeries(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{
- SystemVariables: map[string]string{
- "s1": "'apa'",
- "s2": "42",
- },
- })
-
- want := []string{"set s1 = 'apa', s2 = 42"}
- preQueries := session.SetPreQueries()
-
- if !reflect.DeepEqual(want, preQueries) {
- t.Errorf("got %v but wanted %v", preQueries, want)
- }
-}
-
-func TestTimeZone(t *testing.T) {
- testCases := []struct {
- tz string
- want string
- }{
- {
- tz: "Europe/Amsterdam",
- want: "Europe/Amsterdam",
- },
- {
- tz: "+02:00",
- want: "UTC+02:00",
- },
- {
- tz: "foo",
- want: (*time.Location)(nil).String(),
- },
- }
-
- for _, tc := range testCases {
- t.Run(tc.tz, func(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{
- SystemVariables: map[string]string{
- "time_zone": tc.tz,
- },
- })
-
- assert.Equal(t, tc.want, session.TimeZone().String())
- })
- }
-}
diff --git a/go/vt/vtgate/sandbox_test.go b/go/vt/vtgate/sandbox_test.go
index 70b96a63126..fa5ffbbffd8 100644
--- a/go/vt/vtgate/sandbox_test.go
+++ b/go/vt/vtgate/sandbox_test.go
@@ -48,6 +48,7 @@ const (
func init() {
ksToSandbox = make(map[string]*sandbox)
+ sandboxMirrorRules = `{"rules":[]}`
createSandbox(KsTestSharded)
createSandbox(KsTestUnsharded)
createSandbox(KsTestBadVSchema)
@@ -57,6 +58,7 @@ func init() {
var sandboxMu sync.Mutex
var ksToSandbox map[string]*sandbox
+var sandboxMirrorRules string
func createSandbox(keyspace string) *sandbox {
sandboxMu.Lock()
@@ -86,9 +88,20 @@ func getSandboxSrvVSchema() *vschemapb.SrvVSchema {
}
result.Keyspaces[keyspace] = &vs
}
+ var mrs vschemapb.MirrorRules
+ if err := json2.Unmarshal([]byte(sandboxMirrorRules), &mrs); err != nil {
+ panic(err)
+ }
+ result.MirrorRules = &mrs
return result
}
+func setSandboxMirrorRules(mirrorRules string) {
+ sandboxMu.Lock()
+ defer sandboxMu.Unlock()
+ sandboxMirrorRules = mirrorRules
+}
+
type sandbox struct {
// Use sandmu to access the variables below
sandmu sync.Mutex
diff --git a/go/vt/vtgate/scatter_conn.go b/go/vt/vtgate/scatter_conn.go
index 8b571f7b67d..6e2cf9ad8ba 100644
--- a/go/vt/vtgate/scatter_conn.go
+++ b/go/vt/vtgate/scatter_conn.go
@@ -24,26 +24,25 @@ import (
"sync/atomic"
"time"
- "vitess.io/vitess/go/mysql/sqlerror"
- "vitess.io/vitess/go/vt/sqlparser"
-
"google.golang.org/protobuf/proto"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/vt/concurrency"
"vitess.io/vitess/go/vt/discovery"
"vitess.io/vitess/go/vt/log"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/srvtopo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vttablet/queryservice"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
// ScatterConn is used for executing queries across
@@ -72,6 +71,12 @@ type shardActionFunc func(rs *srvtopo.ResolvedShard, i int) error
// the results and errors for the caller.
type shardActionTransactionFunc func(rs *srvtopo.ResolvedShard, i int, shardActionInfo *shardActionInfo) (*shardActionInfo, error)
+type (
+ nullResultsObserver struct{}
+)
+
+func (nullResultsObserver) Observe(*sqltypes.Result) {}
+
// NewScatterConn creates a new ScatterConn.
func NewScatterConn(statsName string, txConn *TxConn, gw *TabletGateway) *ScatterConn {
// this only works with TabletGateway
@@ -99,7 +104,7 @@ func (stc *ScatterConn) startAction(name string, target *querypb.Target) (time.T
return startTime, statsKey
}
-func (stc *ScatterConn) endAction(startTime time.Time, allErrors *concurrency.AllErrorRecorder, statsKey []string, err *error, session *SafeSession) {
+func (stc *ScatterConn) endAction(startTime time.Time, allErrors *concurrency.AllErrorRecorder, statsKey []string, err *error, session *econtext.SafeSession) {
if *err != nil {
allErrors.RecordError(*err)
// Don't increment the error counter for duplicate
@@ -143,9 +148,10 @@ func (stc *ScatterConn) ExecuteMultiShard(
primitive engine.Primitive,
rss []*srvtopo.ResolvedShard,
queries []*querypb.BoundQuery,
- session *SafeSession,
+ session *econtext.SafeSession,
autocommit bool,
ignoreMaxMemoryRows bool,
+ resultsObserver econtext.ResultsObserver,
) (qr *sqltypes.Result, errs []error) {
if len(rss) != len(queries) {
@@ -156,7 +162,7 @@ func (stc *ScatterConn) ExecuteMultiShard(
var mu sync.Mutex
qr = new(sqltypes.Result)
- if session.InLockSession() && session.TriggerLockHeartBeat() {
+ if session.InLockSession() && triggerLockHeartBeat(session) {
go stc.runLockQuery(ctx, session)
}
@@ -214,6 +220,7 @@ func (stc *ScatterConn) ExecuteMultiShard(
retryRequest(func() {
// we seem to have lost our connection. it was a reserved connection, let's try to recreate it
info.actionNeeded = reserve
+ info.ignoreOldSession = true
var state queryservice.ReservedState
state, innerqr, err = qs.ReserveExecute(ctx, rs.Target, session.SetPreQueries(), queries[i].Sql, queries[i].BindVariables, 0 /*transactionId*/, opts)
reservedID = state.ReservedID
@@ -229,6 +236,7 @@ func (stc *ScatterConn) ExecuteMultiShard(
retryRequest(func() {
// we seem to have lost our connection. it was a reserved connection, let's try to recreate it
info.actionNeeded = reserveBegin
+ info.ignoreOldSession = true
var state queryservice.ReservedTransactionState
state, innerqr, err = qs.ReserveBeginExecute(ctx, rs.Target, session.SetPreQueries(), session.SavePoints(), queries[i].Sql, queries[i].BindVariables, opts)
transactionID = state.TransactionID
@@ -250,16 +258,20 @@ func (stc *ScatterConn) ExecuteMultiShard(
default:
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unexpected actionNeeded on query execution: %v", info.actionNeeded)
}
- session.logging.log(primitive, rs.Target, rs.Gateway, queries[i].Sql, info.actionNeeded == begin || info.actionNeeded == reserveBegin, queries[i].BindVariables)
+ session.Log(primitive, rs.Target, rs.Gateway, queries[i].Sql, info.actionNeeded == begin || info.actionNeeded == reserveBegin, queries[i].BindVariables)
// We need to new shard info irrespective of the error.
- newInfo := info.updateTransactionAndReservedID(transactionID, reservedID, alias)
+ newInfo := info.updateTransactionAndReservedID(transactionID, reservedID, alias, innerqr)
if err != nil {
return newInfo, err
}
mu.Lock()
defer mu.Unlock()
+ if innerqr != nil {
+ resultsObserver.Observe(innerqr)
+ }
+
// Don't append more rows if row count is exceeded.
if ignoreMaxMemoryRows || len(qr.Rows) <= maxMemoryRows {
qr.AppendResult(innerqr)
@@ -275,7 +287,13 @@ func (stc *ScatterConn) ExecuteMultiShard(
return qr, allErrors.GetErrors()
}
-func (stc *ScatterConn) runLockQuery(ctx context.Context, session *SafeSession) {
+func triggerLockHeartBeat(session *econtext.SafeSession) bool {
+ now := time.Now().Unix()
+ lastHeartbeat := session.GetLockHeartbeat()
+ return now-lastHeartbeat >= int64(lockHeartbeatTime.Seconds())
+}
+
+func (stc *ScatterConn) runLockQuery(ctx context.Context, session *econtext.SafeSession) {
rs := &srvtopo.ResolvedShard{Target: session.LockSession.Target, Gateway: stc.gateway}
query := &querypb.BoundQuery{Sql: "select 1", BindVariables: nil}
_, lockErr := stc.ExecuteLock(ctx, rs, query, session, sqlparser.IsUsedLock)
@@ -284,7 +302,7 @@ func (stc *ScatterConn) runLockQuery(ctx context.Context, session *SafeSession)
}
}
-func checkAndResetShardSession(info *shardActionInfo, err error, session *SafeSession, target *querypb.Target) reset {
+func checkAndResetShardSession(info *shardActionInfo, err error, session *econtext.SafeSession, target *querypb.Target) reset {
retry := none
if info.reservedID != 0 && info.transactionID == 0 {
if wasConnectionClosed(err) {
@@ -300,7 +318,7 @@ func checkAndResetShardSession(info *shardActionInfo, err error, session *SafeSe
return retry
}
-func getQueryService(ctx context.Context, rs *srvtopo.ResolvedShard, info *shardActionInfo, session *SafeSession, skipReset bool) (queryservice.QueryService, error) {
+func getQueryService(ctx context.Context, rs *srvtopo.ResolvedShard, info *shardActionInfo, session *econtext.SafeSession, skipReset bool) (queryservice.QueryService, error) {
if info.alias == nil {
return rs.Gateway, nil
}
@@ -351,14 +369,21 @@ func (stc *ScatterConn) StreamExecuteMulti(
query string,
rss []*srvtopo.ResolvedShard,
bindVars []map[string]*querypb.BindVariable,
- session *SafeSession,
+ session *econtext.SafeSession,
autocommit bool,
callback func(reply *sqltypes.Result) error,
+ resultsObserver econtext.ResultsObserver,
) []error {
- if session.InLockSession() && session.TriggerLockHeartBeat() {
+ if session.InLockSession() && triggerLockHeartBeat(session) {
go stc.runLockQuery(ctx, session)
}
+ observedCallback := func(reply *sqltypes.Result) error {
+ if reply != nil {
+ resultsObserver.Observe(reply)
+ }
+ return callback(reply)
+ }
allErrors := stc.multiGoTransaction(
ctx,
"StreamExecute",
@@ -407,20 +432,20 @@ func (stc *ScatterConn) StreamExecuteMulti(
switch info.actionNeeded {
case nothing:
- err = qs.StreamExecute(ctx, rs.Target, query, bindVars[i], transactionID, reservedID, opts, callback)
+ err = qs.StreamExecute(ctx, rs.Target, query, bindVars[i], transactionID, reservedID, opts, observedCallback)
if err != nil {
retryRequest(func() {
// we seem to have lost our connection. it was a reserved connection, let's try to recreate it
info.actionNeeded = reserve
var state queryservice.ReservedState
- state, err = qs.ReserveStreamExecute(ctx, rs.Target, session.SetPreQueries(), query, bindVars[i], 0 /*transactionId*/, opts, callback)
+ state, err = qs.ReserveStreamExecute(ctx, rs.Target, session.SetPreQueries(), query, bindVars[i], 0 /*transactionId*/, opts, observedCallback)
reservedID = state.ReservedID
alias = state.TabletAlias
})
}
case begin:
var state queryservice.TransactionState
- state, err = qs.BeginStreamExecute(ctx, rs.Target, session.SavePoints(), query, bindVars[i], reservedID, opts, callback)
+ state, err = qs.BeginStreamExecute(ctx, rs.Target, session.SavePoints(), query, bindVars[i], reservedID, opts, observedCallback)
transactionID = state.TransactionID
alias = state.TabletAlias
if err != nil {
@@ -428,7 +453,7 @@ func (stc *ScatterConn) StreamExecuteMulti(
// we seem to have lost our connection. it was a reserved connection, let's try to recreate it
info.actionNeeded = reserveBegin
var state queryservice.ReservedTransactionState
- state, err = qs.ReserveBeginStreamExecute(ctx, rs.Target, session.SetPreQueries(), session.SavePoints(), query, bindVars[i], opts, callback)
+ state, err = qs.ReserveBeginStreamExecute(ctx, rs.Target, session.SetPreQueries(), session.SavePoints(), query, bindVars[i], opts, observedCallback)
transactionID = state.TransactionID
reservedID = state.ReservedID
alias = state.TabletAlias
@@ -436,22 +461,22 @@ func (stc *ScatterConn) StreamExecuteMulti(
}
case reserve:
var state queryservice.ReservedState
- state, err = qs.ReserveStreamExecute(ctx, rs.Target, session.SetPreQueries(), query, bindVars[i], transactionID, opts, callback)
+ state, err = qs.ReserveStreamExecute(ctx, rs.Target, session.SetPreQueries(), query, bindVars[i], transactionID, opts, observedCallback)
reservedID = state.ReservedID
alias = state.TabletAlias
case reserveBegin:
var state queryservice.ReservedTransactionState
- state, err = qs.ReserveBeginStreamExecute(ctx, rs.Target, session.SetPreQueries(), session.SavePoints(), query, bindVars[i], opts, callback)
+ state, err = qs.ReserveBeginStreamExecute(ctx, rs.Target, session.SetPreQueries(), session.SavePoints(), query, bindVars[i], opts, observedCallback)
transactionID = state.TransactionID
reservedID = state.ReservedID
alias = state.TabletAlias
default:
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] unexpected actionNeeded on query execution: %v", info.actionNeeded)
}
- session.logging.log(primitive, rs.Target, rs.Gateway, query, info.actionNeeded == begin || info.actionNeeded == reserveBegin, bindVars[i])
+ session.Log(primitive, rs.Target, rs.Gateway, query, info.actionNeeded == begin || info.actionNeeded == reserveBegin, bindVars[i])
- // We need to new shard info irrespective of the error.
- newInfo := info.updateTransactionAndReservedID(transactionID, reservedID, alias)
+ // We need the new shard info irrespective of the error.
+ newInfo := info.updateTransactionAndReservedID(transactionID, reservedID, alias, nil)
if err != nil {
return newInfo, err
}
@@ -583,7 +608,7 @@ func (stc *ScatterConn) multiGo(
startTime, statsKey := stc.startAction(name, rs.Target)
// Send a dummy session.
// TODO(sougou): plumb a real session through this call.
- defer stc.endAction(startTime, allErrors, statsKey, &err, NewSafeSession(nil))
+ defer stc.endAction(startTime, allErrors, statsKey, &err, econtext.NewSafeSession(nil))
err = action(rs, i)
}
@@ -625,7 +650,7 @@ func (stc *ScatterConn) multiGoTransaction(
ctx context.Context,
name string,
rss []*srvtopo.ResolvedShard,
- session *SafeSession,
+ session *econtext.SafeSession,
autocommit bool,
action shardActionTransactionFunc,
) (allErrors *concurrency.AllErrorRecorder) {
@@ -641,21 +666,24 @@ func (stc *ScatterConn) multiGoTransaction(
startTime, statsKey := stc.startAction(name, rs.Target)
defer stc.endAction(startTime, allErrors, statsKey, &err, session)
- shardActionInfo, err := actionInfo(ctx, rs.Target, session, autocommit, stc.txConn.mode)
+ info, shardSession, err := actionInfo(ctx, rs.Target, session, autocommit, stc.txConn.mode)
if err != nil {
return
}
- updated, err := action(rs, i, shardActionInfo)
- if updated == nil {
+ info, err = action(rs, i, info)
+ if info == nil {
return
}
- if updated.actionNeeded != nothing && (updated.transactionID != 0 || updated.reservedID != 0) {
- appendErr := session.AppendOrUpdate(&vtgatepb.Session_ShardSession{
- Target: rs.Target,
- TransactionId: updated.transactionID,
- ReservedId: updated.reservedID,
- TabletAlias: updated.alias,
- }, stc.txConn.mode)
+ if info.ignoreOldSession {
+ shardSession = nil
+ }
+ if shardSession != nil && info.rowsAffected {
+ // We might not always update or append in the session.
+ // We need to track if rows were affected in the transaction.
+ shardSession.RowsAffected = info.rowsAffected
+ }
+ if info.actionNeeded != nothing && (info.transactionID != 0 || info.reservedID != 0) {
+ appendErr := session.AppendOrUpdate(rs.Target, info, shardSession, stc.txConn.mode)
if appendErr != nil {
err = appendErr
}
@@ -706,7 +734,7 @@ func (stc *ScatterConn) multiGoTransaction(
// It returns an error recorder in which each shard error is recorded positionally,
// i.e. if rss[2] had an error, then the error recorder will store that error
// in the second position.
-func (stc *ScatterConn) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
+func (stc *ScatterConn) ExecuteLock(ctx context.Context, rs *srvtopo.ResolvedShard, query *querypb.BoundQuery, session *econtext.SafeSession, lockFuncType sqlparser.LockingFuncType) (*sqltypes.Result, error) {
var (
qr *sqltypes.Result
@@ -809,25 +837,25 @@ func requireNewQS(err error, target *querypb.Target) bool {
}
// actionInfo looks at the current session, and returns information about what needs to be done for this tablet
-func actionInfo(ctx context.Context, target *querypb.Target, session *SafeSession, autocommit bool, txMode vtgatepb.TransactionMode) (*shardActionInfo, error) {
+func actionInfo(ctx context.Context, target *querypb.Target, session *econtext.SafeSession, autocommit bool, txMode vtgatepb.TransactionMode) (*shardActionInfo, *vtgatepb.Session_ShardSession, error) {
if !(session.InTransaction() || session.InReservedConn()) {
- return &shardActionInfo{}, nil
+ return &shardActionInfo{}, nil, nil
}
ignoreSession := ctx.Value(engine.IgnoreReserveTxn)
if ignoreSession != nil {
- return &shardActionInfo{}, nil
+ return &shardActionInfo{}, nil, nil
}
// No need to protect ourselves from the race condition between
// Find and AppendOrUpdate. The higher level functions ensure that no
// duplicate (target) tuples can execute
// this at the same time.
- transactionID, reservedID, alias, err := session.FindAndChangeSessionIfInSingleTxMode(target.Keyspace, target.Shard, target.TabletType, txMode)
+ shardSession, err := session.FindAndChangeSessionIfInSingleTxMode(target.Keyspace, target.Shard, target.TabletType, txMode)
if err != nil {
- return nil, err
+ return nil, nil, err
}
- shouldReserve := session.InReservedConn() && reservedID == 0
- shouldBegin := session.InTransaction() && transactionID == 0 && !autocommit
+ shouldReserve := session.InReservedConn() && (shardSession == nil || shardSession.ReservedId == 0)
+ shouldBegin := session.InTransaction() && (shardSession == nil || shardSession.TransactionId == 0) && !autocommit
var act = nothing
switch {
@@ -839,16 +867,20 @@ func actionInfo(ctx context.Context, target *querypb.Target, session *SafeSessio
act = begin
}
- return &shardActionInfo{
- actionNeeded: act,
- transactionID: transactionID,
- reservedID: reservedID,
- alias: alias,
- }, nil
+ info := &shardActionInfo{
+ actionNeeded: act,
+ }
+ if shardSession != nil {
+ info.transactionID = shardSession.TransactionId
+ info.reservedID = shardSession.ReservedId
+ info.alias = shardSession.TabletAlias
+ info.rowsAffected = shardSession.RowsAffected
+ }
+ return info, shardSession, nil
}
// lockInfo looks at the current session, and returns information about what needs to be done for this tablet
-func lockInfo(target *querypb.Target, session *SafeSession, lockFuncType sqlparser.LockingFuncType) (*shardActionInfo, error) {
+func lockInfo(target *querypb.Target, session *econtext.SafeSession, lockFuncType sqlparser.LockingFuncType) (*shardActionInfo, error) {
info := &shardActionInfo{actionNeeded: nothing}
if session.LockSession != nil {
if !proto.Equal(target, session.LockSession.Target) {
@@ -873,10 +905,35 @@ type shardActionInfo struct {
actionNeeded actionNeeded
reservedID, transactionID int64
alias *topodatapb.TabletAlias
+
+ // ignoreOldSession is used when there is a retry on the same shard due to connection loss for a reserved connection.
+ // The old reference should be ignored and new shard session should be added to the session.
+ ignoreOldSession bool
+ rowsAffected bool
}
-func (sai *shardActionInfo) updateTransactionAndReservedID(txID int64, rID int64, alias *topodatapb.TabletAlias) *shardActionInfo {
- if txID == sai.transactionID && rID == sai.reservedID {
+func (sai *shardActionInfo) TransactionID() int64 {
+ return sai.transactionID
+}
+
+func (sai *shardActionInfo) ReservedID() int64 {
+ return sai.reservedID
+}
+
+func (sai *shardActionInfo) RowsAffected() bool {
+ return sai.rowsAffected
+}
+
+func (sai *shardActionInfo) Alias() *topodatapb.TabletAlias {
+ return sai.alias
+}
+
+func (sai *shardActionInfo) updateTransactionAndReservedID(txID int64, rID int64, alias *topodatapb.TabletAlias, qr *sqltypes.Result) *shardActionInfo {
+ firstTimeRowsAffected := false
+ if txID != 0 && qr != nil && !sai.rowsAffected {
+ firstTimeRowsAffected = qr.RowsAffected > 0
+ }
+ if txID == sai.transactionID && rID == sai.reservedID && !firstTimeRowsAffected {
// As transaction id and reserved id have not changed, there is nothing to update in session shard sessions.
return nil
}
@@ -884,6 +941,7 @@ func (sai *shardActionInfo) updateTransactionAndReservedID(txID int64, rID int64
newInfo.reservedID = rID
newInfo.transactionID = txID
newInfo.alias = alias
+ newInfo.rowsAffected = firstTimeRowsAffected
return &newInfo
}
diff --git a/go/vt/vtgate/scatter_conn_test.go b/go/vt/vtgate/scatter_conn_test.go
index 0e863805d9c..ab8680ca5e6 100644
--- a/go/vt/vtgate/scatter_conn_test.go
+++ b/go/vt/vtgate/scatter_conn_test.go
@@ -21,6 +21,7 @@ import (
"testing"
"vitess.io/vitess/go/vt/log"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/mysql/sqlerror"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
@@ -100,7 +101,7 @@ func TestExecuteFailOnAutocommit(t *testing.T) {
},
Autocommit: false,
}
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, NewSafeSession(session), true /*autocommit*/, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss, queries, econtext.NewSafeSession(session), true /*autocommit*/, false, nullResultsObserver{})
err := vterrors.Aggregate(errs)
require.Error(t, err)
require.Contains(t, err.Error(), "in autocommit mode, transactionID should be zero but was: 123")
@@ -183,7 +184,7 @@ func TestExecutePanic(t *testing.T) {
require.Contains(t, logMessage, "(*ScatterConn).multiGoTransaction")
}()
- _, _ = sc.ExecuteMultiShard(ctx, nil, rss, queries, NewSafeSession(session), true /*autocommit*/, false)
+ _, _ = sc.ExecuteMultiShard(ctx, nil, rss, queries, econtext.NewSafeSession(session), true /*autocommit*/, false, nullResultsObserver{})
}
@@ -204,7 +205,7 @@ func TestReservedOnMultiReplica(t *testing.T) {
res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
destinations := []key.Destination{key.DestinationShard("0")}
for i := 0; i < 10; i++ {
executeOnShards(t, ctx, res, keyspace, sc, session, destinations)
@@ -351,7 +352,7 @@ func TestReservedBeginTableDriven(t *testing.T) {
res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa")
t.Run(test.name, func(t *testing.T) {
- session := NewSafeSession(&vtgatepb.Session{})
+ session := econtext.NewSafeSession(&vtgatepb.Session{})
for _, action := range test.actions {
session.Session.InTransaction = action.transaction
session.Session.InReservedConn = action.reserved
@@ -384,7 +385,7 @@ func TestReservedConnFail(t *testing.T) {
_ = hc.AddTestTablet("aa", "1", 1, keyspace, "1", topodatapb.TabletType_REPLICA, true, 1, nil)
res := srvtopo.NewResolver(newSandboxForCells(ctx, []string{"aa"}), sc.gateway, "aa")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: false, InReservedConn: true})
destinations := []key.Destination{key.DestinationShard("0")}
executeOnShards(t, ctx, res, keyspace, sc, session, destinations)
diff --git a/go/vt/vtgate/semantics/FakeSI.go b/go/vt/vtgate/semantics/FakeSI.go
index 1ca6718f1a8..cb1b9cec094 100644
--- a/go/vt/vtgate/semantics/FakeSI.go
+++ b/go/vt/vtgate/semantics/FakeSI.go
@@ -85,3 +85,8 @@ func (s *FakeSI) KeyspaceError(keyspace string) error {
func (s *FakeSI) GetAggregateUDFs() []string {
return s.UDFs
}
+
+// FindMirrorRule implements SchemaInformation.
+func (s *FakeSI) FindMirrorRule(tablename sqlparser.TableName) (*vindexes.MirrorRule, error) {
+ return nil, nil
+}
diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go
index 8bb7cc393fc..0a9d2480d9b 100644
--- a/go/vt/vtgate/semantics/analyzer.go
+++ b/go/vt/vtgate/semantics/analyzer.go
@@ -357,7 +357,7 @@ func (a *analyzer) collationEnv() *collations.Environment {
}
func (a *analyzer) analyze(statement sqlparser.Statement) error {
- _ = sqlparser.Rewrite(statement, nil, a.earlyUp)
+ _ = sqlparser.Rewrite(statement, a.earlyTables.down, a.earlyTables.up)
if a.err != nil {
return a.err
}
@@ -387,7 +387,14 @@ func (a *analyzer) reAnalyze(statement sqlparser.SQLNode) error {
// canShortCut checks if we are dealing with a single unsharded keyspace and no tables that have managed foreign keys
// if so, we can stop the analyzer early
func (a *analyzer) canShortCut(statement sqlparser.Statement) (canShortCut bool) {
- ks, _ := singleUnshardedKeyspace(a.earlyTables.Tables)
+ var ks *vindexes.Keyspace
+ switch statement.(type) {
+ case sqlparser.SelectStatement:
+ ks, canShortCut = canTakeSelectUnshardedShortcut(a.earlyTables.Tables)
+ default:
+ ks, canShortCut = canTakeUnshardedShortcut(a.earlyTables.Tables)
+ }
+
a.singleUnshardedKeyspace = ks != nil
if !a.singleUnshardedKeyspace {
return false
@@ -424,13 +431,6 @@ func (a *analyzer) canShortCut(statement sqlparser.Statement) (canShortCut bool)
return true
}
-// earlyUp collects tables in the query, so we can check
-// if this a single unsharded query we are dealing with
-func (a *analyzer) earlyUp(cursor *sqlparser.Cursor) bool {
- a.earlyTables.up(cursor)
- return true
-}
-
func (a *analyzer) shouldContinue() bool {
return a.err == nil
}
@@ -455,6 +455,10 @@ func (a *analyzer) noteQuerySignature(node sqlparser.SQLNode) {
if node.GroupBy != nil {
a.sig.Aggregation = true
}
+ case *sqlparser.With:
+ if node.Recursive {
+ a.sig.RecursiveCTE = true
+ }
case sqlparser.AggrFunc:
a.sig.Aggregation = true
case *sqlparser.Delete, *sqlparser.Update, *sqlparser.Insert:
diff --git a/go/vt/vtgate/semantics/analyzer_test.go b/go/vt/vtgate/semantics/analyzer_test.go
index 0fbf0911f3a..0c42456b0ab 100644
--- a/go/vt/vtgate/semantics/analyzer_test.go
+++ b/go/vt/vtgate/semantics/analyzer_test.go
@@ -195,6 +195,59 @@ func TestBindingMultiTablePositive(t *testing.T) {
}
}
+func TestBindingRecursiveCTEs(t *testing.T) {
+ type testCase struct {
+ query string
+ rdeps TableSet
+ ddeps TableSet
+ }
+ queries := []testCase{{
+ query: "with recursive x as (select id from user union select x.id + 1 from x where x.id < 15) select t.id from x join x t;",
+ rdeps: TS3,
+ ddeps: TS3,
+ }, {
+ query: "WITH RECURSIVE user_cte AS (SELECT id, name FROM user WHERE id = 42 UNION ALL SELECT u.id, u.name FROM user u JOIN user_cte cte ON u.id = cte.id + 1 WHERE u.id = 42) SELECT id FROM user_cte",
+ rdeps: TS3,
+ ddeps: TS3,
+ }}
+ for _, query := range queries {
+ t.Run(query.query, func(t *testing.T) {
+ stmt, semTable := parseAndAnalyzeStrict(t, query.query, "user")
+ sel := stmt.(*sqlparser.Select)
+ assert.Equal(t, query.rdeps, semTable.RecursiveDeps(extract(sel, 0)), "recursive")
+ assert.Equal(t, query.ddeps, semTable.DirectDeps(extract(sel, 0)), "direct")
+ })
+ }
+}
+
+func TestRecursiveCTEChecking(t *testing.T) {
+ type testCase struct {
+ name, query, err string
+ }
+ queries := []testCase{{
+ name: "recursive CTE using aggregation",
+ query: "with recursive x as (select id from user union select count(*) from x) select t.id from x join x t",
+ err: "VT09027: Recursive Common Table Expression 'x' can contain neither aggregation nor window functions in recursive query block",
+ }, {
+ name: "recursive CTE using grouping",
+ query: "with recursive x as (select id from user union select id+1 from x where id < 10 group by 1) select t.id from x join x t",
+ err: "VT09027: Recursive Common Table Expression 'x' can contain neither aggregation nor window functions in recursive query block",
+ }, {
+ name: "use the same recursive cte twice in definition",
+ query: "with recursive x as (select 1 union select id+1 from x where id < 10 union select id+2 from x where id < 20) select t.id from x",
+ err: "VT09029: In recursive query block of Recursive Common Table Expression x, the recursive table must be referenced only once, and not in any subquery",
+ }}
+ for _, tc := range queries {
+ t.Run(tc.query, func(t *testing.T) {
+ parse, err := sqlparser.NewTestParser().Parse(tc.query)
+ require.NoError(t, err)
+
+ _, err = AnalyzeStrict(parse, "user", fakeSchemaInfo())
+ require.EqualError(t, err, tc.err)
+ })
+ }
+}
+
func TestBindingMultiAliasedTablePositive(t *testing.T) {
type testCase struct {
query string
@@ -887,9 +940,6 @@ func TestInvalidQueries(t *testing.T) {
}, {
sql: "select 1 from t1 where (id, id) in (select 1, 2, 3)",
serr: "Operand should contain 2 column(s)",
- }, {
- sql: "WITH RECURSIVE cte (n) AS (SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5) SELECT * FROM cte",
- serr: "VT12001: unsupported: recursive common table expression",
}, {
sql: "with x as (select 1), x as (select 1) select * from x",
serr: "VT03013: not unique table/alias: 'x'",
@@ -956,7 +1006,7 @@ func TestScopingWithWITH(t *testing.T) {
}, {
query: "with c as (select x as foo from user), t as (select foo as id from c) select id from t",
recursive: TS0,
- direct: TS3,
+ direct: TS2,
}, {
query: "with t as (select foo as id from user) select t.id from t",
recursive: TS0,
diff --git a/go/vt/vtgate/semantics/check_invalid.go b/go/vt/vtgate/semantics/check_invalid.go
index a739e857c00..6509f5f5ee8 100644
--- a/go/vt/vtgate/semantics/check_invalid.go
+++ b/go/vt/vtgate/semantics/check_invalid.go
@@ -48,10 +48,6 @@ func (a *analyzer) checkForInvalidConstructs(cursor *sqlparser.Cursor) error {
}
case *sqlparser.Subquery:
return a.checkSubqueryColumns(cursor.Parent(), node)
- case *sqlparser.With:
- if node.Recursive {
- return vterrors.VT12001("recursive common table expression")
- }
case *sqlparser.Insert:
if !a.singleUnshardedKeyspace && node.Action == sqlparser.ReplaceAct {
return ShardedError{Inner: &UnsupportedConstruct{errString: "REPLACE INTO with sharded keyspace"}}
diff --git a/go/vt/vtgate/semantics/cte_table.go b/go/vt/vtgate/semantics/cte_table.go
new file mode 100644
index 00000000000..498fc5076c1
--- /dev/null
+++ b/go/vt/vtgate/semantics/cte_table.go
@@ -0,0 +1,182 @@
+/*
+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 semantics
+
+import (
+ "strings"
+
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/evalengine"
+ "vitess.io/vitess/go/vt/vtgate/vindexes"
+)
+
+// CTETable contains the information about the CTE table.
+// This is a special TableInfo that is used to represent the recursive table inside a CTE. For the query:
+// WITH RECURSIVE cte AS (SELECT 1 UNION ALL SELECT * FROM cte as C1) SELECT * FROM cte as C2
+// The CTE table C1 is represented by a CTETable.
+type CTETable struct {
+ TableName string
+ ASTNode *sqlparser.AliasedTableExpr
+ *CTE
+}
+
+var _ TableInfo = (*CTETable)(nil)
+
+func newCTETable(node *sqlparser.AliasedTableExpr, t sqlparser.TableName, cteDef *CTE) *CTETable {
+ var name string
+ if node.As.IsEmpty() {
+ name = t.Name.String()
+ } else {
+ name = node.As.String()
+ }
+
+ authoritative := true
+ for _, expr := range cteDef.Query.GetColumns() {
+ _, isStar := expr.(*sqlparser.StarExpr)
+ if isStar {
+ authoritative = false
+ break
+ }
+ }
+ cteDef.isAuthoritative = authoritative
+
+ return &CTETable{
+ TableName: name,
+ ASTNode: node,
+ CTE: cteDef,
+ }
+}
+
+func (cte *CTETable) Name() (sqlparser.TableName, error) {
+ return sqlparser.NewTableName(cte.TableName), nil
+}
+
+func (cte *CTETable) GetVindexTable() *vindexes.Table {
+ return nil
+}
+
+func (cte *CTETable) IsInfSchema() bool {
+ return false
+}
+
+func (cte *CTETable) matches(name sqlparser.TableName) bool {
+ return cte.TableName == name.Name.String() && name.Qualifier.IsEmpty()
+}
+
+func (cte *CTETable) authoritative() bool {
+ return cte.isAuthoritative
+}
+
+func (cte *CTETable) GetAliasedTableExpr() *sqlparser.AliasedTableExpr {
+ return cte.ASTNode
+}
+
+func (cte *CTETable) canShortCut() shortCut {
+ return canShortCut
+}
+
+func (cte *CTETable) getColumns(bool) []ColumnInfo {
+ selExprs := cte.Query.GetColumns()
+ cols := make([]ColumnInfo, 0, len(selExprs))
+ for i, selExpr := range selExprs {
+ ae, isAe := selExpr.(*sqlparser.AliasedExpr)
+ if !isAe {
+ panic(vterrors.VT12001("should not be called"))
+ }
+ if len(cte.Columns) == 0 {
+ cols = append(cols, ColumnInfo{Name: ae.ColumnName()})
+ continue
+ }
+
+ // We have column aliases defined on the CTE
+ cols = append(cols, ColumnInfo{Name: cte.Columns[i].String()})
+ }
+ return cols
+}
+
+func (cte *CTETable) dependencies(colName string, org originable) (dependencies, error) {
+ directDeps := org.tableSetFor(cte.ASTNode)
+ columns := cte.getColumns(false)
+ for _, columnInfo := range columns {
+ if strings.EqualFold(columnInfo.Name, colName) {
+ return createCertain(directDeps, directDeps, evalengine.NewUnknownType()), nil
+ }
+ }
+
+ if cte.authoritative() {
+ return ¬hing{}, nil
+ }
+
+ return createUncertain(directDeps, directDeps), nil
+}
+
+func (cte *CTETable) getExprFor(s string) (sqlparser.Expr, error) {
+ for _, se := range cte.Query.GetColumns() {
+ ae, ok := se.(*sqlparser.AliasedExpr)
+ if !ok {
+ return nil, vterrors.VT09015()
+ }
+ if ae.ColumnName() == s {
+ return ae.Expr, nil
+ }
+ }
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "Unknown column '%s' in 'field list'", s)
+}
+
+func (cte *CTETable) getTableSet(org originable) TableSet {
+ return org.tableSetFor(cte.ASTNode)
+}
+
+// GetMirrorRule implements TableInfo.
+func (cte *CTETable) GetMirrorRule() *vindexes.MirrorRule {
+ return nil
+}
+
+type CTE struct {
+ Name string
+ Query sqlparser.SelectStatement
+ isAuthoritative bool
+ recursiveDeps *TableSet
+ Columns sqlparser.Columns
+ IDForRecurse *TableSet
+
+ // Was this CTE marked for being recursive?
+ Recursive bool
+
+ // The CTE had the seed and term parts merged
+ Merged bool
+}
+
+func (cte *CTE) recursive(org originable) (id TableSet) {
+ if cte.recursiveDeps != nil {
+ return *cte.recursiveDeps
+ }
+
+ // We need to find the recursive dependencies of the CTE
+ // We'll do this by walking the inner query and finding all the tables
+ _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
+ ate, ok := node.(*sqlparser.AliasedTableExpr)
+ if !ok {
+ return true, nil
+ }
+ id = id.Merge(org.tableSetFor(ate))
+ return true, nil
+ }, cte.Query)
+ return
+}
diff --git a/go/vt/vtgate/semantics/dependencies.go b/go/vt/vtgate/semantics/dependencies.go
index 70167ff02fc..42b7a918384 100644
--- a/go/vt/vtgate/semantics/dependencies.go
+++ b/go/vt/vtgate/semantics/dependencies.go
@@ -17,6 +17,8 @@ limitations under the License.
package semantics
import (
+ "fmt"
+
querypb "vitess.io/vitess/go/vt/proto/query"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtgate/evalengine"
@@ -29,6 +31,7 @@ type (
empty() bool
get(col *sqlparser.ColName) (dependency, error)
merge(other dependencies, allowMulti bool) dependencies
+ debugString() string
}
dependency struct {
certain bool
@@ -100,6 +103,10 @@ func (u *uncertain) merge(d dependencies, _ bool) dependencies {
}
}
+func (u *uncertain) debugString() string {
+ return fmt.Sprintf("uncertain: %v %v %s", u.direct, u.recursive, u.typ.Type().String())
+}
+
func (c *certain) empty() bool {
return false
}
@@ -117,26 +124,34 @@ func (c *certain) merge(d dependencies, allowMulti bool) dependencies {
if d.recursive == c.recursive {
return c
}
- c.direct = c.direct.Merge(d.direct)
- c.recursive = c.recursive.Merge(d.recursive)
+
+ res := createCertain(c.direct.Merge(d.direct), c.recursive.Merge(d.recursive), c.typ)
if !allowMulti {
- c.err = true
+ res.err = true
}
- return c
+ return res
}
return c
}
-func (n *nothing) empty() bool {
+func (c *certain) debugString() string {
+ return fmt.Sprintf("certain: %v %v %s", c.direct, c.recursive, c.typ.Type().String())
+}
+
+func (*nothing) empty() bool {
return true
}
-func (n *nothing) get(*sqlparser.ColName) (dependency, error) {
+func (*nothing) get(*sqlparser.ColName) (dependency, error) {
return dependency{certain: true}, nil
}
-func (n *nothing) merge(d dependencies, _ bool) dependencies {
+func (*nothing) merge(d dependencies, _ bool) dependencies {
return d
}
+
+func (*nothing) debugString() string {
+ return "nothing"
+}
diff --git a/go/vt/vtgate/semantics/derived_table.go b/go/vt/vtgate/semantics/derived_table.go
index aabbe9f0b22..fc7e1cb391c 100644
--- a/go/vt/vtgate/semantics/derived_table.go
+++ b/go/vt/vtgate/semantics/derived_table.go
@@ -146,7 +146,7 @@ func (dt *DerivedTable) GetAliasedTableExpr() *sqlparser.AliasedTableExpr {
}
func (dt *DerivedTable) canShortCut() shortCut {
- panic(vterrors.VT12001("should not be called"))
+ return canShortCut
}
// GetVindexTable implements the TableInfo interface
@@ -195,3 +195,8 @@ func (dt *DerivedTable) checkForDuplicates() error {
}
return nil
}
+
+// GetMirrorRule implements TableInfo.
+func (dt *DerivedTable) GetMirrorRule() *vindexes.MirrorRule {
+ return nil
+}
diff --git a/go/vt/vtgate/semantics/early_rewriter.go b/go/vt/vtgate/semantics/early_rewriter.go
index 611c91e512c..3e53ed0816a 100644
--- a/go/vt/vtgate/semantics/early_rewriter.go
+++ b/go/vt/vtgate/semantics/early_rewriter.go
@@ -24,7 +24,6 @@ import (
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vtgate/evalengine"
)
type earlyRewriter struct {
@@ -48,16 +47,14 @@ func (r *earlyRewriter) down(cursor *sqlparser.Cursor) error {
switch node := cursor.Node().(type) {
case sqlparser.SelectExprs:
return r.handleSelectExprs(cursor, node)
- case *sqlparser.OrExpr:
- rewriteOrExpr(r.env, cursor, node)
- case *sqlparser.AndExpr:
- rewriteAndExpr(r.env, cursor, node)
case *sqlparser.NotExpr:
rewriteNotExpr(cursor, node)
case *sqlparser.ComparisonExpr:
return handleComparisonExpr(cursor, node)
case *sqlparser.With:
- return r.handleWith(node)
+ if !node.Recursive {
+ return r.handleWith(node)
+ }
case *sqlparser.AliasedTableExpr:
return r.handleAliasedTable(node)
case *sqlparser.Delete:
@@ -144,7 +141,7 @@ func (r *earlyRewriter) handleAliasedTable(node *sqlparser.AliasedTableExpr) err
node.As = tbl.Name
}
node.Expr = &sqlparser.DerivedTable{
- Select: cte.Subquery.Select,
+ Select: cte.Subquery,
}
if len(cte.Columns) > 0 {
node.Columns = cte.Columns
@@ -852,57 +849,6 @@ func (r *earlyRewriter) rewriteGroupByExpr(node *sqlparser.Literal) (sqlparser.E
return realCloneOfColNames(aliasedExpr.Expr, false), nil
}
-// rewriteOrExpr rewrites OR expressions when the right side is FALSE.
-func rewriteOrExpr(env *vtenv.Environment, cursor *sqlparser.Cursor, node *sqlparser.OrExpr) {
- newNode := rewriteOrFalse(env, *node)
- if newNode != nil {
- cursor.ReplaceAndRevisit(newNode)
- }
-}
-
-// rewriteAndExpr rewrites AND expressions when either side is TRUE.
-func rewriteAndExpr(env *vtenv.Environment, cursor *sqlparser.Cursor, node *sqlparser.AndExpr) {
- newNode := rewriteAndTrue(env, *node)
- if newNode != nil {
- cursor.ReplaceAndRevisit(newNode)
- }
-}
-
-func rewriteAndTrue(env *vtenv.Environment, andExpr sqlparser.AndExpr) sqlparser.Expr {
- // we are looking for the pattern `WHERE c = 1 AND 1 = 1`
- isTrue := func(subExpr sqlparser.Expr) bool {
- coll := env.CollationEnv().DefaultConnectionCharset()
- evalEnginePred, err := evalengine.Translate(subExpr, &evalengine.Config{
- Environment: env,
- Collation: coll,
- })
- if err != nil {
- return false
- }
-
- env := evalengine.EmptyExpressionEnv(env)
- res, err := env.Evaluate(evalEnginePred)
- if err != nil {
- return false
- }
-
- boolValue, err := res.Value(coll).ToBool()
- if err != nil {
- return false
- }
-
- return boolValue
- }
-
- if isTrue(andExpr.Left) {
- return andExpr.Right
- } else if isTrue(andExpr.Right) {
- return andExpr.Left
- }
-
- return nil
-}
-
// handleComparisonExpr processes Comparison expressions, specifically for tuples with equal length and EqualOp operator.
func handleComparisonExpr(cursor *sqlparser.Cursor, node *sqlparser.ComparisonExpr) error {
lft, lftOK := node.Left.(sqlparser.ValTuple)
@@ -968,41 +914,6 @@ func realCloneOfColNames(expr sqlparser.Expr, union bool) sqlparser.Expr {
}, nil).(sqlparser.Expr)
}
-func rewriteOrFalse(env *vtenv.Environment, orExpr sqlparser.OrExpr) sqlparser.Expr {
- // we are looking for the pattern `WHERE c = 1 OR 1 = 0`
- isFalse := func(subExpr sqlparser.Expr) bool {
- coll := env.CollationEnv().DefaultConnectionCharset()
- evalEnginePred, err := evalengine.Translate(subExpr, &evalengine.Config{
- Environment: env,
- Collation: coll,
- })
- if err != nil {
- return false
- }
-
- env := evalengine.EmptyExpressionEnv(env)
- res, err := env.Evaluate(evalEnginePred)
- if err != nil {
- return false
- }
-
- boolValue, err := res.Value(coll).ToBool()
- if err != nil {
- return false
- }
-
- return !boolValue
- }
-
- if isFalse(orExpr.Left) {
- return orExpr.Right
- } else if isFalse(orExpr.Right) {
- return orExpr.Left
- }
-
- return nil
-}
-
// rewriteJoinUsing rewrites SQL JOINs that use the USING clause to their equivalent
// JOINs with the ON condition. This function finds all the tables that have the
// specified columns in the USING clause, constructs an equality predicate for
diff --git a/go/vt/vtgate/semantics/early_rewriter_test.go b/go/vt/vtgate/semantics/early_rewriter_test.go
index 16b3756189f..1ec7786a46c 100644
--- a/go/vt/vtgate/semantics/early_rewriter_test.go
+++ b/go/vt/vtgate/semantics/early_rewriter_test.go
@@ -184,6 +184,9 @@ func TestExpandStar(t *testing.T) {
// if we are only star-expanding authoritative tables, we don't need to stop the expansion
sql: "SELECT * FROM (SELECT t2.*, 12 AS foo FROM t3, t2) as results",
expSQL: "select c1, c2, foo from (select t2.c1, t2.c2, 12 as foo from t3, t2) as results",
+ }, {
+ sql: "with recursive hierarchy as (select t1.a, t1.b from t1 where t1.a is null union select t1.a, t1.b from t1 join hierarchy on t1.a = hierarchy.b) select * from hierarchy",
+ expSQL: "with recursive hierarchy as (select t1.a, t1.b from t1 where t1.a is null union select t1.a, t1.b from t1 join hierarchy on t1.a = hierarchy.b) select a, b from hierarchy",
}}
for _, tcase := range tcases {
t.Run(tcase.sql, func(t *testing.T) {
@@ -837,6 +840,9 @@ func TestRewriteNot(t *testing.T) {
}, {
sql: "select a from t1 where not a > 12",
expected: "select a from t1 where a <= 12",
+ }, {
+ sql: "select (not (1 like ('a' is null)))",
+ expected: "select 1 not like ('a' is null) from dual",
}}
for _, tcase := range tcases {
t.Run(tcase.sql, func(t *testing.T) {
@@ -902,53 +908,6 @@ func TestOrderByDerivedTable(t *testing.T) {
}
}
-// TestConstantFolding tests that the rewriter is able to do various constant foldings properly.
-func TestConstantFolding(t *testing.T) {
- ks := &vindexes.Keyspace{
- Name: "main",
- Sharded: true,
- }
- schemaInfo := &FakeSI{
- Tables: map[string]*vindexes.Table{
- "t1": {
- Keyspace: ks,
- Name: sqlparser.NewIdentifierCS("t1"),
- Columns: []vindexes.Column{{
- Name: sqlparser.NewIdentifierCI("a"),
- Type: sqltypes.VarChar,
- }, {
- Name: sqlparser.NewIdentifierCI("b"),
- Type: sqltypes.VarChar,
- }, {
- Name: sqlparser.NewIdentifierCI("c"),
- Type: sqltypes.VarChar,
- }},
- ColumnListAuthoritative: true,
- },
- },
- }
- cDB := "db"
- tcases := []struct {
- sql string
- expSQL string
- }{{
- sql: "select 1 from t1 where (a, b) in ::fkc_vals and (2 is null or (1 is null or a in (1)))",
- expSQL: "select 1 from t1 where (a, b) in ::fkc_vals and a in (1)",
- }, {
- sql: "select 1 from t1 where (false or (false or a in (1)))",
- expSQL: "select 1 from t1 where a in (1)",
- }}
- for _, tcase := range tcases {
- t.Run(tcase.sql, func(t *testing.T) {
- ast, err := sqlparser.NewTestParser().Parse(tcase.sql)
- require.NoError(t, err)
- _, err = Analyze(ast, cDB, schemaInfo)
- require.NoError(t, err)
- require.Equal(t, tcase.expSQL, sqlparser.String(ast))
- })
- }
-}
-
// TestCTEToDerivedTableRewrite checks that CTEs are correctly rewritten to derived tables
func TestCTEToDerivedTableRewrite(t *testing.T) {
cDB := "db"
diff --git a/go/vt/vtgate/semantics/foreign_keys_test.go b/go/vt/vtgate/semantics/foreign_keys_test.go
index e1c26ecf569..a46c67c9710 100644
--- a/go/vt/vtgate/semantics/foreign_keys_test.go
+++ b/go/vt/vtgate/semantics/foreign_keys_test.go
@@ -141,13 +141,10 @@ func TestGetAllManagedForeignKeys(t *testing.T) {
{
name: "Collect all foreign key constraints",
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t0"],
- tbl["t1"],
- &DerivedTable{},
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t0"],
+ tbl["t1"],
+ &DerivedTable{}),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -171,12 +168,10 @@ func TestGetAllManagedForeignKeys(t *testing.T) {
{
name: "keyspace not found in schema information",
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t2"],
- tbl["t3"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t2"],
+ tbl["t3"],
+ ),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -188,12 +183,9 @@ func TestGetAllManagedForeignKeys(t *testing.T) {
{
name: "Cyclic fk constraints error",
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t0"], tbl["t1"],
- &DerivedTable{},
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t0"], tbl["t1"],
+ &DerivedTable{}),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -236,17 +228,11 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) {
},
},
getError: func() error { return fmt.Errorf("ambiguous test error") },
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t4"],
- tbl["t5"],
- },
- si: &FakeSI{
- KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
- "ks": vschemapb.Keyspace_managed,
- },
- },
- },
+ tables: makeTableCollector(&FakeSI{
+ KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
+ "ks": vschemapb.Keyspace_managed,
+ }}, tbl["t4"],
+ tbl["t5"]),
}
updateExprs := sqlparser.UpdateExprs{
&sqlparser.UpdateExpr{Name: cola, Expr: sqlparser.NewIntLiteral("1")},
@@ -350,12 +336,10 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
name: "Delete Query",
stmt: &sqlparser.Delete{},
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t0"],
- tbl["t1"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t0"],
+ tbl["t1"],
+ ),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -389,12 +373,10 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
cold: SingleTableSet(1),
},
},
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t4"],
- tbl["t5"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t4"],
+ tbl["t5"],
+ ),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -433,12 +415,10 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
Action: sqlparser.ReplaceAct,
},
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t0"],
- tbl["t1"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t0"],
+ tbl["t1"],
+ ),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -465,12 +445,9 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
Action: sqlparser.InsertAct,
},
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t0"],
- tbl["t1"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t0"],
+ tbl["t1"]),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -502,12 +479,9 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
colb: SingleTableSet(0),
},
},
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t6"],
- tbl["t1"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t6"],
+ tbl["t1"]),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -536,12 +510,9 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
name: "Insert error",
stmt: &sqlparser.Insert{},
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t2"],
- tbl["t3"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t2"],
+ tbl["t3"]),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -554,12 +525,9 @@ func TestGetInvolvedForeignKeys(t *testing.T) {
name: "Update error",
stmt: &sqlparser.Update{},
fkManager: &fkManager{
- tables: &tableCollector{
- Tables: []TableInfo{
- tbl["t2"],
- tbl["t3"],
- },
- },
+ tables: makeTableCollector(nil,
+ tbl["t2"],
+ tbl["t3"]),
si: &FakeSI{
KsForeignKeyMode: map[string]vschemapb.Keyspace_ForeignKeyMode{
"ks": vschemapb.Keyspace_managed,
@@ -600,3 +568,12 @@ func pkInfo(parentTable *vindexes.Table, pCols []string, cCols []string) vindexe
ChildColumns: sqlparser.MakeColumns(cCols...),
}
}
+
+func makeTableCollector(si SchemaInformation, tables ...TableInfo) *tableCollector {
+ return &tableCollector{
+ earlyTableCollector: earlyTableCollector{
+ Tables: tables,
+ si: si,
+ },
+ }
+}
diff --git a/go/vt/vtgate/semantics/info_schema.go b/go/vt/vtgate/semantics/info_schema.go
index 11e577f3fa7..127f4a00960 100644
--- a/go/vt/vtgate/semantics/info_schema.go
+++ b/go/vt/vtgate/semantics/info_schema.go
@@ -1603,11 +1603,15 @@ type infoSchemaWithColumns struct {
infoSchemaData map[string][]vindexes.Column
}
+var _ SchemaInformation = (*infoSchemaWithColumns)(nil)
+
// MySQLVersion implements SchemaInformation.
// We cache this information, since these are maps that are not changed
-var infoSchema57 = getInfoSchema57()
-var infoSchema80 = getInfoSchema80()
+var (
+ infoSchema57 = getInfoSchema57()
+ infoSchema80 = getInfoSchema80()
+)
// newSchemaInfo returns a SchemaInformation that has the column information for all info_schema tables
func newSchemaInfo(inner SchemaInformation) SchemaInformation {
@@ -1665,3 +1669,8 @@ func (i *infoSchemaWithColumns) KeyspaceError(keyspace string) error {
func (i *infoSchemaWithColumns) GetAggregateUDFs() []string {
return i.inner.GetAggregateUDFs()
}
+
+// FindMirrorRule implements SchemaInformation.
+func (i *infoSchemaWithColumns) FindMirrorRule(tablename sqlparser.TableName) (*vindexes.MirrorRule, error) {
+ return i.inner.FindMirrorRule(tablename)
+}
diff --git a/go/vt/vtgate/semantics/real_table.go b/go/vt/vtgate/semantics/real_table.go
index 4f1639d0897..64f3ac5f3f0 100644
--- a/go/vt/vtgate/semantics/real_table.go
+++ b/go/vt/vtgate/semantics/real_table.go
@@ -20,9 +20,11 @@ import (
"strings"
"vitess.io/vitess/go/mysql/collations"
+ "vitess.io/vitess/go/slice"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/evalengine"
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
@@ -31,26 +33,41 @@ type RealTable struct {
dbName, tableName string
ASTNode *sqlparser.AliasedTableExpr
Table *vindexes.Table
+ CTE *CTE
VindexHint *sqlparser.IndexHint
+ MirrorRule *vindexes.MirrorRule
isInfSchema bool
collationEnv *collations.Environment
+ cache map[string]dependencies
}
var _ TableInfo = (*RealTable)(nil)
// dependencies implements the TableInfo interface
-func (r *RealTable) dependencies(colName string, org originable) (dependencies, error) {
- ts := org.tableSetFor(r.ASTNode)
- for _, info := range r.getColumns(false /* ignoreInvisbleCol */) {
- if strings.EqualFold(info.Name, colName) {
- return createCertain(ts, ts, info.Type), nil
+func (r *RealTable) dependencies(colName string, org originable) (deps dependencies, err error) {
+ var myID *TableSet
+ if r.cache == nil {
+ r.cache = make(map[string]dependencies)
+ ts := org.tableSetFor(r.ASTNode)
+ myID = &ts
+ for _, info := range r.getColumns(false /* ignoreInvisbleCol */) {
+ r.cache[strings.ToLower(info.Name)] = createCertain(ts, ts, info.Type)
}
}
+ if deps, ok := r.cache[strings.ToLower(colName)]; ok {
+ return deps, nil
+ }
+
if r.authoritative() {
return ¬hing{}, nil
}
- return createUncertain(ts, ts), nil
+
+ if myID == nil {
+ ts := org.tableSetFor(r.ASTNode)
+ myID = &ts
+ }
+ return createUncertain(*myID, *myID), nil
}
// GetTables implements the TableInfo interface
@@ -70,9 +87,17 @@ func (r *RealTable) IsInfSchema() bool {
// GetColumns implements the TableInfo interface
func (r *RealTable) getColumns(ignoreInvisbleCol bool) []ColumnInfo {
- if r.Table == nil {
+ switch {
+ case r.CTE != nil:
+ return r.getCTEColumns()
+ case r.Table == nil:
return nil
+ default:
+ return r.getVindexTableColumns(ignoreInvisbleCol)
}
+}
+
+func (r *RealTable) getVindexTableColumns(ignoreInvisbleCol bool) []ColumnInfo {
nameMap := map[string]any{}
cols := make([]ColumnInfo, 0, len(r.Table.Columns))
for _, col := range r.Table.Columns {
@@ -105,6 +130,57 @@ func (r *RealTable) getColumns(ignoreInvisbleCol bool) []ColumnInfo {
return cols
}
+func (r *RealTable) getCTEColumns() []ColumnInfo {
+ selectExprs := r.CTE.Query.GetColumns()
+ ci := extractColumnsFromCTE(r.CTE.Columns, selectExprs)
+ if ci != nil {
+ return ci
+ }
+ return extractSelectExprsFromCTE(selectExprs)
+}
+
+// Authoritative implements the TableInfo interface
+func (r *RealTable) authoritative() bool {
+ switch {
+ case r.Table != nil:
+ return r.Table.ColumnListAuthoritative
+ case r.CTE != nil:
+ return r.CTE.isAuthoritative
+ default:
+ return false
+ }
+}
+
+func extractSelectExprsFromCTE(selectExprs sqlparser.SelectExprs) []ColumnInfo {
+ var ci []ColumnInfo
+ for _, expr := range selectExprs {
+ ae, ok := expr.(*sqlparser.AliasedExpr)
+ if !ok {
+ return nil
+ }
+ ci = append(ci, ColumnInfo{
+ Name: ae.ColumnName(),
+ Type: evalengine.NewUnknownType(), // TODO: set the proper type
+ })
+ }
+ return ci
+}
+
+func extractColumnsFromCTE(columns sqlparser.Columns, selectExprs sqlparser.SelectExprs) []ColumnInfo {
+ if len(columns) == 0 {
+ return nil
+ }
+ if len(selectExprs) != len(columns) {
+ panic("mismatch of columns")
+ }
+ return slice.Map(columns, func(from sqlparser.IdentifierCI) ColumnInfo {
+ return ColumnInfo{
+ Name: from.String(),
+ Type: evalengine.NewUnknownType(),
+ }
+ })
+}
+
// GetExpr implements the TableInfo interface
func (r *RealTable) GetAliasedTableExpr() *sqlparser.AliasedTableExpr {
return r.ASTNode
@@ -145,12 +221,12 @@ func (r *RealTable) Name() (sqlparser.TableName, error) {
return r.ASTNode.TableName()
}
-// Authoritative implements the TableInfo interface
-func (r *RealTable) authoritative() bool {
- return r.Table != nil && r.Table.ColumnListAuthoritative
-}
-
// Matches implements the TableInfo interface
func (r *RealTable) matches(name sqlparser.TableName) bool {
return (name.Qualifier.IsEmpty() || name.Qualifier.String() == r.dbName) && r.tableName == name.Name.String()
}
+
+// GetMirrorRule implements TableInfo.
+func (r *RealTable) GetMirrorRule() *vindexes.MirrorRule {
+ return r.MirrorRule
+}
diff --git a/go/vt/vtgate/semantics/scoper.go b/go/vt/vtgate/semantics/scoper.go
index ae3e5b7e88d..9d596d9ecd1 100644
--- a/go/vt/vtgate/semantics/scoper.go
+++ b/go/vt/vtgate/semantics/scoper.go
@@ -35,9 +35,10 @@ type (
binder *binder
// These scopes are only used for rewriting ORDER BY 1 and GROUP BY 1
- specialExprScopes map[*sqlparser.Literal]*scope
- statementIDs map[sqlparser.Statement]TableSet
- si SchemaInformation
+ specialExprScopes map[*sqlparser.Literal]*scope
+ statementIDs map[sqlparser.Statement]TableSet
+ commonTableExprScopes []*sqlparser.CommonTableExpr
+ si SchemaInformation
}
scope struct {
@@ -105,6 +106,8 @@ func (s *scoper) down(cursor *sqlparser.Cursor) error {
s.currentScope().inHaving = true
return nil
}
+ case *sqlparser.CommonTableExpr:
+ s.commonTableExprScopes = append(s.commonTableExprScopes, node)
}
return nil
}
@@ -240,6 +243,9 @@ func (s *scoper) up(cursor *sqlparser.Cursor) error {
case sqlparser.AggrFunc:
s.currentScope().inHavingAggr = false
case sqlparser.TableExpr:
+ // inside joins and derived tables, we can only see the tables in the table/join.
+ // we also want the tables available in the outer query, for SELECT expressions and the WHERE clause,
+ // so we copy the tables from the current scope to the parent scope
if isParentSelect(cursor) {
curScope := s.currentScope()
s.popScope()
@@ -258,6 +264,8 @@ func (s *scoper) up(cursor *sqlparser.Cursor) error {
s.binder.usingJoinInfo[ts] = m
}
}
+ case *sqlparser.CommonTableExpr:
+ s.commonTableExprScopes = s.commonTableExprScopes[:len(s.commonTableExprScopes)-1]
}
return nil
}
@@ -367,7 +375,7 @@ func checkForInvalidAliasUse(cte *sqlparser.CommonTableExpr, name string) (err e
}
return err == nil
}
- _ = sqlparser.CopyOnRewrite(cte.Subquery.Select, down, nil, nil)
+ _ = sqlparser.CopyOnRewrite(cte.Subquery, down, nil, nil)
return err
}
diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_table.go
similarity index 94%
rename from go/vt/vtgate/semantics/semantic_state.go
rename to go/vt/vtgate/semantics/semantic_table.go
index ac2fd9c1604..f9856a901a6 100644
--- a/go/vt/vtgate/semantics/semantic_state.go
+++ b/go/vt/vtgate/semantics/semantic_table.go
@@ -63,6 +63,9 @@ type (
dependencies(colName string, org originable) (dependencies, error)
getExprFor(s string) (sqlparser.Expr, error)
getTableSet(org originable) TableSet
+
+ // GetMirrorRule returns the vschema mirror rule for this TableInfo
+ GetMirrorRule() *vindexes.MirrorRule
}
// ColumnInfo contains information about columns
@@ -77,12 +80,19 @@ type (
// QuerySignature is used to identify shortcuts in the planning process
QuerySignature struct {
- Aggregation bool
- DML bool
- Distinct bool
- HashJoin bool
- SubQueries bool
- Union bool
+ Aggregation bool
+ DML bool
+ Distinct bool
+ HashJoin bool
+ SubQueries bool
+ Union bool
+ RecursiveCTE bool
+ }
+
+ // MirrorInfo stores information used to produce mirror
+ // operators.
+ MirrorInfo struct {
+ Percent float32
}
// SemTable contains semantic analysis information about the query.
@@ -162,6 +172,7 @@ type (
GetForeignKeyChecksState() *bool
KeyspaceError(keyspace string) error
GetAggregateUDFs() []string
+ FindMirrorRule(tablename sqlparser.TableName) (*vindexes.MirrorRule, error)
}
shortCut = int
@@ -173,10 +184,8 @@ const (
dependsOnKeyspace
)
-var (
- // ErrNotSingleTable refers to an error happening when something should be used only for single tables
- ErrNotSingleTable = vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] should only be used for single tables")
-)
+// ErrNotSingleTable refers to an error happening when something should be used only for single tables
+var ErrNotSingleTable = vterrors.VT13001("should only be used for single tables")
// CopyDependencies copies the dependencies from one expression into the other
func (st *SemTable) CopyDependencies(from, to sqlparser.Expr) {
@@ -733,7 +742,6 @@ func RewriteDerivedTableExpression(expr sqlparser.Expr, vt TableInfo) sqlparser.
col := *node
col.Qualifier = sqlparser.TableName{}
cursor.Replace(&col)
-
}, nil).(sqlparser.Expr)
}
@@ -773,10 +781,6 @@ func singleUnshardedKeyspace(tableInfos []TableInfo) (ks *vindexes.Keyspace, tab
}
for _, table := range tableInfos {
- if _, isDT := table.(*DerivedTable); isDT {
- continue
- }
-
sc := table.canShortCut()
var vtbl *vindexes.Table
@@ -800,6 +804,7 @@ func singleUnshardedKeyspace(tableInfos []TableInfo) (ks *vindexes.Keyspace, tab
tables = append(tables, vtbl)
}
+
return ks, tables
}
@@ -983,3 +988,45 @@ func (st *SemTable) NewTableId() TableSet {
st.Tables = append(st.Tables, nil)
return tableID
}
+
+func (st *SemTable) CanTakeSelectUnshardedShortcut() (*vindexes.Keyspace, bool) {
+ return canTakeSelectUnshardedShortcut(st.Tables)
+}
+
+func (st *SemTable) CanTakeUnshardedShortcut() (*vindexes.Keyspace, bool) {
+ return canTakeUnshardedShortcut(st.Tables)
+}
+
+func canTakeUnshardedShortcut(tableInfos []TableInfo) (*vindexes.Keyspace, bool) {
+ uks, _ := singleUnshardedKeyspace(tableInfos)
+ return uks, uks != nil
+}
+
+func canTakeSelectUnshardedShortcut(tableInfos []TableInfo) (*vindexes.Keyspace, bool) {
+ if mi := mirrorInfo(tableInfos); mi.Percent > 0 {
+ return nil, false
+ }
+ return canTakeUnshardedShortcut(tableInfos)
+}
+
+func (st *SemTable) GetMirrorInfo() MirrorInfo {
+ return mirrorInfo(st.Tables)
+}
+
+// mirrorInfo looks through all tables with mirror rules defined, and returns a
+// MirrorInfo containing the lowest mirror percentage found across all rules.
+//
+// The idea here is that if you have two tables with mirror rules both involved
+// in a query, and one of those rules is 1% while the other is 100%, to mirror
+// the query with 1% chance.
+func mirrorInfo(tableInfos []TableInfo) MirrorInfo {
+ mi := MirrorInfo{}
+ for _, t := range tableInfos {
+ if mr := t.GetMirrorRule(); mr != nil {
+ if mi.Percent == 0 || mr.Percent < mi.Percent {
+ mi.Percent = mr.Percent
+ }
+ }
+ }
+ return mi
+}
diff --git a/go/vt/vtgate/semantics/semantic_state_test.go b/go/vt/vtgate/semantics/semantic_table_test.go
similarity index 99%
rename from go/vt/vtgate/semantics/semantic_state_test.go
rename to go/vt/vtgate/semantics/semantic_table_test.go
index 84f8cec6cf9..1f324215326 100644
--- a/go/vt/vtgate/semantics/semantic_state_test.go
+++ b/go/vt/vtgate/semantics/semantic_table_test.go
@@ -464,7 +464,7 @@ func TestRemoveParentForeignKey(t *testing.T) {
},
},
},
- expectedErr: "[BUG] should only be used for single tables",
+ expectedErr: "VT13001: [BUG] should only be used for single tables",
},
}
for _, tt := range tests {
@@ -716,7 +716,7 @@ func TestRemoveNonRequiredForeignKeys(t *testing.T) {
SingleTableSet(0).Merge(SingleTableSet(1)): {},
},
},
- expectedErr: "[BUG] should only be used for single tables",
+ expectedErr: "VT13001: [BUG] should only be used for single tables",
},
{
name: "Error - Reading table info for child foreign keys",
@@ -734,7 +734,7 @@ func TestRemoveNonRequiredForeignKeys(t *testing.T) {
},
parentForeignKeysInvolved: map[TableSet][]vindexes.ParentFKInfo{},
},
- expectedErr: "[BUG] should only be used for single tables",
+ expectedErr: "VT13001: [BUG] should only be used for single tables",
},
}
for _, tt := range tests {
diff --git a/go/vt/vtgate/semantics/table_collector.go b/go/vt/vtgate/semantics/table_collector.go
index 948edb37d47..191d9c3b38e 100644
--- a/go/vt/vtgate/semantics/table_collector.go
+++ b/go/vt/vtgate/semantics/table_collector.go
@@ -28,45 +28,59 @@ import (
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
-// tableCollector is responsible for gathering information about the tables listed in the FROM clause,
-// and adding them to the current scope, plus keeping the global list of tables used in the query
-type tableCollector struct {
- Tables []TableInfo
- scoper *scoper
- si SchemaInformation
- currentDb string
- org originable
- unionInfo map[*sqlparser.Union]unionInfo
- done map[*sqlparser.AliasedTableExpr]TableInfo
-}
+type (
+ // tableCollector is responsible for gathering information about the tables listed in the FROM clause,
+ // and adding them to the current scope, plus keeping the global list of tables used in the query
+ tableCollector struct {
+ earlyTableCollector
+ scoper *scoper
+ org originable
+ unionInfo map[*sqlparser.Union]unionInfo
+ }
-type earlyTableCollector struct {
- si SchemaInformation
- currentDb string
- Tables []TableInfo
- done map[*sqlparser.AliasedTableExpr]TableInfo
- withTables map[sqlparser.IdentifierCS]any
-}
+ earlyTableCollector struct {
+ si SchemaInformation
+ currentDb string
+ Tables []TableInfo
+ done map[*sqlparser.AliasedTableExpr]TableInfo
+
+ // cte is a map of CTE definitions that are used in the query
+ cte map[string]*CTE
+ }
+)
func newEarlyTableCollector(si SchemaInformation, currentDb string) *earlyTableCollector {
return &earlyTableCollector{
- si: si,
- currentDb: currentDb,
- done: map[*sqlparser.AliasedTableExpr]TableInfo{},
- withTables: map[sqlparser.IdentifierCS]any{},
+ si: si,
+ currentDb: currentDb,
+ done: map[*sqlparser.AliasedTableExpr]TableInfo{},
+ cte: map[string]*CTE{},
}
}
-func (etc *earlyTableCollector) up(cursor *sqlparser.Cursor) {
- switch node := cursor.Node().(type) {
- case *sqlparser.AliasedTableExpr:
- etc.visitAliasedTableExpr(node)
- case *sqlparser.With:
- for _, cte := range node.CTEs {
- etc.withTables[cte.ID] = nil
+func (etc *earlyTableCollector) down(cursor *sqlparser.Cursor) bool {
+ with, ok := cursor.Node().(*sqlparser.With)
+ if !ok {
+ return true
+ }
+ for _, cte := range with.CTEs {
+ etc.cte[cte.ID.String()] = &CTE{
+ Name: cte.ID.String(),
+ Query: cte.Subquery,
+ Columns: cte.Columns,
+ Recursive: with.Recursive,
}
}
+ return true
+}
+func (etc *earlyTableCollector) up(cursor *sqlparser.Cursor) bool {
+ ate, ok := cursor.Node().(*sqlparser.AliasedTableExpr)
+ if !ok {
+ return true
+ }
+ etc.visitAliasedTableExpr(ate)
+ return true
}
func (etc *earlyTableCollector) visitAliasedTableExpr(aet *sqlparser.AliasedTableExpr) {
@@ -79,25 +93,22 @@ func (etc *earlyTableCollector) visitAliasedTableExpr(aet *sqlparser.AliasedTabl
func (etc *earlyTableCollector) newTableCollector(scoper *scoper, org originable) *tableCollector {
return &tableCollector{
- Tables: etc.Tables,
- scoper: scoper,
- si: etc.si,
- currentDb: etc.currentDb,
- unionInfo: map[*sqlparser.Union]unionInfo{},
- done: etc.done,
- org: org,
+ earlyTableCollector: *etc,
+ scoper: scoper,
+ unionInfo: map[*sqlparser.Union]unionInfo{},
+ org: org,
}
}
func (etc *earlyTableCollector) handleTableName(tbl sqlparser.TableName, aet *sqlparser.AliasedTableExpr) {
if tbl.Qualifier.IsEmpty() {
- _, isCTE := etc.withTables[tbl.Name]
+ _, isCTE := etc.cte[tbl.Name.String()]
if isCTE {
// no need to handle these tables here, we wait for the late phase instead
return
}
}
- tableInfo, err := getTableInfo(aet, tbl, etc.si, etc.currentDb)
+ tableInfo, err := etc.getTableInfo(aet, tbl, nil)
if err != nil {
// this could just be a CTE that we haven't processed, so we'll give it the benefit of the doubt for now
return
@@ -304,7 +315,7 @@ func (tc *tableCollector) handleTableName(node *sqlparser.AliasedTableExpr, t sq
tableInfo, found = tc.done[node]
if !found {
- tableInfo, err = getTableInfo(node, t, tc.si, tc.currentDb)
+ tableInfo, err = tc.earlyTableCollector.getTableInfo(node, t, tc.scoper)
if err != nil {
return err
}
@@ -315,12 +326,32 @@ func (tc *tableCollector) handleTableName(node *sqlparser.AliasedTableExpr, t sq
return scope.addTable(tableInfo)
}
-func getTableInfo(node *sqlparser.AliasedTableExpr, t sqlparser.TableName, si SchemaInformation, currentDb string) (TableInfo, error) {
+func (etc *earlyTableCollector) getCTE(t sqlparser.TableName) *CTE {
+ if t.Qualifier.NotEmpty() {
+ return nil
+ }
+
+ return etc.cte[t.Name.String()]
+}
+
+func (etc *earlyTableCollector) getTableInfo(node *sqlparser.AliasedTableExpr, t sqlparser.TableName, sc *scoper) (TableInfo, error) {
var tbl *vindexes.Table
var vindex vindexes.Vindex
+ if cteDef := etc.getCTE(t); cteDef != nil {
+ cte, err := etc.buildRecursiveCTE(node, t, sc, cteDef)
+ if err != nil {
+ return nil, err
+ }
+ if cte != nil {
+ // if we didn't get a table, it means we can't build a recursive CTE,
+ // so we need to look for a regular table instead
+ return cte, nil
+ }
+ }
+
isInfSchema := sqlparser.SystemSchema(t.Qualifier.String())
var err error
- tbl, vindex, _, _, _, err = si.FindTableOrVindex(t)
+ tbl, vindex, _, _, _, err = etc.si.FindTableOrVindex(t)
if err != nil && !isInfSchema {
// if we are dealing with a system table, it might not be available in the vschema, but that is OK
return nil, err
@@ -329,13 +360,64 @@ func getTableInfo(node *sqlparser.AliasedTableExpr, t sqlparser.TableName, si Sc
tbl = newVindexTable(t.Name)
}
- tableInfo, err := createTable(t, node, tbl, isInfSchema, vindex, si, currentDb)
+ tableInfo, err := etc.createTable(t, node, tbl, isInfSchema, vindex)
if err != nil {
return nil, err
}
return tableInfo, nil
}
+func (etc *earlyTableCollector) buildRecursiveCTE(node *sqlparser.AliasedTableExpr, t sqlparser.TableName, sc *scoper, cteDef *CTE) (TableInfo, error) {
+ // If sc is nil, then we are in the early table collector.
+ // In early table collector, we don't go over the CTE definitions, so we must be seeing a usage of the CTE.
+ if sc != nil && len(sc.commonTableExprScopes) > 0 {
+ cte := sc.commonTableExprScopes[len(sc.commonTableExprScopes)-1]
+ if cte.ID.String() == t.Name.String() {
+
+ if err := checkValidRecursiveCTE(cteDef); err != nil {
+ return nil, err
+ }
+
+ cteTable := newCTETable(node, t, cteDef)
+ cteTableSet := SingleTableSet(len(etc.Tables))
+ cteDef.IDForRecurse = &cteTableSet
+ if !cteDef.Recursive {
+ return nil, nil
+ }
+ return cteTable, nil
+ }
+ }
+ return &RealTable{
+ tableName: node.TableNameString(),
+ ASTNode: node,
+ CTE: cteDef,
+ collationEnv: etc.si.Environment().CollationEnv(),
+ }, nil
+}
+
+func checkValidRecursiveCTE(cteDef *CTE) error {
+ if cteDef.IDForRecurse != nil {
+ return vterrors.VT09029(cteDef.Name)
+ }
+
+ union, isUnion := cteDef.Query.(*sqlparser.Union)
+ if !isUnion {
+ return vterrors.VT09026(cteDef.Name)
+ }
+
+ firstSelect := sqlparser.GetFirstSelect(union.Right)
+ if firstSelect.GroupBy != nil {
+ return vterrors.VT09027(cteDef.Name)
+ }
+
+ for _, expr := range firstSelect.GetColumns() {
+ if sqlparser.ContainsAggregation(expr) {
+ return vterrors.VT09027(cteDef.Name)
+ }
+ }
+ return nil
+}
+
func (tc *tableCollector) handleDerivedTable(node *sqlparser.AliasedTableExpr, t *sqlparser.DerivedTable) error {
switch sel := t.Select.(type) {
case *sqlparser.Select:
@@ -437,14 +519,12 @@ func (tc *tableCollector) tableInfoFor(id TableSet) (TableInfo, error) {
return tc.Tables[offset], nil
}
-func createTable(
+func (etc *earlyTableCollector) createTable(
t sqlparser.TableName,
alias *sqlparser.AliasedTableExpr,
tbl *vindexes.Table,
isInfSchema bool,
vindex vindexes.Vindex,
- si SchemaInformation,
- currentDb string,
) (TableInfo, error) {
hint := getVindexHint(alias.Hints)
@@ -452,19 +532,28 @@ func createTable(
return nil, err
}
+ mr, err := etc.si.FindMirrorRule(t)
+ if err != nil {
+ // Mirroring is best effort. If we get an error while mirroring, keep going
+ // as if mirroring was disabled. We don't want to interrupt production work
+ // because of an issue with mirroring.
+ mr = nil
+ }
+
table := &RealTable{
tableName: alias.As.String(),
ASTNode: alias,
Table: tbl,
VindexHint: hint,
+ MirrorRule: mr,
isInfSchema: isInfSchema,
- collationEnv: si.Environment().CollationEnv(),
+ collationEnv: etc.si.Environment().CollationEnv(),
}
if alias.As.IsEmpty() {
dbName := t.Qualifier.String()
if dbName == "" {
- dbName = currentDb
+ dbName = etc.currentDb
}
table.dbName = dbName
diff --git a/go/vt/vtgate/semantics/vindex_table.go b/go/vt/vtgate/semantics/vindex_table.go
index b598c93f36a..c8ef271af5d 100644
--- a/go/vt/vtgate/semantics/vindex_table.go
+++ b/go/vt/vtgate/semantics/vindex_table.go
@@ -84,3 +84,8 @@ func (v *VindexTable) getColumns(ignoreInvisbleCol bool) []ColumnInfo {
func (v *VindexTable) IsInfSchema() bool {
return v.Table.IsInfSchema()
}
+
+// GetMirrorRule implements TableInfo.
+func (v *VindexTable) GetMirrorRule() *vindexes.MirrorRule {
+ return nil
+}
diff --git a/go/vt/vtgate/semantics/vtable.go b/go/vt/vtgate/semantics/vtable.go
index 14519a7e938..6cd7e34aecc 100644
--- a/go/vt/vtgate/semantics/vtable.go
+++ b/go/vt/vtgate/semantics/vtable.go
@@ -175,3 +175,8 @@ func selectExprsToInfos(
}
return
}
+
+// GetMirrorRule implements TableInfo.
+func (v *vTableInfo) GetMirrorRule() *vindexes.MirrorRule {
+ return nil
+}
diff --git a/go/vt/vtgate/tabletgateway.go b/go/vt/vtgate/tabletgateway.go
index 63ae836d715..c36c6981fa2 100644
--- a/go/vt/vtgate/tabletgateway.go
+++ b/go/vt/vtgate/tabletgateway.go
@@ -20,7 +20,9 @@ import (
"context"
"fmt"
"math/rand/v2"
+ "net/http"
"runtime/debug"
+ "slices"
"sort"
"sync"
"sync/atomic"
@@ -37,6 +39,7 @@ import (
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vtgate/balancer"
"vitess.io/vitess/go/vt/vtgate/buffer"
"vitess.io/vitess/go/vt/vttablet/queryservice"
@@ -54,6 +57,11 @@ var (
// retryCount is the number of times a query will be retried on error
retryCount = 2
+ // configuration flags for the tablet balancer
+ balancerEnabled bool
+ balancerVtgateCells []string
+ balancerKeyspaces []string
+
logCollations = logutil.NewThrottledLogger("CollationInconsistent", 1*time.Minute)
)
@@ -62,6 +70,9 @@ func init() {
fs.StringVar(&CellsToWatch, "cells_to_watch", "", "comma-separated list of cells for watching tablets")
fs.DurationVar(&initialTabletTimeout, "gateway_initial_tablet_timeout", 30*time.Second, "At startup, the tabletGateway will wait up to this duration to get at least one tablet per keyspace/shard/tablet type")
fs.IntVar(&retryCount, "retry-count", 2, "retry count")
+ fs.BoolVar(&balancerEnabled, "enable-balancer", false, "Enable the tablet balancer to evenly spread query load for a given tablet type")
+ fs.StringSliceVar(&balancerVtgateCells, "balancer-vtgate-cells", []string{}, "When in balanced mode, a comma-separated list of cells that contain vtgates (required)")
+ fs.StringSliceVar(&balancerKeyspaces, "balancer-keyspaces", []string{}, "When in balanced mode, a comma-separated list of keyspaces for which to use the balancer (optional)")
})
}
@@ -84,10 +95,17 @@ type TabletGateway struct {
// buffer, if enabled, buffers requests during a detected PRIMARY failover.
buffer *buffer.Buffer
+
+ // balancer used for routing to tablets
+ balancer balancer.TabletBalancer
}
func createHealthCheck(ctx context.Context, retryDelay, timeout time.Duration, ts *topo.Server, cell, cellsToWatch string) discovery.HealthCheck {
- return discovery.NewHealthCheck(ctx, retryDelay, timeout, ts, cell, cellsToWatch)
+ filters, err := discovery.NewVTGateHealthCheckFilters()
+ if err != nil {
+ log.Exit(err)
+ }
+ return discovery.NewHealthCheck(ctx, retryDelay, timeout, ts, cell, cellsToWatch, filters)
}
// NewTabletGateway creates and returns a new TabletGateway
@@ -112,6 +130,9 @@ func NewTabletGateway(ctx context.Context, hc discovery.HealthCheck, serv srvtop
statusAggregators: make(map[string]*TabletStatusAggregator),
}
gw.setupBuffering(ctx)
+ if balancerEnabled {
+ gw.setupBalancer(ctx)
+ }
gw.QueryService = queryservice.Wrap(nil, gw.withRetry)
return gw
}
@@ -145,6 +166,13 @@ func (gw *TabletGateway) setupBuffering(ctx context.Context) {
}(bufferCtx, ksChan, gw.buffer)
}
+func (gw *TabletGateway) setupBalancer(ctx context.Context) {
+ if len(balancerVtgateCells) == 0 {
+ log.Exitf("balancer-vtgate-cells is required for balanced mode")
+ }
+ gw.balancer = balancer.NewTabletBalancer(gw.localCell, balancerVtgateCells)
+}
+
// QueryServiceByAlias satisfies the Gateway interface
func (gw *TabletGateway) QueryServiceByAlias(ctx context.Context, alias *topodatapb.TabletAlias, target *querypb.Target) (queryservice.QueryService, error) {
qs, err := gw.hc.TabletConnection(ctx, alias, target)
@@ -191,11 +219,24 @@ func (gw *TabletGateway) WaitForTablets(ctx context.Context, tabletTypesToWait [
}
// Finds the targets to look for.
- targets, err := srvtopo.FindAllTargets(ctx, gw.srvTopoServer, gw.localCell, discovery.KeyspacesToWatch, tabletTypesToWait)
+ targets, keyspaces, err := srvtopo.FindAllTargetsAndKeyspaces(ctx, gw.srvTopoServer, gw.localCell, discovery.KeyspacesToWatch, tabletTypesToWait)
+ if err != nil {
+ return err
+ }
+ err = gw.hc.WaitForAllServingTablets(ctx, targets)
if err != nil {
return err
}
- return gw.hc.WaitForAllServingTablets(ctx, targets)
+ // After having waited for all serving tablets. We should also wait for the keyspace event watcher to have seen
+ // the updates and marked all the keyspaces as consistent (if we want to wait for primary tablets).
+ // Otherwise, we could be in a situation where even though the healthchecks have arrived, the keyspace event watcher hasn't finished processing them.
+ // So, if a primary tablet goes non-serving (because of a PRS or some other reason), we won't be able to start buffering.
+ // Waiting for the keyspaces to become consistent ensures that all the primary tablets for all the shards should be serving as seen by the keyspace event watcher
+ // and any disruption from now on, will make sure we start buffering properly.
+ if topoproto.IsTypeInList(topodatapb.TabletType_PRIMARY, tabletTypesToWait) && gw.kev != nil {
+ return gw.kev.WaitForConsistentKeyspaces(ctx, keyspaces)
+ }
+ return nil
}
// Close shuts down underlying connections.
@@ -220,6 +261,15 @@ func (gw *TabletGateway) CacheStatus() TabletCacheStatusList {
return res
}
+func (gw *TabletGateway) DebugBalancerHandler(w http.ResponseWriter, r *http.Request) {
+ if balancerEnabled {
+ gw.balancer.DebugHandler(w, r)
+ } else {
+ w.Header().Set("Content-Type", "text/plain")
+ w.Write([]byte("not enabled"))
+ }
+}
+
// withRetry gets available connections and executes the action. If there are retryable errors,
// it retries retryCount times before failing. It does not retry if the connection is in
// the middle of a transaction. While returning the error check if it maybe a result of
@@ -260,7 +310,7 @@ func (gw *TabletGateway) withRetry(ctx context.Context, target *querypb.Target,
// b) no transaction was created yet.
if gw.buffer != nil && !bufferedOnce && !inTransaction && target.TabletType == topodatapb.TabletType_PRIMARY {
// The next call blocks if we should buffer during a failover.
- retryDone, bufferErr := gw.buffer.WaitForFailoverEnd(ctx, target.Keyspace, target.Shard, err)
+ retryDone, bufferErr := gw.buffer.WaitForFailoverEnd(ctx, target.Keyspace, target.Shard, gw.kev, err)
// Request may have been buffered.
if retryDone != nil {
@@ -282,18 +332,21 @@ func (gw *TabletGateway) withRetry(ctx context.Context, target *querypb.Target,
if len(tablets) == 0 {
// if we have a keyspace event watcher, check if the reason why our primary is not available is that it's currently being resharded
// or if a reparent operation is in progress.
- if kev := gw.kev; kev != nil {
+ // We only check for whether reshard is ongoing or primary is serving or not, only if the target is primary. We don't want to buffer
+ // replica queries, so it doesn't make any sense to check for resharding or reparenting in that case.
+ if kev := gw.kev; kev != nil && target.TabletType == topodatapb.TabletType_PRIMARY {
if kev.TargetIsBeingResharded(ctx, target) {
log.V(2).Infof("current keyspace is being resharded, retrying: %s: %s", target.Keyspace, debug.Stack())
err = vterrors.Errorf(vtrpcpb.Code_CLUSTER_EVENT, buffer.ClusterEventReshardingInProgress)
continue
}
- primary, notServing := kev.PrimaryIsNotServing(ctx, target)
- if notServing {
+ primary, shouldBuffer := kev.ShouldStartBufferingForTarget(ctx, target)
+ if shouldBuffer {
err = vterrors.Errorf(vtrpcpb.Code_CLUSTER_EVENT, buffer.ClusterEventReparentInProgress)
continue
}
- // if primary is serving, but we initially found no tablet, we're in an inconsistent state
+ // if the keyspace event manager doesn't think we should buffer queries, and also sees a primary tablet,
+ // but we initially found no tablet, we're in an inconsistent state
// we then retry the entire loop
if primary != nil {
err = vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "inconsistent state detected, primary is serving but initially found no available tablet")
@@ -306,16 +359,35 @@ func (gw *TabletGateway) withRetry(ctx context.Context, target *querypb.Target,
break
}
- gw.shuffleTablets(gw.localCell, tablets)
-
var th *discovery.TabletHealth
- // skip tablets we tried before
- for _, t := range tablets {
- if _, ok := invalidTablets[topoproto.TabletAliasString(t.Tablet.Alias)]; !ok {
- th = t
- break
+
+ useBalancer := balancerEnabled
+ if balancerEnabled && len(balancerKeyspaces) > 0 {
+ useBalancer = slices.Contains(balancerKeyspaces, target.Keyspace)
+ }
+ if useBalancer {
+ // filter out the tablets that we've tried before (if any), then pick the best one
+ if len(invalidTablets) > 0 {
+ tablets = slices.DeleteFunc(tablets, func(t *discovery.TabletHealth) bool {
+ _, isInvalid := invalidTablets[topoproto.TabletAliasString(t.Tablet.Alias)]
+ return isInvalid
+ })
+ }
+
+ th = gw.balancer.Pick(target, tablets)
+
+ } else {
+ gw.shuffleTablets(gw.localCell, tablets)
+
+ // skip tablets we tried before
+ for _, t := range tablets {
+ if _, ok := invalidTablets[topoproto.TabletAliasString(t.Tablet.Alias)]; !ok {
+ th = t
+ break
+ }
}
}
+
if th == nil {
// do not override error from last attempt.
if err == nil {
diff --git a/go/vt/vtgate/tabletgateway_flaky_test.go b/go/vt/vtgate/tabletgateway_flaky_test.go
index 21107c8d30e..124997bea9e 100644
--- a/go/vt/vtgate/tabletgateway_flaky_test.go
+++ b/go/vt/vtgate/tabletgateway_flaky_test.go
@@ -20,6 +20,8 @@ import (
"testing"
"time"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/mysql/collations"
@@ -53,7 +55,7 @@ func TestGatewayBufferingWhenPrimarySwitchesServingState(t *testing.T) {
TabletType: tabletType,
}
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
// create a new fake health check. We want to check the buffering code which uses Subscribe, so we must also pass a channel
hc := discovery.NewFakeHealthCheck(make(chan *discovery.TabletHealth))
// create a new tablet gateway
@@ -67,7 +69,7 @@ func TestGatewayBufferingWhenPrimarySwitchesServingState(t *testing.T) {
waitForBuffering := func(enabled bool) {
timer := time.NewTimer(bufferingWaitTimeout)
defer timer.Stop()
- for _, buffering := tg.kev.PrimaryIsNotServing(ctx, target); buffering != enabled; _, buffering = tg.kev.PrimaryIsNotServing(ctx, target) {
+ for _, buffering := tg.kev.ShouldStartBufferingForTarget(ctx, target); buffering != enabled; _, buffering = tg.kev.ShouldStartBufferingForTarget(ctx, target) {
select {
case <-timer.C:
require.Fail(t, "timed out waiting for buffering of enabled: %t", enabled)
@@ -156,7 +158,7 @@ func TestGatewayBufferingWhileReparenting(t *testing.T) {
TabletType: tabletType,
}
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
// create a new fake health check. We want to check the buffering code which uses Subscribe, so we must also pass a channel
hc := discovery.NewFakeHealthCheck(make(chan *discovery.TabletHealth))
// create a new tablet gateway
@@ -213,8 +215,8 @@ func TestGatewayBufferingWhileReparenting(t *testing.T) {
hc.Broadcast(primaryTablet)
require.Len(t, tg.hc.GetHealthyTabletStats(target), 0, "GetHealthyTabletStats has tablets even though it shouldn't")
- _, isNotServing := tg.kev.PrimaryIsNotServing(ctx, target)
- require.True(t, isNotServing)
+ _, shouldStartBuffering := tg.kev.ShouldStartBufferingForTarget(ctx, target)
+ require.True(t, shouldStartBuffering)
// add a result to the sandbox connection of the new primary
sbcReplica.SetResults([]*sqltypes.Result{sqlResult1})
@@ -234,6 +236,7 @@ func TestGatewayBufferingWhileReparenting(t *testing.T) {
hc.SetTabletType(primaryTablet, topodatapb.TabletType_REPLICA)
hc.Broadcast(primaryTablet)
hc.SetTabletType(replicaTablet, topodatapb.TabletType_PRIMARY)
+ hc.SetPrimaryTimestamp(replicaTablet, 100) // We set a higher timestamp than before to simulate a PRS.
hc.SetServing(replicaTablet, true)
hc.Broadcast(replicaTablet)
@@ -244,8 +247,8 @@ outer:
case <-timeout:
require.Fail(t, "timed out - could not verify the new primary")
case <-time.After(10 * time.Millisecond):
- newPrimary, notServing := tg.kev.PrimaryIsNotServing(ctx, target)
- if newPrimary != nil && newPrimary.Uid == 1 && !notServing {
+ newPrimary, shouldBuffer := tg.kev.ShouldStartBufferingForTarget(ctx, target)
+ if newPrimary != nil && newPrimary.Uid == replicaTablet.Alias.Uid && !shouldBuffer {
break outer
}
}
@@ -285,7 +288,7 @@ func TestInconsistentStateDetectedBuffering(t *testing.T) {
TabletType: tabletType,
}
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
// create a new fake health check. We want to check the buffering code which uses Subscribe, so we must also pass a channel
hc := discovery.NewFakeHealthCheck(make(chan *discovery.TabletHealth))
// create a new tablet gateway
diff --git a/go/vt/vtgate/tabletgateway_test.go b/go/vt/vtgate/tabletgateway_test.go
index 32d18dcc9ab..b318cb84981 100644
--- a/go/vt/vtgate/tabletgateway_test.go
+++ b/go/vt/vtgate/tabletgateway_test.go
@@ -22,10 +22,13 @@ import (
"strings"
"testing"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/test/utils"
+ "vitess.io/vitess/go/vt/vttablet/queryservice"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/discovery"
@@ -34,6 +37,7 @@ import (
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/sandboxconn"
)
func TestTabletGatewayExecute(t *testing.T) {
@@ -41,7 +45,10 @@ func TestTabletGatewayExecute(t *testing.T) {
testTabletGatewayGeneric(t, ctx, func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error {
_, err := tg.Execute(ctx, target, "query", nil, 0, 0, nil)
return err
- })
+ },
+ func(t *testing.T, sc *sandboxconn.SandboxConn, want int64) {
+ assert.Equal(t, want, sc.ExecCount.Load())
+ })
testTabletGatewayTransact(t, ctx, func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error {
_, err := tg.Execute(ctx, target, "query", nil, 1, 0, nil)
return err
@@ -55,7 +62,10 @@ func TestTabletGatewayExecuteStream(t *testing.T) {
return nil
})
return err
- })
+ },
+ func(t *testing.T, sc *sandboxconn.SandboxConn, want int64) {
+ assert.Equal(t, want, sc.ExecCount.Load())
+ })
}
func TestTabletGatewayBegin(t *testing.T) {
@@ -63,7 +73,10 @@ func TestTabletGatewayBegin(t *testing.T) {
testTabletGatewayGeneric(t, ctx, func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error {
_, err := tg.Begin(ctx, target, nil)
return err
- })
+ },
+ func(t *testing.T, sc *sandboxconn.SandboxConn, want int64) {
+ assert.Equal(t, want, sc.BeginCount.Load())
+ })
}
func TestTabletGatewayCommit(t *testing.T) {
@@ -87,14 +100,18 @@ func TestTabletGatewayBeginExecute(t *testing.T) {
testTabletGatewayGeneric(t, ctx, func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error {
_, _, err := tg.BeginExecute(ctx, target, nil, "query", nil, 0, nil)
return err
- })
+ },
+ func(t *testing.T, sc *sandboxconn.SandboxConn, want int64) {
+ t.Helper()
+ assert.Equal(t, want, sc.BeginCount.Load())
+ })
}
func TestTabletGatewayShuffleTablets(t *testing.T) {
ctx := utils.LeakCheckContext(t)
hc := discovery.NewFakeHealthCheck(nil)
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
tg := NewTabletGateway(ctx, hc, ts, "local")
defer tg.Close(ctx)
@@ -168,7 +185,7 @@ func TestTabletGatewayReplicaTransactionError(t *testing.T) {
TabletType: tabletType,
}
hc := discovery.NewFakeHealthCheck(nil)
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
tg := NewTabletGateway(ctx, hc, ts, "cell")
defer tg.Close(ctx)
@@ -177,7 +194,20 @@ func TestTabletGatewayReplicaTransactionError(t *testing.T) {
verifyContainsError(t, err, "query service can only be used for non-transactional queries on replicas", vtrpcpb.Code_INTERNAL)
}
-func testTabletGatewayGeneric(t *testing.T, ctx context.Context, f func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error) {
+func testTabletGatewayGeneric(t *testing.T, ctx context.Context, f func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error, verifyExpectedCount func(t *testing.T, sc *sandboxconn.SandboxConn, want int64)) {
+ t.Helper()
+ testTabletGatewayGenericHelper(t, ctx, f, verifyExpectedCount)
+
+ // test again with the balancer enabled assuming vtgates in both cells where there
+ // are tablets, so that it will still route to the local cell always, but this way
+ // it will test both implementations of skipping invalid tablets for retry
+ balancerEnabled = true
+ balancerVtgateCells = []string{"cell", "cell2"}
+ testTabletGatewayGenericHelper(t, ctx, f, verifyExpectedCount)
+ balancerEnabled = false
+}
+
+func testTabletGatewayGenericHelper(t *testing.T, ctx context.Context, f func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error, verifyExpectedCount func(t *testing.T, sc *sandboxconn.SandboxConn, want int64)) {
t.Helper()
keyspace := "ks"
shard := "0"
@@ -190,10 +220,9 @@ func testTabletGatewayGeneric(t *testing.T, ctx context.Context, f func(ctx cont
TabletType: tabletType,
}
hc := discovery.NewFakeHealthCheck(nil)
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
tg := NewTabletGateway(ctx, hc, ts, "cell")
defer tg.Close(ctx)
-
// no tablet
want := []string{"target: ks.0.replica", `no healthy tablet available for 'keyspace:"ks" shard:"0" tablet_type:REPLICA`}
err := f(ctx, tg, target)
@@ -217,31 +246,50 @@ func testTabletGatewayGeneric(t *testing.T, ctx context.Context, f func(ctx cont
sc2 := hc.AddTestTablet("cell", host, port+1, keyspace, shard, tabletType, true, 10, nil)
sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1
sc2.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1
-
err = f(ctx, tg, target)
verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_FAILED_PRECONDITION)
+ verifyExpectedCount(t, sc1, 1)
+ verifyExpectedCount(t, sc2, 1)
// fatal error
hc.Reset()
sc1 = hc.AddTestTablet("cell", host, port, keyspace, shard, tabletType, true, 10, nil)
- sc2 = hc.AddTestTablet("cell", host, port+1, keyspace, shard, tabletType, true, 10, nil)
+ sc2 = hc.AddTestTablet("cell2", host, port+1, keyspace, shard, tabletType, true, 10, nil)
sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1
sc2.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1
err = f(ctx, tg, target)
verifyContainsError(t, err, "target: ks.0.replica", vtrpcpb.Code_FAILED_PRECONDITION)
+ verifyExpectedCount(t, sc1, 1)
+ verifyExpectedCount(t, sc2, 1)
// server error - no retry
hc.Reset()
sc1 = hc.AddTestTablet("cell", host, port, keyspace, shard, tabletType, true, 10, nil)
+ sc2 = hc.AddTestTablet("cell2", host, port+1, keyspace, shard, tabletType, true, 10, nil)
sc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1
err = f(ctx, tg, target)
assert.Equal(t, vtrpcpb.Code_INVALID_ARGUMENT, vterrors.Code(err))
+ verifyExpectedCount(t, sc1, 1)
+ verifyExpectedCount(t, sc2, 0)
// no failure
hc.Reset()
- hc.AddTestTablet("cell", host, port, keyspace, shard, tabletType, true, 10, nil)
+ sc1 = hc.AddTestTablet("cell", host, port, keyspace, shard, tabletType, true, 10, nil)
+ sc2 = hc.AddTestTablet("cell2", host, port, keyspace, shard, tabletType, true, 10, nil)
err = f(ctx, tg, target)
assert.NoError(t, err)
+ verifyExpectedCount(t, sc1, 0)
+ verifyExpectedCount(t, sc2, 1)
+
+ // retry successful to other cell
+ hc.Reset()
+ sc1 = hc.AddTestTablet("cell", host, port, keyspace, shard, tabletType, true, 10, nil)
+ sc2 = hc.AddTestTablet("cell2", host, port+1, keyspace, shard, tabletType, true, 10, nil)
+ sc1.MustFailCodes[vtrpcpb.Code_FAILED_PRECONDITION] = 1
+ err = f(ctx, tg, target)
+ assert.NoError(t, err)
+ verifyExpectedCount(t, sc1, 1)
+ verifyExpectedCount(t, sc2, 1)
}
func testTabletGatewayTransact(t *testing.T, ctx context.Context, f func(ctx context.Context, tg *TabletGateway, target *querypb.Target) error) {
@@ -260,7 +308,7 @@ func testTabletGatewayTransact(t *testing.T, ctx context.Context, f func(ctx con
TabletType: tabletType,
}
hc := discovery.NewFakeHealthCheck(nil)
- ts := &fakeTopoServer{}
+ ts := &econtext.FakeTopoServer{}
tg := NewTabletGateway(ctx, hc, ts, "cell")
defer tg.Close(ctx)
@@ -298,3 +346,58 @@ func verifyShardErrors(t *testing.T, err error, wantErrors []string, wantCode vt
}
require.Equal(t, vterrors.Code(err), wantCode, "wanted error code: %s, got: %v", wantCode, vterrors.Code(err))
}
+
+// TestWithRetry tests the functionality of withRetry function in different circumstances.
+func TestWithRetry(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ tg := NewTabletGateway(ctx, discovery.NewFakeHealthCheck(nil), &econtext.FakeTopoServer{}, "cell")
+ tg.kev = discovery.NewKeyspaceEventWatcher(ctx, tg.srvTopoServer, tg.hc, tg.localCell)
+ defer func() {
+ cancel()
+ tg.Close(ctx)
+ }()
+
+ testcases := []struct {
+ name string
+ target *querypb.Target
+ inTransaction bool
+ inner func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error)
+ expectedErr string
+ }{
+ {
+ name: "Transaction on a replica",
+ target: &querypb.Target{
+ Keyspace: "ks",
+ Shard: "0",
+ TabletType: topodatapb.TabletType_REPLICA,
+ },
+ inTransaction: true,
+ inner: func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error) {
+ return false, nil
+ },
+ expectedErr: "tabletGateway's query service can only be used for non-transactional queries on replicas",
+ }, {
+ name: "No replica tablets available",
+ target: &querypb.Target{
+ Keyspace: "ks",
+ Shard: "0",
+ TabletType: topodatapb.TabletType_REPLICA,
+ },
+ inTransaction: false,
+ inner: func(ctx context.Context, target *querypb.Target, conn queryservice.QueryService) (bool, error) {
+ return false, nil
+ },
+ expectedErr: `target: ks.0.replica: no healthy tablet available for 'keyspace:"ks" shard:"0" tablet_type:REPLICA'`,
+ },
+ }
+ for _, tt := range testcases {
+ t.Run(tt.name, func(t *testing.T) {
+ err := tg.withRetry(ctx, tt.target, nil, "", tt.inTransaction, tt.inner)
+ if tt.expectedErr == "" {
+ require.NoError(t, err)
+ } else {
+ require.ErrorContains(t, err, tt.expectedErr)
+ }
+ })
+ }
+}
diff --git a/go/vt/vtgate/testdata/executor_vexplain.json b/go/vt/vtgate/testdata/executor_vexplain.json
new file mode 100644
index 00000000000..2b893588aa5
--- /dev/null
+++ b/go/vt/vtgate/testdata/executor_vexplain.json
@@ -0,0 +1,129 @@
+[
+ {
+ "query": "select count(*), col2 from music group by col2",
+ "expected": {
+ "statementType": "SELECT",
+ "groupingColumns": [
+ "music.col2"
+ ],
+ "selectColumns": [
+ "music.col2"
+ ]
+ }
+ },
+ {
+ "query": "select * from user u join user_extra ue on u.id = ue.user_id where u.col1 \u003e 100 and ue.noLimit = 'foo'",
+ "expected": {
+ "statementType": "SELECT",
+ "filterColumns": [
+ "`user`.col1 gt",
+ "user_extra.noLimit ="
+ ],
+ "joinPredicates": [
+ "`user`.id = user_extra.user_id"
+ ]
+ }
+ },
+ {
+ "query": "select * from user_extra ue, user u where ue.noLimit = 'foo' and u.col1 \u003e 100 and ue.user_id = u.id",
+ "expected": {
+ "statementType": "SELECT",
+ "filterColumns": [
+ "`user`.col1 gt",
+ "user_extra.noLimit ="
+ ],
+ "joinPredicates": [
+ "`user`.id = user_extra.user_id"
+ ]
+ }
+ },
+ {
+ "query": "select u.foo, ue.bar, count(*) from user u join user_extra ue on u.id = ue.user_id where u.name = 'John Doe' group by 1, 2",
+ "expected": {
+ "statementType": "SELECT",
+ "groupingColumns": [
+ "`user`.foo",
+ "user_extra.bar"
+ ],
+ "filterColumns": [
+ "`user`.`name` ="
+ ],
+ "selectColumns": [
+ "`user`.foo",
+ "user_extra.bar"
+ ],
+ "joinPredicates": [
+ "`user`.id = user_extra.user_id"
+ ]
+ }
+ },
+ {
+ "query": "select * from (select * from user) as derived where derived.amount \u003e 1000",
+ "expected": {
+ "statementType": "SELECT"
+ }
+ },
+ {
+ "query": "select name, sum(amount) from user group by name",
+ "expected": {
+ "statementType": "SELECT",
+ "groupingColumns": [
+ "`user`.`name`"
+ ],
+ "selectColumns": [
+ "`user`.`name`",
+ "`user`.amount"
+ ]
+ }
+ },
+ {
+ "query": "select name from user where age \u003e 30",
+ "expected": {
+ "statementType": "SELECT",
+ "filterColumns": [
+ "`user`.age gt"
+ ],
+ "selectColumns": [
+ "`user`.`name`"
+ ]
+ }
+ },
+ {
+ "query": "select * from user where name = 'apa' union select * from user_extra where name = 'monkey'",
+ "expected": {
+ "statementType": "SELECT",
+ "filterColumns": [
+ "`user`.`name` =",
+ "user_extra.`name` ="
+ ]
+ }
+ },
+ {
+ "query": "update user set name = 'Jane Doe' where id = 1",
+ "expected": {
+ "statementType": "UPDATE",
+ "filterColumns": [
+ "`user`.id ="
+ ]
+ }
+ },
+ {
+ "query": "delete from user where order_date \u003c '2023-01-01'",
+ "expected": {
+ "statementType": "DELETE",
+ "filterColumns": [
+ "`user`.order_date lt"
+ ]
+ }
+ },
+ {
+ "query": "select * from user where name between 'A' and 'C'",
+ "expected": {
+ "statementType": "SELECT",
+ "filterColumns": [
+ "`user`.`name` ge",
+ "`user`.`name` le"
+ ]
+ }
+ }
+]
\ No newline at end of file
diff --git a/go/vt/vtgate/tx_conn.go b/go/vt/vtgate/tx_conn.go
index e388740ee6a..cadb1392eca 100644
--- a/go/vt/vtgate/tx_conn.go
+++ b/go/vt/vtgate/tx_conn.go
@@ -21,6 +21,7 @@ import (
"fmt"
"strings"
"sync"
+ "time"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/vt/concurrency"
@@ -32,6 +33,7 @@ import (
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vttablet/queryservice"
)
@@ -69,9 +71,17 @@ const (
Commit2pcConclude
)
+var phaseMessage = map[commitPhase]string{
+ Commit2pcCreateTransaction: "Create Transaction",
+ Commit2pcPrepare: "Prepare",
+ Commit2pcStartCommit: "Start Commit",
+ Commit2pcPrepareCommit: "Prepare Commit",
+ Commit2pcConclude: "Conclude",
+}
+
// Begin begins a new transaction. If one is already in progress, it commits it
// and starts a new one.
-func (txc *TxConn) Begin(ctx context.Context, session *SafeSession, txAccessModes []sqlparser.TxAccessMode) error {
+func (txc *TxConn) Begin(ctx context.Context, session *econtext.SafeSession, txAccessModes []sqlparser.TxAccessMode) error {
if session.InTransaction() {
if err := txc.Commit(ctx, session); err != nil {
return err
@@ -93,7 +103,7 @@ func (txc *TxConn) Begin(ctx context.Context, session *SafeSession, txAccessMode
// Commit commits the current transaction. The type of commit can be
// best effort or 2pc depending on the session setting.
-func (txc *TxConn) Commit(ctx context.Context, session *SafeSession) error {
+func (txc *TxConn) Commit(ctx context.Context, session *econtext.SafeSession) error {
defer session.ResetTx()
if !session.InTransaction() {
return nil
@@ -107,10 +117,48 @@ func (txc *TxConn) Commit(ctx context.Context, session *SafeSession) error {
twopc = txc.mode == vtgatepb.TransactionMode_TWOPC
}
+ defer recordCommitTime(session, twopc, time.Now())
+
+ err := txc.runSessions(ctx, session.PreSessions, session.GetLogger(), txc.commitShard)
+ if err != nil {
+ _ = txc.Release(ctx, session)
+ return err
+ }
+
if twopc {
- return txc.commit2PC(ctx, session)
+ err = txc.commit2PC(ctx, session)
+ } else {
+ err = txc.commitNormal(ctx, session)
+ }
+
+ if err != nil {
+ _ = txc.Release(ctx, session)
+ return err
+ }
+
+ err = txc.runSessions(ctx, session.PostSessions, session.GetLogger(), txc.commitShard)
+ if err != nil {
+ // If last commit fails, there will be nothing to rollback.
+ session.RecordWarning(&querypb.QueryWarning{Message: fmt.Sprintf("post-operation transaction had an error: %v", err)})
+ // With reserved connection we should release them.
+ if session.InReservedConn() {
+ _ = txc.Release(ctx, session)
+ }
+ }
+ return nil
+}
+
+func recordCommitTime(session *econtext.SafeSession, twopc bool, startTime time.Time) {
+ switch {
+ case len(session.ShardSessions) == 0:
+ // No-op
+ case len(session.ShardSessions) == 1:
+ commitMode.Record("Single", startTime)
+ case twopc:
+ commitMode.Record("TwoPC", startTime)
+ default:
+ commitMode.Record("Multi", startTime)
}
- return txc.commitNormal(ctx, session)
}
func (txc *TxConn) queryService(ctx context.Context, alias *topodatapb.TabletAlias) (queryservice.QueryService, error) {
@@ -120,7 +168,7 @@ func (txc *TxConn) queryService(ctx context.Context, alias *topodatapb.TabletAli
return txc.tabletGateway.QueryServiceByAlias(ctx, alias, nil)
}
-func (txc *TxConn) commitShard(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *executeLogger) error {
+func (txc *TxConn) commitShard(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
if s.TransactionId == 0 {
return nil
}
@@ -136,19 +184,14 @@ func (txc *TxConn) commitShard(ctx context.Context, s *vtgatepb.Session_ShardSes
}
s.TransactionId = 0
s.ReservedId = reservedID
- logging.log(nil, s.Target, nil, "commit", false, nil)
+ logging.Log(nil, s.Target, nil, "commit", false, nil)
return nil
}
-func (txc *TxConn) commitNormal(ctx context.Context, session *SafeSession) error {
- if err := txc.runSessions(ctx, session.PreSessions, session.logging, txc.commitShard); err != nil {
- _ = txc.Release(ctx, session)
- return err
- }
-
+func (txc *TxConn) commitNormal(ctx context.Context, session *econtext.SafeSession) error {
// Retain backward compatibility on commit order for the normal session.
for i, shardSession := range session.ShardSessions {
- if err := txc.commitShard(ctx, shardSession, session.logging); err != nil {
+ if err := txc.commitShard(ctx, shardSession, session.GetLogger()); err != nil {
if i > 0 {
nShards := i
elipsis := false
@@ -169,29 +212,14 @@ func (txc *TxConn) commitNormal(ctx context.Context, session *SafeSession) error
})
warnings.Add("NonAtomicCommit", 1)
}
- _ = txc.Release(ctx, session)
return err
}
}
-
- if err := txc.runSessions(ctx, session.PostSessions, session.logging, txc.commitShard); err != nil {
- // If last commit fails, there will be nothing to rollback.
- session.RecordWarning(&querypb.QueryWarning{Message: fmt.Sprintf("post-operation transaction had an error: %v", err)})
- // With reserved connection we should release them.
- if session.InReservedConn() {
- _ = txc.Release(ctx, session)
- }
- }
return nil
}
// commit2PC will not used the pinned tablets - to make sure we use the current source, we need to use the gateway's queryservice
-func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err error) {
- if len(session.PreSessions) != 0 || len(session.PostSessions) != 0 {
- _ = txc.Rollback(ctx, session)
- return vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "pre or post actions not allowed for 2PC commits")
- }
-
+func (txc *TxConn) commit2PC(ctx context.Context, session *econtext.SafeSession) (err error) {
// If the number of participants is one or less, then it's a normal commit.
if len(session.ShardSessions) <= 1 {
return txc.commitNormal(ctx, session)
@@ -206,11 +234,12 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
var txPhase commitPhase
+ var startCommitState querypb.StartCommitState
defer func() {
if err == nil {
return
}
- txc.errActionAndLogWarn(ctx, session, txPhase, dtid, mmShard, rmShards)
+ txc.errActionAndLogWarn(ctx, session, txPhase, startCommitState, dtid, mmShard, rmShards)
}()
txPhase = Commit2pcCreateTransaction
@@ -225,7 +254,7 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
txPhase = Commit2pcPrepare
- prepareAction := func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *executeLogger) error {
+ prepareAction := func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
if DebugTwoPc { // Test code to simulate a failure during RM prepare
if terr := checkTestFailure(ctx, "RMPrepare_-40_FailNow", s.Target); terr != nil {
return terr
@@ -233,7 +262,7 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
return txc.tabletGateway.Prepare(ctx, s.Target, s.TransactionId, dtid)
}
- if err = txc.runSessions(ctx, rmShards, session.logging, prepareAction); err != nil {
+ if err = txc.runSessions(ctx, rmShards, session.GetLogger(), prepareAction); err != nil {
return err
}
@@ -244,7 +273,7 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
txPhase = Commit2pcStartCommit
- err = txc.tabletGateway.StartCommit(ctx, mmShard.Target, mmShard.TransactionId, dtid)
+ startCommitState, err = txc.tabletGateway.StartCommit(ctx, mmShard.Target, mmShard.TransactionId, dtid)
if err != nil {
return err
}
@@ -256,7 +285,7 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
txPhase = Commit2pcPrepareCommit
- prepareCommitAction := func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *executeLogger) error {
+ prepareCommitAction := func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
if DebugTwoPc { // Test code to simulate a failure during RM prepare
if terr := checkTestFailure(ctx, "RMCommit_-40_FailNow", s.Target); terr != nil {
return terr
@@ -264,7 +293,7 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
}
return txc.tabletGateway.CommitPrepared(ctx, s.Target, dtid)
}
- if err = txc.runSessions(ctx, rmShards, session.logging, prepareCommitAction); err != nil {
+ if err = txc.runSessions(ctx, rmShards, session.GetLogger(), prepareCommitAction); err != nil {
return err
}
@@ -276,19 +305,38 @@ func (txc *TxConn) commit2PC(ctx context.Context, session *SafeSession) (err err
return nil
}
-func (txc *TxConn) errActionAndLogWarn(ctx context.Context, session *SafeSession, txPhase commitPhase, dtid string, mmShard *vtgatepb.Session_ShardSession, rmShards []*vtgatepb.Session_ShardSession) {
+func (txc *TxConn) errActionAndLogWarn(
+ ctx context.Context,
+ session *econtext.SafeSession,
+ txPhase commitPhase,
+ startCommitState querypb.StartCommitState,
+ dtid string,
+ mmShard *vtgatepb.Session_ShardSession,
+ rmShards []*vtgatepb.Session_ShardSession,
+) {
+ var rollbackErr error
switch txPhase {
case Commit2pcCreateTransaction:
// Normal rollback is safe because nothing was prepared yet.
- if rollbackErr := txc.Rollback(ctx, session); rollbackErr != nil {
- log.Warningf("Rollback failed after Create Transaction failure: %v", rollbackErr)
- }
+ rollbackErr = txc.Rollback(ctx, session)
case Commit2pcPrepare:
// Rollback the prepared and unprepared transactions.
- if resumeErr := txc.rollbackTx(ctx, dtid, mmShard, rmShards, session.logging); resumeErr != nil {
- log.Warningf("Rollback failed after Prepare failure: %v", resumeErr)
+ rollbackErr = txc.rollbackTx(ctx, dtid, mmShard, rmShards, session.GetLogger())
+ case Commit2pcStartCommit:
+ // Failed to store the commit decision on MM.
+ // If the failure state is certain, then the only option is to rollback the prepared transactions on the RMs.
+ if startCommitState == querypb.StartCommitState_Fail {
+ rollbackErr = txc.rollbackTx(ctx, dtid, mmShard, rmShards, session.GetLogger())
}
+ fallthrough
+ case Commit2pcPrepareCommit:
+ commitUnresolved.Add(1)
}
+ if rollbackErr != nil {
+ log.Warningf("Rollback failed after %s failure: %v", phaseMessage[txPhase], rollbackErr)
+ commitUnresolved.Add(1)
+ }
+
session.RecordWarning(&querypb.QueryWarning{
Code: uint32(sqlerror.ERInAtomicRecovery),
Message: createWarningMessage(dtid, txPhase)})
@@ -312,7 +360,7 @@ func createWarningMessage(dtid string, txPhase commitPhase) string {
}
// Rollback rolls back the current transaction. There are no retries on this operation.
-func (txc *TxConn) Rollback(ctx context.Context, session *SafeSession) error {
+func (txc *TxConn) Rollback(ctx context.Context, session *econtext.SafeSession) error {
if !session.InTransaction() {
return nil
}
@@ -321,7 +369,7 @@ func (txc *TxConn) Rollback(ctx context.Context, session *SafeSession) error {
allsessions := append(session.PreSessions, session.ShardSessions...)
allsessions = append(allsessions, session.PostSessions...)
- err := txc.runSessions(ctx, allsessions, session.logging, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *executeLogger) error {
+ err := txc.runSessions(ctx, allsessions, session.GetLogger(), func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
if s.TransactionId == 0 {
return nil
}
@@ -335,7 +383,7 @@ func (txc *TxConn) Rollback(ctx context.Context, session *SafeSession) error {
}
s.TransactionId = 0
s.ReservedId = reservedID
- logging.log(nil, s.Target, nil, "rollback", false, nil)
+ logging.Log(nil, s.Target, nil, "rollback", false, nil)
return nil
})
if err != nil {
@@ -348,7 +396,7 @@ func (txc *TxConn) Rollback(ctx context.Context, session *SafeSession) error {
}
// Release releases the reserved connection and/or rollbacks the transaction
-func (txc *TxConn) Release(ctx context.Context, session *SafeSession) error {
+func (txc *TxConn) Release(ctx context.Context, session *econtext.SafeSession) error {
if !session.InTransaction() && !session.InReservedConn() {
return nil
}
@@ -357,7 +405,7 @@ func (txc *TxConn) Release(ctx context.Context, session *SafeSession) error {
allsessions := append(session.PreSessions, session.ShardSessions...)
allsessions = append(allsessions, session.PostSessions...)
- return txc.runSessions(ctx, allsessions, session.logging, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *executeLogger) error {
+ return txc.runSessions(ctx, allsessions, session.GetLogger(), func(ctx context.Context, s *vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
if s.ReservedId == 0 && s.TransactionId == 0 {
return nil
}
@@ -376,7 +424,7 @@ func (txc *TxConn) Release(ctx context.Context, session *SafeSession) error {
}
// ReleaseLock releases the reserved connection used for locking.
-func (txc *TxConn) ReleaseLock(ctx context.Context, session *SafeSession) error {
+func (txc *TxConn) ReleaseLock(ctx context.Context, session *econtext.SafeSession) error {
if !session.InLockSession() {
return nil
}
@@ -395,7 +443,7 @@ func (txc *TxConn) ReleaseLock(ctx context.Context, session *SafeSession) error
}
// ReleaseAll releases all the shard sessions and lock session.
-func (txc *TxConn) ReleaseAll(ctx context.Context, session *SafeSession) error {
+func (txc *TxConn) ReleaseAll(ctx context.Context, session *econtext.SafeSession) error {
if !session.InTransaction() && !session.InReservedConn() && !session.InLockSession() {
return nil
}
@@ -407,7 +455,7 @@ func (txc *TxConn) ReleaseAll(ctx context.Context, session *SafeSession) error {
allsessions = append(allsessions, session.LockSession)
}
- return txc.runSessions(ctx, allsessions, session.logging, func(ctx context.Context, s *vtgatepb.Session_ShardSession, loggging *executeLogger) error {
+ return txc.runSessions(ctx, allsessions, session.GetLogger(), func(ctx context.Context, s *vtgatepb.Session_ShardSession, loggging *econtext.ExecuteLogger) error {
if s.ReservedId == 0 && s.TransactionId == 0 {
return nil
}
@@ -427,7 +475,7 @@ func (txc *TxConn) ReleaseAll(ctx context.Context, session *SafeSession) error {
// ResolveTransactions fetches all unresolved transactions and resolves them.
func (txc *TxConn) ResolveTransactions(ctx context.Context, target *querypb.Target) error {
- transactions, err := txc.tabletGateway.UnresolvedTransactions(ctx, target)
+ transactions, err := txc.tabletGateway.UnresolvedTransactions(ctx, target, 0 /* abandonAgeSeconds */)
if err != nil {
return err
}
@@ -479,12 +527,12 @@ func (txc *TxConn) resolveTx(ctx context.Context, target *querypb.Target, transa
// rollbackTx rollbacks the specified distributed transaction.
// Rollbacks happens on the metadata manager and all participants irrespective of the failure.
-func (txc *TxConn) rollbackTx(ctx context.Context, dtid string, mmShard *vtgatepb.Session_ShardSession, participants []*vtgatepb.Session_ShardSession, logging *executeLogger) error {
+func (txc *TxConn) rollbackTx(ctx context.Context, dtid string, mmShard *vtgatepb.Session_ShardSession, participants []*vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger) error {
var errs []error
if mmErr := txc.rollbackMM(ctx, dtid, mmShard); mmErr != nil {
errs = append(errs, mmErr)
}
- if rmErr := txc.runSessions(ctx, participants, logging, func(ctx context.Context, session *vtgatepb.Session_ShardSession, logger *executeLogger) error {
+ if rmErr := txc.runSessions(ctx, participants, logging, func(ctx context.Context, session *vtgatepb.Session_ShardSession, logger *econtext.ExecuteLogger) error {
return txc.tabletGateway.RollbackPrepared(ctx, session.Target, dtid, session.TransactionId)
}); rmErr != nil {
errs = append(errs, rmErr)
@@ -525,7 +573,7 @@ func (txc *TxConn) resumeCommit(ctx context.Context, target *querypb.Target, tra
}
// runSessions executes the action for all shardSessions in parallel and returns a consolidated error.
-func (txc *TxConn) runSessions(ctx context.Context, shardSessions []*vtgatepb.Session_ShardSession, logging *executeLogger, action func(context.Context, *vtgatepb.Session_ShardSession, *executeLogger) error) error {
+func (txc *TxConn) runSessions(ctx context.Context, shardSessions []*vtgatepb.Session_ShardSession, logging *econtext.ExecuteLogger, action func(context.Context, *vtgatepb.Session_ShardSession, *econtext.ExecuteLogger) error) error {
// Fastpath.
if len(shardSessions) == 1 {
return action(ctx, shardSessions[0], logging)
@@ -574,3 +622,19 @@ func (txc *TxConn) ReadTransaction(ctx context.Context, transactionID string) (*
}
return txc.tabletGateway.ReadTransaction(ctx, mmShard.Target, transactionID)
}
+
+func (txc *TxConn) UnresolvedTransactions(ctx context.Context, targets []*querypb.Target) ([]*querypb.TransactionMetadata, error) {
+ var tmList []*querypb.TransactionMetadata
+ var mu sync.Mutex
+ err := txc.runTargets(targets, func(target *querypb.Target) error {
+ res, err := txc.tabletGateway.UnresolvedTransactions(ctx, target, 0 /* abandonAgeSeconds */)
+ if err != nil {
+ return err
+ }
+ mu.Lock()
+ defer mu.Unlock()
+ tmList = append(tmList, res...)
+ return nil
+ })
+ return tmList, err
+}
diff --git a/go/vt/vtgate/tx_conn_test.go b/go/vt/vtgate/tx_conn_test.go
index ed977b75051..333094569c8 100644
--- a/go/vt/vtgate/tx_conn_test.go
+++ b/go/vt/vtgate/tx_conn_test.go
@@ -26,6 +26,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
+
"vitess.io/vitess/go/event/syslogger"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/test/utils"
@@ -51,12 +53,12 @@ func TestTxConnBegin(t *testing.T) {
session := &vtgatepb.Session{}
// begin
- safeSession := NewSafeSession(session)
+ safeSession := econtext.NewSafeSession(session)
err := sc.txConn.Begin(ctx, safeSession, nil)
require.NoError(t, err)
wantSession := vtgatepb.Session{InTransaction: true}
utils.MustMatch(t, &wantSession, session, "Session")
- _, errors := sc.ExecuteMultiShard(ctx, nil, rss0, queries, safeSession, false, false)
+ _, errors := sc.ExecuteMultiShard(ctx, nil, rss0, queries, safeSession, false, false, nullResultsObserver{})
require.Empty(t, errors)
// Begin again should cause a commit and a new begin.
@@ -75,8 +77,8 @@ func TestTxConnCommitFailure(t *testing.T) {
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rssm[0], queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rssm[0], queries, session, false, false, nullResultsObserver{})
wantSession := vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -91,7 +93,7 @@ func TestTxConnCommitFailure(t *testing.T) {
}
utils.MustMatch(t, &wantSession, session.Session, "Session")
- sc.ExecuteMultiShard(ctx, nil, rssm[1], queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rssm[1], queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -114,7 +116,7 @@ func TestTxConnCommitFailure(t *testing.T) {
}
utils.MustMatch(t, &wantSession, session.Session, "Session")
- sc.ExecuteMultiShard(ctx, nil, rssa, threeQueries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rssa, threeQueries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -176,14 +178,14 @@ func TestTxConnCommitFailureAfterNonAtomicCommitMaxShards(t *testing.T) {
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
wantSession := vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{},
}
for i := 0; i < 18; i++ {
- sc.ExecuteMultiShard(ctx, nil, rssm[i], queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rssm[i], queries, session, false, false, nullResultsObserver{})
wantSession.ShardSessions = append(wantSession.ShardSessions, &vtgatepb.Session_ShardSession{
Target: &querypb.Target{
Keyspace: "TestTxConn",
@@ -230,14 +232,14 @@ func TestTxConnCommitFailureBeforeNonAtomicCommitMaxShards(t *testing.T) {
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
wantSession := vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{},
}
for i := 0; i < 17; i++ {
- sc.ExecuteMultiShard(ctx, nil, rssm[i], queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rssm[i], queries, session, false, false, nullResultsObserver{})
wantSession.ShardSessions = append(wantSession.ShardSessions, &vtgatepb.Session_ShardSession{
Target: &querypb.Target{
Keyspace: "TestTxConn",
@@ -282,8 +284,8 @@ func TestTxConnCommitSuccess(t *testing.T) {
sc.txConn.mode = vtgatepb.TransactionMode_MULTI
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession := vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -297,7 +299,7 @@ func TestTxConnCommitSuccess(t *testing.T) {
}},
}
utils.MustMatch(t, &wantSession, session.Session, "Session")
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -335,8 +337,8 @@ func TestTxConnReservedCommitSuccess(t *testing.T) {
sc.txConn.mode = vtgatepb.TransactionMode_MULTI
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession := vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -352,7 +354,7 @@ func TestTxConnReservedCommitSuccess(t *testing.T) {
}},
}
utils.MustMatch(t, &wantSession, session.Session, "Session")
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -420,12 +422,12 @@ func TestTxConnReservedOn2ShardTxOn1ShardAndCommit(t *testing.T) {
sc.txConn.mode = vtgatepb.TransactionMode_MULTI
// Sequence the executes to ensure shard session order
- session := NewSafeSession(&vtgatepb.Session{InReservedConn: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InReservedConn: true})
// this will create reserved connections against all tablets
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errs)
- _, errs = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ _, errs = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errs)
wantSession := vtgatepb.Session{
@@ -453,7 +455,7 @@ func TestTxConnReservedOn2ShardTxOn1ShardAndCommit(t *testing.T) {
session.Session.InTransaction = true
// start a transaction against rss0
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -515,12 +517,12 @@ func TestTxConnReservedOn2ShardTxOn1ShardAndRollback(t *testing.T) {
sc.txConn.mode = vtgatepb.TransactionMode_MULTI
// Sequence the executes to ensure shard session order
- session := NewSafeSession(&vtgatepb.Session{InReservedConn: true})
+ session := econtext.NewSafeSession(&vtgatepb.Session{InReservedConn: true})
// this will create reserved connections against all tablets
- _, errs := sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ _, errs := sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errs)
- _, errs = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ _, errs = sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
require.Empty(t, errs)
wantSession := vtgatepb.Session{
@@ -548,7 +550,7 @@ func TestTxConnReservedOn2ShardTxOn1ShardAndRollback(t *testing.T) {
session.Session.InTransaction = true
// start a transaction against rss0
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -611,14 +613,14 @@ func TestTxConnCommitOrderFailure1(t *testing.T) {
queries := []*querypb.BoundQuery{{Sql: "query1"}}
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_PRE)
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_POST)
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
sbc0.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1
err := sc.txConn.Commit(ctx, session)
@@ -646,14 +648,14 @@ func TestTxConnCommitOrderFailure2(t *testing.T) {
}}
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(context.Background(), nil, rss1, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(context.Background(), nil, rss1, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_PRE)
- sc.ExecuteMultiShard(context.Background(), nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(context.Background(), nil, rss0, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_POST)
- sc.ExecuteMultiShard(context.Background(), nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(context.Background(), nil, rss1, queries, session, false, false, nullResultsObserver{})
sbc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1
err := sc.txConn.Commit(ctx, session)
@@ -680,14 +682,14 @@ func TestTxConnCommitOrderFailure3(t *testing.T) {
}}
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_PRE)
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
session.SetCommitOrder(vtgatepb.CommitOrder_POST)
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
sbc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1
require.NoError(t,
@@ -722,8 +724,8 @@ func TestTxConnCommitOrderSuccess(t *testing.T) {
}}
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession := vtgatepb.Session{
InTransaction: true,
ShardSessions: []*vtgatepb.Session_ShardSession{{
@@ -739,7 +741,7 @@ func TestTxConnCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
session.SetCommitOrder(vtgatepb.CommitOrder_PRE)
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
PreSessions: []*vtgatepb.Session_ShardSession{{
@@ -764,7 +766,7 @@ func TestTxConnCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
session.SetCommitOrder(vtgatepb.CommitOrder_POST)
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
PreSessions: []*vtgatepb.Session_ShardSession{{
@@ -798,7 +800,7 @@ func TestTxConnCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
// Ensure nothing changes if we reuse a transaction.
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
utils.MustMatch(t, &wantSession, session.Session, "Session")
require.NoError(t,
@@ -820,8 +822,8 @@ func TestTxConnReservedCommitOrderSuccess(t *testing.T) {
}}
// Sequence the executes to ensure commit order
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession := vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -839,7 +841,7 @@ func TestTxConnReservedCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
session.SetCommitOrder(vtgatepb.CommitOrder_PRE)
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -867,7 +869,7 @@ func TestTxConnReservedCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
session.SetCommitOrder(vtgatepb.CommitOrder_POST)
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
wantSession = vtgatepb.Session{
InTransaction: true,
InReservedConn: true,
@@ -905,7 +907,7 @@ func TestTxConnReservedCommitOrderSuccess(t *testing.T) {
utils.MustMatch(t, &wantSession, session.Session, "Session")
// Ensure nothing changes if we reuse a transaction.
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
utils.MustMatch(t, &wantSession, session.Session, "Session")
require.NoError(t,
@@ -957,9 +959,9 @@ func TestTxConnCommit2PC(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PC")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
require.NoError(t,
sc.txConn.Commit(ctx, session))
@@ -974,8 +976,8 @@ func TestTxConnCommit2PCOneParticipant(t *testing.T) {
ctx := utils.LeakCheckContext(t)
sc, sbc0, _, rss0, _, _ := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCOneParticipant")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
require.NoError(t,
sc.txConn.Commit(ctx, session))
@@ -987,16 +989,14 @@ func TestTxConnCommit2PCCreateTransactionFail(t *testing.T) {
sc, sbc0, sbc1, rss0, rss1, _ := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCCreateTransactionFail")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss1, queries, session, false, false, nullResultsObserver{})
sbc0.MustFailCreateTransaction = 1
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
err := sc.txConn.Commit(ctx, session)
- want := "error: err"
- require.Error(t, err)
- assert.Contains(t, err.Error(), want, "Commit")
+ require.ErrorContains(t, err, "target: TestTxConnCommit2PCCreateTransactionFail.0.primary: error: err")
assert.EqualValues(t, 1, sbc0.CreateTransactionCount.Load(), "sbc0.CreateTransactionCount")
assert.EqualValues(t, 1, sbc0.RollbackCount.Load(), "sbc0.RollbackCount")
assert.EqualValues(t, 1, sbc1.RollbackCount.Load(), "sbc1.RollbackCount")
@@ -1011,16 +1011,14 @@ func TestTxConnCommit2PCPrepareFail(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCPrepareFail")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
sbc1.MustFailPrepare = 1
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
err := sc.txConn.Commit(ctx, session)
- want := "error: err"
- require.Error(t, err)
- assert.Contains(t, err.Error(), want, "Commit")
+ require.ErrorContains(t, err, "target: TestTxConnCommit2PCPrepareFail.1.primary: error: err")
assert.EqualValues(t, 1, sbc0.CreateTransactionCount.Load(), "sbc0.CreateTransactionCount")
assert.EqualValues(t, 1, sbc1.PrepareCount.Load(), "sbc1.PrepareCount")
// Prepared failed on RM, so no commit on MM or RMs.
@@ -1039,20 +1037,40 @@ func TestTxConnCommit2PCStartCommitFail(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCStartCommitFail")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
sbc0.MustFailStartCommit = 1
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
err := sc.txConn.Commit(ctx, session)
- want := "error: err"
- require.Error(t, err)
- assert.Contains(t, err.Error(), want, "Commit")
+ require.ErrorContains(t, err, "target: TestTxConnCommit2PCStartCommitFail.0.primary: error: err")
+ assert.EqualValues(t, 1, sbc0.CreateTransactionCount.Load(), "sbc0.CreateTransactionCount")
+ assert.EqualValues(t, 1, sbc1.PrepareCount.Load(), "sbc1.PrepareCount")
+ assert.EqualValues(t, 1, sbc0.StartCommitCount.Load(), "sbc0.StartCommitCount")
+ assert.EqualValues(t, 0, sbc1.CommitPreparedCount.Load(), "sbc1.CommitPreparedCount")
+ assert.EqualValues(t, 1, sbc0.SetRollbackCount.Load(), "MM")
+ assert.EqualValues(t, 1, sbc1.RollbackPreparedCount.Load(), "RM")
+ assert.EqualValues(t, 1, sbc0.ConcludeTransactionCount.Load(), "sbc0.ConcludeTransactionCount")
+
+ sbc0.ResetCounter()
+ sbc1.ResetCounter()
+
+ session = econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
+
+ // Here the StartCommit failure is in uncertain state so rollback is not called and neither conclude.
+ sbc0.MustFailStartCommitUncertain = 1
+ session.TransactionMode = vtgatepb.TransactionMode_TWOPC
+ err = sc.txConn.Commit(ctx, session)
+ require.ErrorContains(t, err, "target: TestTxConnCommit2PCStartCommitFail.0.primary: uncertain error")
assert.EqualValues(t, 1, sbc0.CreateTransactionCount.Load(), "sbc0.CreateTransactionCount")
assert.EqualValues(t, 1, sbc1.PrepareCount.Load(), "sbc1.PrepareCount")
assert.EqualValues(t, 1, sbc0.StartCommitCount.Load(), "sbc0.StartCommitCount")
assert.EqualValues(t, 0, sbc1.CommitPreparedCount.Load(), "sbc1.CommitPreparedCount")
+ assert.EqualValues(t, 0, sbc0.SetRollbackCount.Load(), "MM")
+ assert.EqualValues(t, 0, sbc1.RollbackPreparedCount.Load(), "RM")
assert.EqualValues(t, 0, sbc0.ConcludeTransactionCount.Load(), "sbc0.ConcludeTransactionCount")
}
@@ -1061,16 +1079,14 @@ func TestTxConnCommit2PCCommitPreparedFail(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCCommitPreparedFail")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
sbc1.MustFailCommitPrepared = 1
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
err := sc.txConn.Commit(ctx, session)
- want := "error: err"
- require.Error(t, err)
- assert.Contains(t, err.Error(), want, "Commit")
+ require.ErrorContains(t, err, "target: TestTxConnCommit2PCCommitPreparedFail.1.primary: error: err")
assert.EqualValues(t, 1, sbc0.CreateTransactionCount.Load(), "sbc0.CreateTransactionCount")
assert.EqualValues(t, 1, sbc1.PrepareCount.Load(), "sbc1.PrepareCount")
assert.EqualValues(t, 1, sbc0.StartCommitCount.Load(), "sbc0.StartCommitCount")
@@ -1083,9 +1099,9 @@ func TestTxConnCommit2PCConcludeTransactionFail(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TestTxConnCommit2PCConcludeTransactionFail")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
sbc0.MustFailConcludeTransaction = 1
session.TransactionMode = vtgatepb.TransactionMode_TWOPC
@@ -1103,9 +1119,9 @@ func TestTxConnRollback(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TxConnRollback")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
require.NoError(t,
sc.txConn.Rollback(ctx, session))
wantSession := vtgatepb.Session{}
@@ -1119,9 +1135,9 @@ func TestTxConnReservedRollback(t *testing.T) {
sc, sbc0, sbc1, rss0, _, rss01 := newTestTxConnEnv(t, ctx, "TxConnReservedRollback")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
require.NoError(t,
sc.txConn.Rollback(ctx, session))
wantSession := vtgatepb.Session{
@@ -1156,9 +1172,9 @@ func TestTxConnReservedRollbackFailure(t *testing.T) {
sc, sbc0, sbc1, rss0, rss1, rss01 := newTestTxConnEnv(t, ctx, "TxConnReservedRollback")
- session := NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
- sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false)
- sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false)
+ session := econtext.NewSafeSession(&vtgatepb.Session{InTransaction: true, InReservedConn: true})
+ sc.ExecuteMultiShard(ctx, nil, rss0, queries, session, false, false, nullResultsObserver{})
+ sc.ExecuteMultiShard(ctx, nil, rss01, twoQueries, session, false, false, nullResultsObserver{})
sbc1.MustFailCodes[vtrpcpb.Code_INVALID_ARGUMENT] = 1
assert.Error(t,
@@ -1435,7 +1451,7 @@ func TestTxConnMultiGoSessions(t *testing.T) {
Keyspace: "0",
},
}}
- err := txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *executeLogger) error {
+ err := txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *econtext.ExecuteLogger) error {
return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "err %s", s.Target.Keyspace)
})
want := "err 0"
@@ -1450,7 +1466,7 @@ func TestTxConnMultiGoSessions(t *testing.T) {
Keyspace: "1",
},
}}
- err = txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *executeLogger) error {
+ err = txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *econtext.ExecuteLogger) error {
return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "err %s", s.Target.Keyspace)
})
want = "err 0\nerr 1"
@@ -1458,7 +1474,7 @@ func TestTxConnMultiGoSessions(t *testing.T) {
wantCode := vtrpcpb.Code_INTERNAL
assert.Equal(t, wantCode, vterrors.Code(err), "error code")
- err = txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *executeLogger) error {
+ err = txc.runSessions(ctx, input, nil, func(ctx context.Context, s *vtgatepb.Session_ShardSession, logger *econtext.ExecuteLogger) error {
return nil
})
require.NoError(t, err)
@@ -1501,7 +1517,7 @@ func TestTxConnAccessModeReset(t *testing.T) {
tcases := []struct {
name string
- f func(ctx context.Context, session *SafeSession) error
+ f func(ctx context.Context, session *econtext.SafeSession) error
}{{
name: "begin-commit",
f: sc.txConn.Commit,
@@ -1518,7 +1534,7 @@ func TestTxConnAccessModeReset(t *testing.T) {
for _, tcase := range tcases {
t.Run(tcase.name, func(t *testing.T) {
- safeSession := NewSafeSession(&vtgatepb.Session{
+ safeSession := econtext.NewSafeSession(&vtgatepb.Session{
Options: &querypb.ExecuteOptions{
TransactionAccessMode: []querypb.ExecuteOptions_TransactionAccessMode{querypb.ExecuteOptions_READ_ONLY},
},
diff --git a/go/vt/vtgate/vcursor_impl_test.go b/go/vt/vtgate/vcursor_impl_test.go
deleted file mode 100644
index b8e4a0d3a0a..00000000000
--- a/go/vt/vtgate/vcursor_impl_test.go
+++ /dev/null
@@ -1,328 +0,0 @@
-package vtgate
-
-import (
- "context"
- "encoding/hex"
- "fmt"
- "strconv"
- "strings"
- "testing"
-
- "github.com/stretchr/testify/require"
-
- "vitess.io/vitess/go/vt/key"
- "vitess.io/vitess/go/vt/sqlparser"
- "vitess.io/vitess/go/vt/srvtopo"
- "vitess.io/vitess/go/vt/topo"
- "vitess.io/vitess/go/vt/vtgate/vindexes"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vschemapb "vitess.io/vitess/go/vt/proto/vschema"
- vtgatepb "vitess.io/vitess/go/vt/proto/vtgate"
-)
-
-var _ VSchemaOperator = (*fakeVSchemaOperator)(nil)
-
-type fakeVSchemaOperator struct {
- vschema *vindexes.VSchema
-}
-
-func (f fakeVSchemaOperator) GetCurrentSrvVschema() *vschemapb.SrvVSchema {
- panic("implement me")
-}
-
-func (f fakeVSchemaOperator) UpdateVSchema(ctx context.Context, ksName string, vschema *vschemapb.SrvVSchema) error {
- panic("implement me")
-}
-
-type fakeTopoServer struct {
-}
-
-// GetTopoServer returns the full topo.Server instance.
-func (f *fakeTopoServer) GetTopoServer() (*topo.Server, error) {
- return nil, nil
-}
-
-// GetSrvKeyspaceNames returns the list of keyspaces served in
-// the provided cell.
-func (f *fakeTopoServer) GetSrvKeyspaceNames(ctx context.Context, cell string, staleOK bool) ([]string, error) {
- return []string{"ks1"}, nil
-}
-
-// GetSrvKeyspace returns the SrvKeyspace for a cell/keyspace.
-func (f *fakeTopoServer) GetSrvKeyspace(ctx context.Context, cell, keyspace string) (*topodatapb.SrvKeyspace, error) {
- zeroHexBytes, _ := hex.DecodeString("")
- eightyHexBytes, _ := hex.DecodeString("80")
- ks := &topodatapb.SrvKeyspace{
- Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{
- {
- ServedType: topodatapb.TabletType_PRIMARY,
- ShardReferences: []*topodatapb.ShardReference{
- {Name: "-80", KeyRange: &topodatapb.KeyRange{Start: zeroHexBytes, End: eightyHexBytes}},
- {Name: "80-", KeyRange: &topodatapb.KeyRange{Start: eightyHexBytes, End: zeroHexBytes}},
- },
- },
- },
- }
- return ks, nil
-}
-
-func (f *fakeTopoServer) WatchSrvKeyspace(ctx context.Context, cell, keyspace string, callback func(*topodatapb.SrvKeyspace, error) bool) {
- ks, err := f.GetSrvKeyspace(ctx, cell, keyspace)
- callback(ks, err)
-}
-
-// WatchSrvVSchema starts watching the SrvVSchema object for
-// the provided cell. It will call the callback when
-// a new value or an error occurs.
-func (f *fakeTopoServer) WatchSrvVSchema(ctx context.Context, cell string, callback func(*vschemapb.SrvVSchema, error) bool) {
-
-}
-
-func TestDestinationKeyspace(t *testing.T) {
- ks1 := &vindexes.Keyspace{
- Name: "ks1",
- Sharded: false,
- }
- ks1Schema := &vindexes.KeyspaceSchema{
- Keyspace: ks1,
- Tables: nil,
- Vindexes: nil,
- Error: nil,
- }
- ks2 := &vindexes.Keyspace{
- Name: "ks2",
- Sharded: false,
- }
- ks2Schema := &vindexes.KeyspaceSchema{
- Keyspace: ks2,
- Tables: nil,
- Vindexes: nil,
- Error: nil,
- }
- vschemaWith2KS := &vindexes.VSchema{
- Keyspaces: map[string]*vindexes.KeyspaceSchema{
- ks1.Name: ks1Schema,
- ks2.Name: ks2Schema,
- }}
-
- vschemaWith1KS := &vindexes.VSchema{
- Keyspaces: map[string]*vindexes.KeyspaceSchema{
- ks1.Name: ks1Schema,
- }}
-
- type testCase struct {
- vschema *vindexes.VSchema
- targetString, qualifier string
- expectedError string
- expectedKeyspace string
- expectedDest key.Destination
- expectedTabletType topodatapb.TabletType
- }
-
- tests := []testCase{{
- vschema: vschemaWith1KS,
- targetString: "",
- qualifier: "",
- expectedKeyspace: ks1.Name,
- expectedDest: nil,
- expectedTabletType: topodatapb.TabletType_PRIMARY,
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1",
- qualifier: "",
- expectedKeyspace: ks1.Name,
- expectedDest: nil,
- expectedTabletType: topodatapb.TabletType_PRIMARY,
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1:-80",
- qualifier: "",
- expectedKeyspace: ks1.Name,
- expectedDest: key.DestinationShard("-80"),
- expectedTabletType: topodatapb.TabletType_PRIMARY,
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1@replica",
- qualifier: "",
- expectedKeyspace: ks1.Name,
- expectedDest: nil,
- expectedTabletType: topodatapb.TabletType_REPLICA,
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1:-80@replica",
- qualifier: "",
- expectedKeyspace: ks1.Name,
- expectedDest: key.DestinationShard("-80"),
- expectedTabletType: topodatapb.TabletType_REPLICA,
- }, {
- vschema: vschemaWith1KS,
- targetString: "",
- qualifier: "ks1",
- expectedKeyspace: ks1.Name,
- expectedDest: nil,
- expectedTabletType: topodatapb.TabletType_PRIMARY,
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks2",
- qualifier: "",
- expectedError: "VT05003: unknown database 'ks2' in vschema",
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks2:-80",
- qualifier: "",
- expectedError: "VT05003: unknown database 'ks2' in vschema",
- }, {
- vschema: vschemaWith1KS,
- targetString: "",
- qualifier: "ks2",
- expectedError: "VT05003: unknown database 'ks2' in vschema",
- }, {
- vschema: vschemaWith2KS,
- targetString: "",
- expectedError: errNoKeyspace.Error(),
- }}
-
- r, _, _, _, _ := createExecutorEnv(t)
- for i, tc := range tests {
- t.Run(strconv.Itoa(i)+tc.targetString, func(t *testing.T) {
- impl, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{TargetString: tc.targetString}), sqlparser.MarginComments{}, r, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, nil, nil, false, querypb.ExecuteOptions_Gen4)
- impl.vschema = tc.vschema
- dest, keyspace, tabletType, err := impl.TargetDestination(tc.qualifier)
- if tc.expectedError == "" {
- require.NoError(t, err)
- require.Equal(t, tc.expectedDest, dest)
- require.Equal(t, tc.expectedKeyspace, keyspace.Name)
- require.Equal(t, tc.expectedTabletType, tabletType)
- } else {
- require.EqualError(t, err, tc.expectedError)
- }
- })
- }
-}
-
-var ks1 = &vindexes.Keyspace{Name: "ks1"}
-var ks1Schema = &vindexes.KeyspaceSchema{Keyspace: ks1}
-var ks2 = &vindexes.Keyspace{Name: "ks2"}
-var ks2Schema = &vindexes.KeyspaceSchema{Keyspace: ks2}
-var vschemaWith1KS = &vindexes.VSchema{
- Keyspaces: map[string]*vindexes.KeyspaceSchema{
- ks1.Name: ks1Schema,
- },
-}
-var vschemaWith2KS = &vindexes.VSchema{
- Keyspaces: map[string]*vindexes.KeyspaceSchema{
- ks1.Name: ks1Schema,
- ks2.Name: ks2Schema,
- }}
-
-func TestSetTarget(t *testing.T) {
- type testCase struct {
- vschema *vindexes.VSchema
- targetString string
- expectedError string
- }
-
- tests := []testCase{{
- vschema: vschemaWith2KS,
- targetString: "",
- }, {
- vschema: vschemaWith2KS,
- targetString: "ks1",
- }, {
- vschema: vschemaWith2KS,
- targetString: "ks2",
- }, {
- vschema: vschemaWith2KS,
- targetString: "ks3",
- expectedError: "VT05003: unknown database 'ks3' in vschema",
- }, {
- vschema: vschemaWith2KS,
- targetString: "ks2@replica",
- expectedError: "can't execute the given command because you have an active transaction",
- }}
-
- r, _, _, _, _ := createExecutorEnv(t)
- for i, tc := range tests {
- t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) {
- vc, _ := newVCursorImpl(NewSafeSession(&vtgatepb.Session{InTransaction: true}), sqlparser.MarginComments{}, r, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, nil, nil, false, querypb.ExecuteOptions_Gen4)
- vc.vschema = tc.vschema
- err := vc.SetTarget(tc.targetString)
- if tc.expectedError == "" {
- require.NoError(t, err)
- require.Equal(t, vc.safeSession.TargetString, tc.targetString)
- } else {
- require.EqualError(t, err, tc.expectedError)
- }
- })
- }
-}
-
-func TestKeyForPlan(t *testing.T) {
- type testCase struct {
- vschema *vindexes.VSchema
- targetString string
- expectedPlanPrefixKey string
- }
-
- tests := []testCase{{
- vschema: vschemaWith1KS,
- targetString: "",
- expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1@replica",
- expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1:-80",
- expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+DestinationShard(-80)+Query:SELECT 1",
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1[deadbeef]",
- expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+KsIDsResolved:80-+Query:SELECT 1",
- }, {
- vschema: vschemaWith1KS,
- targetString: "",
- expectedPlanPrefixKey: "ks1@primary+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
- }, {
- vschema: vschemaWith1KS,
- targetString: "ks1@replica",
- expectedPlanPrefixKey: "ks1@replica+Collate:utf8mb4_0900_ai_ci+Query:SELECT 1",
- }}
-
- r, _, _, _, _ := createExecutorEnv(t)
- for i, tc := range tests {
- t.Run(fmt.Sprintf("%d#%s", i, tc.targetString), func(t *testing.T) {
- ss := NewSafeSession(&vtgatepb.Session{InTransaction: false})
- ss.SetTargetString(tc.targetString)
- vc, err := newVCursorImpl(ss, sqlparser.MarginComments{}, r, nil, &fakeVSchemaOperator{vschema: tc.vschema}, tc.vschema, srvtopo.NewResolver(&fakeTopoServer{}, nil, ""), nil, false, querypb.ExecuteOptions_Gen4)
- require.NoError(t, err)
- vc.vschema = tc.vschema
-
- var buf strings.Builder
- vc.keyForPlan(context.Background(), "SELECT 1", &buf)
- require.Equal(t, tc.expectedPlanPrefixKey, buf.String())
- })
- }
-}
-
-func TestFirstSortedKeyspace(t *testing.T) {
- ks1Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "xks1"}}
- ks2Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "aks2"}}
- ks3Schema := &vindexes.KeyspaceSchema{Keyspace: &vindexes.Keyspace{Name: "aks1"}}
- vschemaWith2KS := &vindexes.VSchema{
- Keyspaces: map[string]*vindexes.KeyspaceSchema{
- ks1Schema.Keyspace.Name: ks1Schema,
- ks2Schema.Keyspace.Name: ks2Schema,
- ks3Schema.Keyspace.Name: ks3Schema,
- }}
-
- r, _, _, _, _ := createExecutorEnv(t)
- vc, err := newVCursorImpl(NewSafeSession(nil), sqlparser.MarginComments{}, r, nil, &fakeVSchemaOperator{vschema: vschemaWith2KS}, vschemaWith2KS, srvtopo.NewResolver(&fakeTopoServer{}, nil, ""), nil, false, querypb.ExecuteOptions_Gen4)
- require.NoError(t, err)
- ks, err := vc.FirstSortedKeyspace()
- require.NoError(t, err)
- require.Equal(t, ks3Schema.Keyspace, ks)
-}
diff --git a/go/vt/vtgate/vindexes/cached_size.go b/go/vt/vtgate/vindexes/cached_size.go
index a97411a6ac8..eeadb69b532 100644
--- a/go/vt/vtgate/vindexes/cached_size.go
+++ b/go/vt/vtgate/vindexes/cached_size.go
@@ -175,6 +175,18 @@ func (cached *Keyspace) CachedSize(alloc bool) int64 {
size += hack.RuntimeAllocSize(int64(len(cached.Name)))
return size
}
+func (cached *LookupCost) CachedSize(alloc bool) int64 {
+ if cached == nil {
+ return int64(0)
+ }
+ size := int64(0)
+ if alloc {
+ size += int64(16)
+ }
+ // field LookupNonUnique *vitess.io/vitess/go/vt/vtgate/vindexes.LookupNonUnique
+ size += cached.LookupNonUnique.CachedSize(true)
+ return size
+}
func (cached *LookupHash) CachedSize(alloc bool) int64 {
if cached == nil {
return int64(0)
diff --git a/go/vt/vtgate/vindexes/lookup_cost.go b/go/vt/vtgate/vindexes/lookup_cost.go
new file mode 100644
index 00000000000..6556032cea5
--- /dev/null
+++ b/go/vt/vtgate/vindexes/lookup_cost.go
@@ -0,0 +1,70 @@
+/*
+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 vindexes
+
+import (
+ "strconv"
+)
+
+var (
+ _ SingleColumn = (*LookupCost)(nil)
+ _ Lookup = (*LookupCost)(nil)
+ _ LookupPlanable = (*LookupCost)(nil)
+)
+
+func init() {
+ Register("lookup_cost", newLookupCost)
+}
+
+const defaultCost = 5
+
+// LookupCost defines a test vindex that uses the cost provided by the user.
+// This is a test vindex.
+type LookupCost struct {
+ *LookupNonUnique
+ cost int
+}
+
+// Cost returns the cost of this vindex as provided.
+func (lc *LookupCost) Cost() int {
+ return lc.cost
+}
+
+func newLookupCost(name string, m map[string]string) (Vindex, error) {
+ lookup, err := newLookup(name, m)
+ if err != nil {
+ return nil, err
+ }
+ cost := getInt(m, "cost", defaultCost)
+ return &LookupCost{
+ LookupNonUnique: lookup.(*LookupNonUnique),
+ cost: cost,
+ }, nil
+
+}
+
+func getInt(m map[string]string, key string, defaultVal int) int {
+ val, ok := m[key]
+ if !ok {
+ return defaultVal
+ }
+ intVal, err := strconv.Atoi(val)
+ if err != nil {
+ return defaultVal
+ }
+ return intVal
+}
diff --git a/go/vt/vtgate/vindexes/region_experimental.go b/go/vt/vtgate/vindexes/region_experimental.go
index c116e9bd84d..c6b867647aa 100644
--- a/go/vt/vtgate/vindexes/region_experimental.go
+++ b/go/vt/vtgate/vindexes/region_experimental.go
@@ -20,7 +20,6 @@ import (
"bytes"
"context"
"encoding/binary"
- "fmt"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/key"
@@ -61,7 +60,7 @@ type RegionExperimental struct {
func newRegionExperimental(name string, m map[string]string) (Vindex, error) {
rbs, ok := m[regionExperimentalParamRegionBytes]
if !ok {
- return nil, vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, fmt.Sprintf("region_experimental missing %s param", regionExperimentalParamRegionBytes))
+ return nil, vterrors.Errorf(vtrpc.Code_INVALID_ARGUMENT, "region_experimental missing %s param", regionExperimentalParamRegionBytes)
}
var rb int
switch rbs {
diff --git a/go/vt/vtgate/vindexes/vschema.go b/go/vt/vtgate/vindexes/vschema.go
index eba3ac49969..3852bbfcde3 100644
--- a/go/vt/vtgate/vindexes/vschema.go
+++ b/go/vt/vtgate/vindexes/vschema.go
@@ -63,6 +63,7 @@ const (
// VSchema represents the denormalized version of SrvVSchema,
// used for building routing plans.
type VSchema struct {
+ MirrorRules map[string]*MirrorRule `json:"mirror_rules"`
RoutingRules map[string]*RoutingRule `json:"routing_rules"`
// globalTables contains the name of all tables in all keyspaces. If the
@@ -79,13 +80,34 @@ type VSchema struct {
created time.Time
}
+// MirrorRule represents one mirror rule.
+type MirrorRule struct {
+ Error error
+ Percent float32 `json:"percent,omitempty"`
+ Table *Table `json:"table,omitempty"`
+}
+
+// MarshalJSON returns a JSON representation of MirrorRule.
+func (mr *MirrorRule) MarshalJSON() ([]byte, error) {
+ if mr.Error != nil {
+ return json.Marshal(mr.Error.Error())
+ }
+ return json.Marshal(struct {
+ Percent float32
+ Table *Table
+ }{
+ Percent: mr.Percent,
+ Table: mr.Table,
+ })
+}
+
// RoutingRule represents one routing rule.
type RoutingRule struct {
Tables []*Table
Error error
}
-// MarshalJSON returns a JSON representation of Column.
+// MarshalJSON returns a JSON representation of RoutingRule.
func (rr *RoutingRule) MarshalJSON() ([]byte, error) {
if rr.Error != nil {
return json.Marshal(rr.Error.Error())
@@ -324,6 +346,7 @@ func (source *Source) String() string {
// BuildVSchema builds a VSchema from a SrvVSchema.
func BuildVSchema(source *vschemapb.SrvVSchema, parser *sqlparser.Parser) (vschema *VSchema) {
vschema = &VSchema{
+ MirrorRules: make(map[string]*MirrorRule),
RoutingRules: make(map[string]*RoutingRule),
globalTables: make(map[string]*Table),
uniqueVindexes: make(map[string]Vindex),
@@ -338,6 +361,7 @@ func BuildVSchema(source *vschemapb.SrvVSchema, parser *sqlparser.Parser) (vsche
buildRoutingRule(source, vschema, parser)
buildShardRoutingRule(source, vschema)
buildKeyspaceRoutingRule(source, vschema)
+ buildMirrorRule(source, vschema, parser)
// Resolve auto-increments after routing rules are built since sequence tables also obey routing rules.
resolveAutoIncrement(source, vschema, parser)
return vschema
@@ -895,7 +919,7 @@ func escapeQualifiedTable(qualifiedTableName string) (string, error) {
}
func extractTableParts(tableName string, allowUnqualified bool) (string, string, error) {
- errMsgFormat := "invalid table name: %s, it must be of the "
+ errMsgFormat := "invalid table name: '%s', it must be of the "
if allowUnqualified {
errMsgFormat = errMsgFormat + "unqualified form or the "
}
@@ -914,7 +938,6 @@ func extractTableParts(tableName string, allowUnqualified bool) (string, string,
}
// Using fmt.Errorf instead of vterrors here because this error is always wrapped in vterrors.
return "", "", fmt.Errorf(errMsgFormat, tableName)
-
}
func parseTable(tableName string) (sqlparser.TableName, error) {
@@ -963,7 +986,7 @@ outer:
toTable, err = escapeQualifiedTable(toTable)
if err != nil {
vschema.RoutingRules[rule.FromTable] = &RoutingRule{
- Error: vterrors.Errorf(
+ Error: vterrors.New(
vtrpcpb.Code_INVALID_ARGUMENT,
err.Error(),
),
@@ -972,7 +995,6 @@ outer:
}
toKeyspace, toTableName, err := parser.ParseTable(toTable)
-
if err != nil {
vschema.RoutingRules[rule.FromTable] = &RoutingRule{
Error: err,
@@ -1025,6 +1047,216 @@ func buildKeyspaceRoutingRule(source *vschemapb.SrvVSchema, vschema *VSchema) {
vschema.KeyspaceRoutingRules = rulesMap
}
+func buildMirrorRule(source *vschemapb.SrvVSchema, vschema *VSchema, parser *sqlparser.Parser) {
+ if source.MirrorRules == nil {
+ return
+ }
+
+ // Used to validate no mirror chains exist.
+ fromTableKeyspaces := make(map[string]string)
+ toKeyspaces := make(map[string]struct{})
+
+ for _, rule := range source.MirrorRules.Rules {
+ toTable := rule.ToTable
+
+ //
+ // Forbid duplicate FromTables expressions.
+ //
+
+ if _, ok := vschema.MirrorRules[rule.FromTable]; ok {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_ALREADY_EXISTS,
+ "from table: duplicate rule for entry '%s'",
+ rule.FromTable,
+ ),
+ }
+ continue
+ }
+
+ //
+ // Parse and validate FromTable.
+ //
+
+ // Separate tablet-type from rest of FromTable.
+ fromTableParts := strings.Split(rule.FromTable, "@")
+ if len(fromTableParts) == 0 {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "from table: invalid table name: '%s'",
+ rule.FromTable,
+ ),
+ }
+ }
+
+ // Escape and parse the FromTable, without table-type specifier.
+ fromTable, err := escapeQualifiedTable(fromTableParts[0])
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "from table: %s",
+ err.Error(),
+ ),
+ }
+ continue
+ }
+ fromKeyspace, fromTableName, err := parser.ParseTable(fromTable)
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "from table: invalid table name: '%s'",
+ err.Error(),
+ ),
+ }
+ continue
+ }
+
+ // Find the from table.
+ _, err = vschema.FindTable(fromKeyspace, fromTableName)
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "from table: %s",
+ err.Error(),
+ ),
+ }
+ continue
+ }
+
+ // Validate the table-type, if specified.
+ if len(fromTableParts) > 1 {
+ fromTabletTypeSuffix := "@" + strings.Join(fromTableParts[1:], "")
+ var ok bool
+ for _, tabletTypeSuffix := range TabletTypeSuffix {
+ if tabletTypeSuffix == fromTabletTypeSuffix {
+ ok = true
+ break
+ }
+ }
+ if !ok {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "from table: invalid tablet type: '%s'",
+ rule.FromTable,
+ ),
+ }
+ continue
+ }
+ }
+
+ //
+ // Parse and validate ToTable.
+ //
+
+ // Forbid tablet-type specifier.
+ toTableParts := strings.Split(toTable, "@")
+ if len(toTableParts) != 1 || toTableParts[0] == "@" {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "to table: tablet type may not be specified: '%s'",
+ rule.ToTable,
+ ),
+ }
+ continue
+ }
+
+ // Escape and parse the table.
+ toTable, err = escapeQualifiedTable(toTable)
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "to table: %s",
+ err.Error(),
+ ),
+ }
+ continue
+ }
+ toKeyspace, toTableName, err := parser.ParseTable(toTable)
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "to table: invalid table name: '%s'",
+ rule.ToTable,
+ ),
+ }
+ continue
+ }
+
+ // Forbid self-mirroring.
+ if fromKeyspace == toKeyspace {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "to table: cannot reside in same keyspace as from table",
+ ),
+ }
+ continue
+ }
+
+ //
+ // Find table in VSchema.
+ //
+
+ t, err := vschema.FindTable(toKeyspace, toTableName)
+ if err != nil {
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Error: vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "to table: %s",
+ err.Error(),
+ ),
+ }
+ continue
+ }
+
+ //
+ // Return non-error mirror rule.
+ //
+
+ vschema.MirrorRules[rule.FromTable] = &MirrorRule{
+ Table: t,
+ Percent: rule.Percent,
+ }
+
+ //
+ // Save some info for validating no mirror chains exist
+ //
+
+ fromTableKeyspaces[rule.FromTable] = fromKeyspace
+ toKeyspaces[toKeyspace] = struct{}{}
+ }
+
+ // Forbid mirror chains. Keyspaces which are the target of a mirror rule
+ // may not be the source of another.
+ for fromTable, rule := range vschema.MirrorRules {
+ if rule.Error != nil {
+ continue
+ }
+ fromKeyspace, ok := fromTableKeyspaces[fromTable]
+ if !ok {
+ rule.Error = vterrors.Errorf(
+ vtrpcpb.Code_INTERNAL,
+ "[BUG] from table: failed to determine keyspace",
+ )
+ continue
+ }
+ if _, ok := toKeyspaces[fromKeyspace]; ok {
+ rule.Error = vterrors.Errorf(
+ vtrpcpb.Code_INVALID_ARGUMENT,
+ "mirror chaining is not allowed",
+ )
+ }
+ }
+}
+
// FindTable returns a pointer to the Table. If a keyspace is specified, only tables
// from that keyspace are searched. If the specified keyspace is unsharded
// and no tables matched, it's considered valid: FindTable will construct a table
@@ -1325,6 +1557,28 @@ func (vschema *VSchema) GetAggregateUDFs() (udfs []string) {
return
}
+// FindMirrorRule finds a mirror rule from the keyspace, table name and
+// tablet type.
+func (vschema *VSchema) FindMirrorRule(keyspace, tablename string, tabletType topodatapb.TabletType) (*MirrorRule, error) {
+ qualified := tablename
+ if keyspace != "" {
+ qualified = keyspace + "." + tablename
+ }
+ fqtn := qualified + TabletTypeSuffix[tabletType]
+ // First look for a fully qualified table name: keyspace.table@tablet_type.
+ // Then look for one without tablet type: keyspace.table.
+ for _, name := range []string{fqtn, qualified} {
+ mr, ok := vschema.MirrorRules[name]
+ if ok {
+ if mr.Error != nil {
+ return nil, mr.Error
+ }
+ return mr, nil
+ }
+ }
+ return nil, nil
+}
+
// ByCost provides the interface needed for ColumnVindexes to
// be sorted by cost order.
type ByCost []*ColumnVindex
@@ -1392,7 +1646,7 @@ func ChooseVindexForType(typ querypb.Type) (string, error) {
// FindBestColVindex finds the best ColumnVindex for VReplication.
func FindBestColVindex(table *Table) (*ColumnVindex, error) {
- if table.ColumnVindexes == nil || len(table.ColumnVindexes) == 0 {
+ if len(table.ColumnVindexes) == 0 {
return nil, vterrors.Errorf(
vtrpcpb.Code_INVALID_ARGUMENT,
"table %s has no vindex",
diff --git a/go/vt/vtgate/vindexes/vschema_test.go b/go/vt/vtgate/vindexes/vschema_test.go
index 7761b6ae8ab..f9bcf43ddaa 100644
--- a/go/vt/vtgate/vindexes/vschema_test.go
+++ b/go/vt/vtgate/vindexes/vschema_test.go
@@ -21,6 +21,7 @@ import (
"encoding/json"
"errors"
"fmt"
+ "os"
"reflect"
"strings"
"testing"
@@ -838,6 +839,7 @@ func TestVSchemaRoutingRules(t *testing.T) {
Keyspace: ks2,
}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{
"rt1": {
Error: errors.New("table rt1 has more than one target: [ks1.t1 ks2.t2]"),
@@ -852,10 +854,10 @@ func TestVSchemaRoutingRules(t *testing.T) {
Error: errors.New("duplicate rule for entry dup"),
},
"badname": {
- Error: errors.New("invalid table name: t1.t2.t3, it must be of the qualified form . (dots are not allowed in either name)"),
+ Error: errors.New("invalid table name: 't1.t2.t3', it must be of the qualified form . (dots are not allowed in either name)"),
},
"unqualified": {
- Error: errors.New("invalid table name: t1, it must be of the qualified form . (dots are not allowed in either name)"),
+ Error: errors.New("invalid table name: 't1', it must be of the qualified form . (dots are not allowed in either name)"),
},
"badkeyspace": {
Error: errors.New("VT05003: unknown database 'ks3' in vschema"),
@@ -897,6 +899,282 @@ func TestVSchemaRoutingRules(t *testing.T) {
assert.Equal(t, string(wantb), string(gotb), string(gotb))
}
+func TestVSchemaMirrorRules(t *testing.T) {
+ input := vschemapb.SrvVSchema{
+ MirrorRules: &vschemapb.MirrorRules{
+ Rules: []*vschemapb.MirrorRule{
+ // Empty FromTable not allowed.
+ {
+ FromTable: "",
+ ToTable: "ks2.ks2t1",
+ },
+ // Invalid FromTable, needs to be .[@].
+ {
+ FromTable: "ks1",
+ ToTable: "ks2.ks2t1",
+ },
+ // Invalid ToTable, needs to be ..
+ {
+ FromTable: "ks1.ks1t1",
+ ToTable: "ks2",
+ },
+ // Invalid ToTable, needs to be ..
+ {
+ FromTable: "ks1.ks1t2",
+ ToTable: "ks2.ks2t2.c",
+ },
+ // OK, unsharded => unsharded.
+ {
+ FromTable: "ks1.ks1t3",
+ ToTable: "ks2.ks2t3",
+ Percent: 50,
+ },
+ // Invalid ToTable, needs to be ..
+ {
+ FromTable: "ks1.ks1t4",
+ ToTable: "ks2.ks2t4@replica",
+ },
+ // OK, unsharded@tablet-type => unsharded.
+ {
+ FromTable: "ks1.ks1t5@replica",
+ ToTable: "ks2.ks2t5",
+ },
+ // Invalid FromTable tablet type..
+ {
+ FromTable: "ks1.ks1t6@stone",
+ ToTable: "ks2.ks2t6",
+ },
+ // OK, sharded => sharded.
+ {
+ FromTable: "ks3.ks3t1",
+ ToTable: "ks4.ks4t1",
+ Percent: 50,
+ },
+ // OK, unsharded => sharded.
+ {
+ FromTable: "ks1.ks1t7",
+ ToTable: "ks4.ks4t1",
+ Percent: 50,
+ },
+ // Destination sharded table must be defined in VSchema.
+ {
+ FromTable: "ks1.ks1t8",
+ ToTable: "ks4.ks4t2",
+ Percent: 50,
+ },
+ // Source sharded table must be defined in VSchema.
+ {
+ FromTable: "ks3.ks3t2",
+ ToTable: "ks4.ks4t1",
+ Percent: 50,
+ },
+ // Keyspaces that are the target of a rule may not be the
+ // source of another.
+ {
+ FromTable: "ks2.ks2t9",
+ ToTable: "ks4.ks4t1",
+ Percent: 50,
+ },
+ },
+ },
+ RoutingRules: &vschemapb.RoutingRules{},
+ Keyspaces: map[string]*vschemapb.Keyspace{
+ "ks1": {
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Tables: map[string]*vschemapb.Table{},
+ },
+ "ks2": {
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Tables: map[string]*vschemapb.Table{},
+ },
+ "ks3": {
+ Sharded: true,
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Vindexes: map[string]*vschemapb.Vindex{
+ "stfu1": {
+ Type: "stfu",
+ },
+ },
+ Tables: map[string]*vschemapb.Table{
+ "ks3t1": {
+ ColumnVindexes: []*vschemapb.ColumnVindex{
+ {
+ Column: "id",
+ Name: "stfu1",
+ },
+ },
+ },
+ },
+ },
+ "ks4": {
+ Sharded: true,
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Vindexes: map[string]*vschemapb.Vindex{
+ "stfu1": {
+ Type: "stfu",
+ },
+ },
+ Tables: map[string]*vschemapb.Table{
+ "ks4t1": {
+ ColumnVindexes: []*vschemapb.ColumnVindex{
+ {
+ Column: "id",
+ Name: "stfu1",
+ },
+ },
+ },
+ },
+ },
+ },
+ }
+ got := BuildVSchema(&input, sqlparser.NewTestParser())
+
+ ks1 := &Keyspace{
+ Name: "ks1",
+ Sharded: false,
+ }
+ ks2 := &Keyspace{
+ Name: "ks2",
+ Sharded: false,
+ }
+ ks3 := &Keyspace{
+ Name: "ks3",
+ Sharded: true,
+ }
+ ks4 := &Keyspace{
+ Name: "ks4",
+ Sharded: true,
+ }
+
+ vindex1 := &stFU{
+ name: "stfu1",
+ }
+
+ ks3t1 := &Table{
+ Name: sqlparser.NewIdentifierCS("ks3t1"),
+ Keyspace: ks3,
+ ColumnVindexes: []*ColumnVindex{{
+ Columns: []sqlparser.IdentifierCI{sqlparser.NewIdentifierCI("id")},
+ Type: "stfu",
+ Name: "stfu1",
+ Vindex: vindex1,
+ isUnique: vindex1.IsUnique(),
+ cost: vindex1.Cost(),
+ }},
+ }
+ ks3t1.Ordered = []*ColumnVindex{
+ ks3t1.ColumnVindexes[0],
+ }
+
+ ks4t1 := &Table{
+ Name: sqlparser.NewIdentifierCS("ks4t1"),
+ Keyspace: ks4,
+ ColumnVindexes: []*ColumnVindex{{
+ Columns: []sqlparser.IdentifierCI{sqlparser.NewIdentifierCI("id")},
+ Type: "stfu",
+ Name: "stfu1",
+ Vindex: vindex1,
+ isUnique: vindex1.IsUnique(),
+ cost: vindex1.Cost(),
+ }},
+ }
+ ks4t1.Ordered = []*ColumnVindex{
+ ks4t1.ColumnVindexes[0],
+ }
+
+ want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{
+ "": {
+ Error: errors.New("from table: invalid table name: '', it must be of the qualified form . (dots are not allowed in either name)"),
+ },
+ "ks1": {
+ Error: errors.New("from table: invalid table name: 'ks1', it must be of the qualified form . (dots are not allowed in either name)"),
+ },
+ "ks1.ks1t1": {
+ Error: errors.New("to table: invalid table name: 'ks2', it must be of the qualified form . (dots are not allowed in either name)"),
+ },
+ "ks1.ks1t2": {
+ Error: errors.New("to table: invalid table name: 'ks2.ks2t2.c', it must be of the qualified form . (dots are not allowed in either name)"),
+ },
+ "ks1.ks1t3": {
+ Table: &Table{
+ Name: sqlparser.NewIdentifierCS("ks2t3"),
+ },
+ Percent: 50,
+ },
+ "ks1.ks1t4": {
+ Error: errors.New("to table: tablet type may not be specified: 'ks2.ks2t4@replica'"),
+ },
+ "ks1.ks1t5@replica": {
+ Table: &Table{
+ Name: sqlparser.NewIdentifierCS("ks2t5"),
+ },
+ },
+ "ks1.ks1t6@stone": {
+ Error: errors.New("from table: invalid tablet type: 'ks1.ks1t6@stone'"),
+ },
+ "ks3.ks3t1": {
+ Table: ks4t1,
+ Percent: 50,
+ },
+ "ks1.ks1t7": {
+ Table: ks4t1,
+ Percent: 50,
+ },
+ "ks1.ks1t8": {
+ Error: errors.New("to table: table ks4t2 not found"),
+ },
+ "ks3.ks3t2": {
+ Error: errors.New("from table: table ks3t2 not found"),
+ },
+ "ks2.ks2t9": {
+ Error: errors.New("mirror chaining is not allowed"),
+ },
+ },
+ RoutingRules: map[string]*RoutingRule{},
+ Keyspaces: map[string]*KeyspaceSchema{
+ "ks1": {
+ Keyspace: ks1,
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Tables: map[string]*Table{},
+ Vindexes: map[string]Vindex{},
+ },
+ "ks2": {
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Keyspace: ks2,
+ Tables: map[string]*Table{},
+ Vindexes: map[string]Vindex{},
+ },
+ "ks3": {
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Keyspace: ks3,
+ Tables: map[string]*Table{
+ "ks3t1": ks3t1,
+ },
+ Vindexes: map[string]Vindex{
+ "stfu1": vindex1,
+ },
+ },
+ "ks4": {
+ ForeignKeyMode: vschemapb.Keyspace_unmanaged,
+ Keyspace: ks4,
+ Tables: map[string]*Table{
+ "ks4t1": ks4t1,
+ },
+ Vindexes: map[string]Vindex{
+ "stfu1": vindex1,
+ },
+ },
+ },
+ }
+
+ gotb, err := json.MarshalIndent(got, "", " ")
+ assert.NoError(t, err)
+ wantb, err := json.MarshalIndent(want, "", " ")
+ assert.NoError(t, err)
+ assert.Equal(t, string(wantb), string(gotb), string(gotb))
+}
+
func TestChooseVindexForType(t *testing.T) {
testcases := []struct {
in querypb.Type
@@ -1247,6 +1525,7 @@ func TestShardedVSchemaMultiColumnVindex(t *testing.T) {
t1.ColumnVindexes[0],
}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"t1": t1,
@@ -1323,6 +1602,7 @@ func TestShardedVSchemaNotOwned(t *testing.T) {
t1.ColumnVindexes[1],
t1.ColumnVindexes[0]}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"t1": t1,
@@ -1430,6 +1710,7 @@ func TestBuildVSchemaDupSeq(t *testing.T) {
Keyspace: ksb,
Type: "sequence"}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"t1": nil,
@@ -1491,6 +1772,7 @@ func TestBuildVSchemaDupTable(t *testing.T) {
Keyspace: ksb,
}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"t1": nil,
@@ -1620,6 +1902,7 @@ func TestBuildVSchemaDupVindex(t *testing.T) {
t2.ColumnVindexes[0],
}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"t1": nil,
@@ -2206,6 +2489,7 @@ func TestSequence(t *testing.T) {
t2.ColumnVindexes[0],
}
want := &VSchema{
+ MirrorRules: map[string]*MirrorRule{},
RoutingRules: map[string]*RoutingRule{},
globalTables: map[string]*Table{
"seq": seq,
@@ -3268,6 +3552,20 @@ func TestFindTableWithSequences(t *testing.T) {
}
}
+func TestGlobalTables(t *testing.T) {
+ input, err := os.ReadFile("../planbuilder/testdata/vschemas/schema.json")
+ require.NoError(t, err)
+
+ var vs vschemapb.SrvVSchema
+ err = json2.UnmarshalPB(input, &vs)
+ require.NoError(t, err)
+
+ got := BuildVSchema(&vs, sqlparser.NewTestParser())
+ tbl, err := got.findGlobalTable("user", false)
+ require.NoError(t, err)
+ assert.NotNil(t, tbl)
+}
+
func vindexNames(vindexes []*ColumnVindex) (result []string) {
for _, vindex := range vindexes {
result = append(result, vindex.Name)
diff --git a/go/vt/vtgate/viperconfig.go b/go/vt/vtgate/viperconfig.go
new file mode 100644
index 00000000000..ec77ff62d4f
--- /dev/null
+++ b/go/vt/vtgate/viperconfig.go
@@ -0,0 +1,16 @@
+package vtgate
+
+import "vitess.io/vitess/go/viperutil"
+
+type dynamicViperConfig struct {
+ onlineDDL viperutil.Value[bool]
+ directDDL viperutil.Value[bool]
+}
+
+func (d *dynamicViperConfig) OnlineEnabled() bool {
+ return d.onlineDDL.Get()
+}
+
+func (d *dynamicViperConfig) DirectEnabled() bool {
+ return d.directDDL.Get()
+}
diff --git a/go/vt/vtgate/vschema_manager.go b/go/vt/vtgate/vschema_manager.go
index 2b6761f4a8e..62ea2cd3455 100644
--- a/go/vt/vtgate/vschema_manager.go
+++ b/go/vt/vtgate/vschema_manager.go
@@ -33,8 +33,6 @@ import (
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
)
-var _ VSchemaOperator = (*VSchemaManager)(nil)
-
// VSchemaManager is used to watch for updates to the vschema and to implement
// the DDL commands to add / remove vindexes
type VSchemaManager struct {
diff --git a/go/vt/vtgate/vschema_manager_test.go b/go/vt/vtgate/vschema_manager_test.go
index 8dfb889df0d..8db0c2df05b 100644
--- a/go/vt/vtgate/vschema_manager_test.go
+++ b/go/vt/vtgate/vschema_manager_test.go
@@ -234,6 +234,7 @@ func TestVSchemaUpdate(t *testing.T) {
},
},
expected: &vindexes.VSchema{
+ MirrorRules: map[string]*vindexes.MirrorRule{},
RoutingRules: map[string]*vindexes.RoutingRule{},
Keyspaces: map[string]*vindexes.KeyspaceSchema{
"ks": {
@@ -499,6 +500,7 @@ func TestVSchemaUDFsUpdate(t *testing.T) {
}, nil)
utils.MustMatchFn(".globalTables", ".uniqueVindexes")(t, &vindexes.VSchema{
+ MirrorRules: map[string]*vindexes.MirrorRule{},
RoutingRules: map[string]*vindexes.RoutingRule{},
Keyspaces: map[string]*vindexes.KeyspaceSchema{
"ks": {
@@ -821,6 +823,7 @@ func TestVSchemaUpdateWithFKReferenceToInternalTables(t *testing.T) {
}, nil)
utils.MustMatchFn(".globalTables", ".uniqueVindexes")(t, &vindexes.VSchema{
+ MirrorRules: map[string]*vindexes.MirrorRule{},
RoutingRules: map[string]*vindexes.RoutingRule{},
Keyspaces: map[string]*vindexes.KeyspaceSchema{
"ks": {
@@ -870,6 +873,7 @@ func makeTestVSchema(ks string, sharded bool, tbls map[string]*vindexes.Table) *
func makeTestEmptyVSchema() *vindexes.VSchema {
return &vindexes.VSchema{
+ MirrorRules: map[string]*vindexes.MirrorRule{},
RoutingRules: map[string]*vindexes.RoutingRule{},
Keyspaces: map[string]*vindexes.KeyspaceSchema{},
}
diff --git a/go/vt/vtgate/vschemaacl/vschemaacl.go b/go/vt/vtgate/vschemaacl/vschemaacl.go
index 5345d1437fc..08f6c2b0cd4 100644
--- a/go/vt/vtgate/vschemaacl/vschemaacl.go
+++ b/go/vt/vtgate/vschemaacl/vschemaacl.go
@@ -18,26 +18,67 @@ package vschemaacl
import (
"strings"
- "sync"
"github.com/spf13/pflag"
+ "github.com/spf13/viper"
- "vitess.io/vitess/go/vt/servenv"
-
+ "vitess.io/vitess/go/viperutil"
querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/servenv"
)
-var (
- // AuthorizedDDLUsers specifies the users that can perform ddl operations
- AuthorizedDDLUsers string
-
- // ddlAllowAll is true if the special value of "*" was specified
+type authorizedDDLUsers struct {
allowAll bool
+ acl map[string]struct{}
+ source string
+}
+
+func NewAuthorizedDDLUsers(users string) *authorizedDDLUsers {
+ acl := make(map[string]struct{})
+ allowAll := false
+
+ switch users {
+ case "":
+ case "%":
+ allowAll = true
+ default:
+ for _, user := range strings.Split(users, ",") {
+ user = strings.TrimSpace(user)
+ acl[user] = struct{}{}
+ }
+ }
+
+ return &authorizedDDLUsers{
+ allowAll: allowAll,
+ acl: acl,
+ source: users,
+ }
+}
- // ddlACL contains a set of allowed usernames
- acl map[string]struct{}
+func (a *authorizedDDLUsers) String() string {
+ return a.source
+}
- initMu sync.Mutex
+var (
+ // AuthorizedDDLUsers specifies the users that can perform ddl operations
+ AuthorizedDDLUsers = viperutil.Configure(
+ "vschema_ddl_authorized_users",
+ viperutil.Options[*authorizedDDLUsers]{
+ FlagName: "vschema_ddl_authorized_users",
+ Default: &authorizedDDLUsers{},
+ Dynamic: true,
+ GetFunc: func(v *viper.Viper) func(key string) *authorizedDDLUsers {
+ return func(key string) *authorizedDDLUsers {
+ newVal := v.GetString(key)
+ curVal, ok := v.Get(key).(*authorizedDDLUsers)
+ if ok && newVal == curVal.source {
+ return curVal
+ }
+ return NewAuthorizedDDLUsers(newVal)
+ }
+ },
+ },
+ )
)
// RegisterSchemaACLFlags installs log flags on the given FlagSet.
@@ -46,7 +87,8 @@ var (
// calls this function, or call this function directly before parsing
// command-line arguments.
func RegisterSchemaACLFlags(fs *pflag.FlagSet) {
- fs.StringVar(&AuthorizedDDLUsers, "vschema_ddl_authorized_users", AuthorizedDDLUsers, "List of users authorized to execute vschema ddl operations, or '%' to allow all users.")
+ fs.String("vschema_ddl_authorized_users", "", "List of users authorized to execute vschema ddl operations, or '%' to allow all users.")
+ viperutil.BindFlags(fs, AuthorizedDDLUsers)
}
func init() {
@@ -55,33 +97,14 @@ func init() {
}
}
-// Init parses the users option and sets allowAll / acl accordingly
-func Init() {
- initMu.Lock()
- defer initMu.Unlock()
- acl = make(map[string]struct{})
- allowAll = false
-
- if AuthorizedDDLUsers == "%" {
- allowAll = true
- return
- } else if AuthorizedDDLUsers == "" {
- return
- }
-
- for _, user := range strings.Split(AuthorizedDDLUsers, ",") {
- user = strings.TrimSpace(user)
- acl[user] = struct{}{}
- }
-}
-
// Authorized returns true if the given caller is allowed to execute vschema operations
func Authorized(caller *querypb.VTGateCallerID) bool {
- if allowAll {
+ users := AuthorizedDDLUsers.Get()
+ if users.allowAll {
return true
}
user := caller.GetUsername()
- _, ok := acl[user]
+ _, ok := users.acl[user]
return ok
}
diff --git a/go/vt/vtgate/vschemaacl/vschemaacl_test.go b/go/vt/vtgate/vschemaacl/vschemaacl_test.go
index faa2dbfc294..cfd1de705af 100644
--- a/go/vt/vtgate/vschemaacl/vschemaacl_test.go
+++ b/go/vt/vtgate/vschemaacl/vschemaacl_test.go
@@ -35,8 +35,7 @@ func TestVschemaAcl(t *testing.T) {
}
// Test wildcard
- AuthorizedDDLUsers = "%"
- Init()
+ AuthorizedDDLUsers.Set(NewAuthorizedDDLUsers("%"))
if !Authorized(&redUser) {
t.Errorf("user should be authorized")
@@ -46,8 +45,7 @@ func TestVschemaAcl(t *testing.T) {
}
// Test user list
- AuthorizedDDLUsers = "oneUser, twoUser, redUser, blueUser"
- Init()
+ AuthorizedDDLUsers.Set(NewAuthorizedDDLUsers("oneUser, twoUser, redUser, blueUser"))
if !Authorized(&redUser) {
t.Errorf("user should be authorized")
@@ -57,8 +55,7 @@ func TestVschemaAcl(t *testing.T) {
}
// Revert to baseline state for other tests
- AuthorizedDDLUsers = ""
- Init()
+ AuthorizedDDLUsers.Set(NewAuthorizedDDLUsers(""))
// By default no users are allowed in
if Authorized(&redUser) {
diff --git a/go/vt/vtgate/vstream_manager.go b/go/vt/vtgate/vstream_manager.go
index e0d195853cf..beb556f21f5 100644
--- a/go/vt/vtgate/vstream_manager.go
+++ b/go/vt/vtgate/vstream_manager.go
@@ -61,6 +61,10 @@ const maxSkewTimeoutSeconds = 10 * 60
// for a vstream
const tabletPickerContextTimeout = 90 * time.Second
+// stopOnReshardDelay is how long we wait, at a minimum, after sending a reshard journal event before
+// ending the stream from the tablet.
+const stopOnReshardDelay = 500 * time.Millisecond
+
// vstream contains the metadata for one VStream request.
type vstream struct {
// mu protects parts of vgtid, the semantics of a send, and journaler.
@@ -99,6 +103,10 @@ type vstream struct {
// default behavior is to automatically migrate the resharded streams from the old to the new shards
stopOnReshard bool
+ // This flag is set by the client, default is false.
+ // If true then the reshard journal events are sent in the stream irrespective of the stopOnReshard flag.
+ includeReshardJournalEvents bool
+
// mutex used to synchronize access to skew detection parameters
skewMu sync.Mutex
// channel is created whenever there is a skew detected. closing it implies the current skew has been fixed
@@ -122,6 +130,8 @@ type vstream struct {
ts *topo.Server
tabletPickerOptions discovery.TabletPickerOptions
+
+ flags *vtgatepb.VStreamFlags
}
type journalEvent struct {
@@ -163,26 +173,28 @@ func (vsm *vstreamManager) VStream(ctx context.Context, tabletType topodatapb.Ta
return fmt.Errorf("unable to get topo server")
}
vs := &vstream{
- vgtid: vgtid,
- tabletType: tabletType,
- optCells: flags.Cells,
- filter: filter,
- send: send,
- resolver: vsm.resolver,
- journaler: make(map[int64]*journalEvent),
- minimizeSkew: flags.GetMinimizeSkew(),
- stopOnReshard: flags.GetStopOnReshard(),
- skewTimeoutSeconds: maxSkewTimeoutSeconds,
- timestamps: make(map[string]int64),
- vsm: vsm,
- eventCh: make(chan []*binlogdatapb.VEvent),
- heartbeatInterval: flags.GetHeartbeatInterval(),
- ts: ts,
- copyCompletedShard: make(map[string]struct{}),
+ vgtid: vgtid,
+ tabletType: tabletType,
+ optCells: flags.Cells,
+ filter: filter,
+ send: send,
+ resolver: vsm.resolver,
+ journaler: make(map[int64]*journalEvent),
+ minimizeSkew: flags.GetMinimizeSkew(),
+ stopOnReshard: flags.GetStopOnReshard(),
+ includeReshardJournalEvents: flags.GetIncludeReshardJournalEvents(),
+ skewTimeoutSeconds: maxSkewTimeoutSeconds,
+ timestamps: make(map[string]int64),
+ vsm: vsm,
+ eventCh: make(chan []*binlogdatapb.VEvent),
+ heartbeatInterval: flags.GetHeartbeatInterval(),
+ ts: ts,
+ copyCompletedShard: make(map[string]struct{}),
tabletPickerOptions: discovery.TabletPickerOptions{
CellPreference: flags.GetCellPreference(),
TabletOrder: flags.GetTabletOrder(),
},
+ flags: flags,
}
return vs.stream(ctx)
}
@@ -269,7 +281,7 @@ func (vsm *vstreamManager) resolveParams(ctx context.Context, tabletType topodat
}
}
- //TODO add tablepk validations
+ // TODO add tablepk validations
return newvgtid, filter, flags, nil
}
@@ -575,15 +587,24 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha
})
}()
- log.Infof("Starting to vstream from %s", tablet.Alias.String())
+ var options *binlogdatapb.VStreamOptions
+ const SidecarDBHeartbeatTableName = "heartbeat"
+ if vs.flags.GetStreamKeyspaceHeartbeats() {
+ options = &binlogdatapb.VStreamOptions{
+ InternalTables: []string{SidecarDBHeartbeatTableName},
+ }
+ }
+
// Safe to access sgtid.Gtid here (because it can't change until streaming begins).
req := &binlogdatapb.VStreamRequest{
Target: target,
Position: sgtid.Gtid,
Filter: vs.filter,
TableLastPKs: sgtid.TablePKs,
+ Options: options,
}
var vstreamCreatedOnce sync.Once
+ log.Infof("Starting to vstream from %s, with req %+v", topoproto.TabletAliasString(tablet.Alias), req)
err = tabletConn.VStream(ctx, req, func(events []*binlogdatapb.VEvent) error {
// We received a valid event. Reset error count.
errCount = 0
@@ -608,7 +629,7 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha
}
sendevents := make([]*binlogdatapb.VEvent, 0, len(events))
- for _, event := range events {
+ for i, event := range events {
switch event.Type {
case binlogdatapb.VEventType_FIELD:
// Update table names and send.
@@ -658,12 +679,23 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha
if err := vs.alignStreams(ctx, event, sgtid.Keyspace, sgtid.Shard); err != nil {
return err
}
-
case binlogdatapb.VEventType_JOURNAL:
journal := event.Journal
- // Journal events are not sent to clients by default, but only when StopOnReshard is set
- if vs.stopOnReshard && journal.MigrationType == binlogdatapb.MigrationType_SHARDS {
+ // Journal events are not sent to clients by default, but only when
+ // IncludeReshardJournalEvents or StopOnReshard is set.
+ if (vs.includeReshardJournalEvents || vs.stopOnReshard) &&
+ journal.MigrationType == binlogdatapb.MigrationType_SHARDS {
sendevents = append(sendevents, event)
+ // Read any subsequent events until we get the VGTID->COMMIT events that
+ // always follow the JOURNAL event which is generated as a result of
+ // an autocommit insert into the _vt.resharding_journal table on the
+ // tablet.
+ for j := i + 1; j < len(events); j++ {
+ sendevents = append(sendevents, events[j])
+ if events[j].Type == binlogdatapb.VEventType_COMMIT {
+ break
+ }
+ }
eventss = append(eventss, sendevents)
if err := vs.sendAll(ctx, sgtid, eventss); err != nil {
return err
@@ -676,12 +708,28 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha
return err
}
if je != nil {
- // Wait till all other participants converge and return EOF.
+ var endTimer *time.Timer
+ if vs.stopOnReshard {
+ // We're going to be ending the tablet stream, along with the VStream, so
+ // we ensure a reasonable minimum amount of time is alloted for clients
+ // to Recv the journal event before the VStream's context is cancelled
+ // (which would cause the grpc SendMsg or RecvMsg to fail). If the client
+ // doesn't Recv the journal event before the VStream ends then they'll
+ // have to resume from the last ShardGtid they received before the
+ // journal event.
+ endTimer = time.NewTimer(stopOnReshardDelay)
+ defer endTimer.Stop()
+ }
+ // Wait until all other participants converge and then return EOF after
+ // any minimum delay has passed.
journalDone = je.done
select {
case <-ctx.Done():
return ctx.Err()
case <-journalDone:
+ if endTimer != nil {
+ <-endTimer.C
+ }
return io.EOF
}
}
@@ -690,7 +738,6 @@ func (vs *vstream) streamFromTablet(ctx context.Context, sgtid *binlogdatapb.Sha
}
lag := event.CurrentTime/1e9 - event.Timestamp
vs.vsm.vstreamsLag.Set(labels, lag)
-
}
if len(sendevents) != 0 {
eventss = append(eventss, sendevents)
@@ -954,6 +1001,9 @@ func (vs *vstream) keyspaceHasBeenResharded(ctx context.Context, keyspace string
return false, err
}
+ vs.mu.Lock()
+ defer vs.mu.Unlock()
+
// First check the typical case, where the VGTID shards match the serving shards.
// In that case it's NOT possible that an applicable reshard has happened because
// the VGTID contains shards that are all serving.
diff --git a/go/vt/vtgate/vstream_manager_test.go b/go/vt/vtgate/vstream_manager_test.go
index e51bd2785dd..4e10e60c758 100644
--- a/go/vt/vtgate/vstream_manager_test.go
+++ b/go/vt/vtgate/vstream_manager_test.go
@@ -458,7 +458,7 @@ func TestVStreamRetriableErrors(t *testing.T) {
// Always have the local cell tablet error so it's ignored on retry and we pick the other one
// if the error requires ignoring the tablet on retry.
- sbc0.AddVStreamEvents(nil, vterrors.Errorf(tcase.code, tcase.msg))
+ sbc0.AddVStreamEvents(nil, vterrors.New(tcase.code, tcase.msg))
if tcase.ignoreTablet {
sbc1.AddVStreamEvents(commit, nil)
diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go
index 7d28c0e9697..8bab05479dd 100644
--- a/go/vt/vtgate/vtgate.go
+++ b/go/vt/vtgate/vtgate.go
@@ -34,6 +34,7 @@ import (
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/tb"
+ "vitess.io/vitess/go/viperutil"
"vitess.io/vitess/go/vt/discovery"
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/log"
@@ -51,6 +52,7 @@ import (
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
+ econtext "vitess.io/vitess/go/vt/vtgate/executorcontext"
"vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext"
vtschema "vitess.io/vitess/go/vt/vtgate/schema"
"vitess.io/vitess/go/vt/vtgate/txresolver"
@@ -93,8 +95,24 @@ var (
foreignKeyMode = "allow"
dbDDLPlugin = "fail"
defaultDDLStrategy = string(schema.DDLStrategyDirect)
- enableOnlineDDL = true
- enableDirectDDL = true
+
+ enableOnlineDDL = viperutil.Configure(
+ "enable_online_ddl",
+ viperutil.Options[bool]{
+ FlagName: "enable_online_ddl",
+ Default: true,
+ Dynamic: true,
+ },
+ )
+
+ enableDirectDDL = viperutil.Configure(
+ "enable_direct_ddl",
+ viperutil.Options[bool]{
+ FlagName: "enable_direct_ddl",
+ Default: true,
+ Dynamic: true,
+ },
+ )
// schema tracking flags
enableSchemaChangeSignal = true
@@ -141,8 +159,8 @@ func registerFlags(fs *pflag.FlagSet) {
fs.DurationVar(&lockHeartbeatTime, "lock_heartbeat_time", lockHeartbeatTime, "If there is lock function used. This will keep the lock connection active by using this heartbeat")
fs.BoolVar(&warnShardedOnly, "warn_sharded_only", warnShardedOnly, "If any features that are only available in unsharded mode are used, query execution warnings will be added to the session")
fs.StringVar(&foreignKeyMode, "foreign_key_mode", foreignKeyMode, "This is to provide how to handle foreign key constraint in create/alter table. Valid values are: allow, disallow")
- fs.BoolVar(&enableOnlineDDL, "enable_online_ddl", enableOnlineDDL, "Allow users to submit, review and control Online DDL")
- fs.BoolVar(&enableDirectDDL, "enable_direct_ddl", enableDirectDDL, "Allow users to submit direct DDL statements")
+ fs.Bool("enable_online_ddl", enableOnlineDDL.Default(), "Allow users to submit, review and control Online DDL")
+ fs.Bool("enable_direct_ddl", enableDirectDDL.Default(), "Allow users to submit direct DDL statements")
fs.BoolVar(&enableSchemaChangeSignal, "schema_change_signal", enableSchemaChangeSignal, "Enable the schema tracker; requires queryserver-config-schema-change-signal to be enabled on the underlying vttablets for this to work")
fs.IntVar(&queryTimeout, "query-timeout", queryTimeout, "Sets the default query timeout (in ms). Can be overridden by session variable (query_timeout) or comment directive (QUERY_TIMEOUT_MS)")
fs.StringVar(&queryLogToFile, "log_queries_to_file", queryLogToFile, "Enable query logging to the specified file")
@@ -154,6 +172,8 @@ func registerFlags(fs *pflag.FlagSet) {
fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm")
fs.IntVar(&warmingReadsConcurrency, "warming-reads-concurrency", 500, "Number of concurrent warming reads allowed")
fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries")
+
+ viperutil.BindFlags(fs, enableOnlineDDL, enableDirectDDL)
}
func init() {
@@ -377,6 +397,7 @@ func Init(
})
vtgateInst.registerDebugHealthHandler()
vtgateInst.registerDebugEnvHandler()
+ vtgateInst.registerDebugBalancerHandler()
initAPI(gw.hc)
return vtgateInst
@@ -441,6 +462,12 @@ func (vtg *VTGate) registerDebugHealthHandler() {
})
}
+func (vtg *VTGate) registerDebugBalancerHandler() {
+ http.HandleFunc("/debug/balancer", func(w http.ResponseWriter, r *http.Request) {
+ vtg.Gateway().DebugBalancerHandler(w, r)
+ })
+}
+
// IsHealthy returns nil if server is healthy.
// Otherwise, it returns an error indicating the reason.
func (vtg *VTGate) IsHealthy() error {
@@ -462,7 +489,7 @@ func (vtg *VTGate) Execute(ctx context.Context, mysqlCtx vtgateservice.MySQLConn
if bvErr := sqltypes.ValidateBindVariables(bindVariables); bvErr != nil {
err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", bvErr)
} else {
- safeSession := NewSafeSession(session)
+ safeSession := econtext.NewSafeSession(session)
qr, err = vtg.executor.Execute(ctx, mysqlCtx, "Execute", safeSession, sql, bindVariables)
safeSession.RemoveInternalSavepoint()
}
@@ -519,7 +546,7 @@ func (vtg *VTGate) StreamExecute(ctx context.Context, mysqlCtx vtgateservice.MyS
defer vtg.timings.Record(statsKey, time.Now())
- safeSession := NewSafeSession(session)
+ safeSession := econtext.NewSafeSession(session)
var err error
if bvErr := sqltypes.ValidateBindVariables(bindVariables); bvErr != nil {
err = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", bvErr)
@@ -553,7 +580,7 @@ func (vtg *VTGate) StreamExecute(ctx context.Context, mysqlCtx vtgateservice.MyS
// same effect as if a "rollback" statement was executed, but does not affect the query
// statistics.
func (vtg *VTGate) CloseSession(ctx context.Context, session *vtgatepb.Session) error {
- return vtg.executor.CloseSession(ctx, NewSafeSession(session))
+ return vtg.executor.CloseSession(ctx, econtext.NewSafeSession(session))
}
// Prepare supports non-streaming prepare statement query with multi shards
@@ -568,7 +595,7 @@ func (vtg *VTGate) Prepare(ctx context.Context, session *vtgatepb.Session, sql s
goto handleError
}
- fld, err = vtg.executor.Prepare(ctx, "Prepare", NewSafeSession(session), sql, bindVariables)
+ fld, err = vtg.executor.Prepare(ctx, "Prepare", econtext.NewSafeSession(session), sql, bindVariables)
if err == nil {
return session, fld, nil
}
diff --git a/go/vt/vtorc/config/config.go b/go/vt/vtorc/config/config.go
index 2d21e377cb6..cafff5acce8 100644
--- a/go/vt/vtorc/config/config.go
+++ b/go/vt/vtorc/config/config.go
@@ -17,14 +17,12 @@
package config
import (
- "encoding/json"
- "fmt"
- "os"
"time"
"github.com/spf13/pflag"
- "vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/viperutil"
+ "vitess.io/vitess/go/vt/servenv"
)
var configurationLoaded = make(chan bool)
@@ -42,200 +40,296 @@ const (
)
var (
- sqliteDataFile = "file::memory:?mode=memory&cache=shared"
- instancePollTime = 5 * time.Second
- snapshotTopologyInterval = 0 * time.Hour
- reasonableReplicationLag = 10 * time.Second
- auditFileLocation = ""
- auditToBackend = false
- auditToSyslog = false
- auditPurgeDuration = 7 * 24 * time.Hour // Equivalent of 7 days
- recoveryPeriodBlockDuration = 30 * time.Second
- preventCrossCellFailover = false
- waitReplicasTimeout = 30 * time.Second
- tolerableReplicationLag = 0 * time.Second
- topoInformationRefreshDuration = 15 * time.Second
- recoveryPollDuration = 1 * time.Second
- ersEnabled = true
- convertTabletsWithErrantGTIDs = false
+ instancePollTime = viperutil.Configure(
+ "instance-poll-time",
+ viperutil.Options[time.Duration]{
+ FlagName: "instance-poll-time",
+ Default: 5 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ preventCrossCellFailover = viperutil.Configure(
+ "prevent-cross-cell-failover",
+ viperutil.Options[bool]{
+ FlagName: "prevent-cross-cell-failover",
+ Default: false,
+ Dynamic: true,
+ },
+ )
+
+ sqliteDataFile = viperutil.Configure(
+ "sqlite-data-file",
+ viperutil.Options[string]{
+ FlagName: "sqlite-data-file",
+ Default: "file::memory:?mode=memory&cache=shared",
+ Dynamic: false,
+ },
+ )
+
+ snapshotTopologyInterval = viperutil.Configure(
+ "snapshot-topology-interval",
+ viperutil.Options[time.Duration]{
+ FlagName: "snapshot-topology-interval",
+ Default: 0 * time.Hour,
+ Dynamic: true,
+ },
+ )
+
+ reasonableReplicationLag = viperutil.Configure(
+ "reasonable-replication-lag",
+ viperutil.Options[time.Duration]{
+ FlagName: "reasonable-replication-lag",
+ Default: 10 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ auditFileLocation = viperutil.Configure(
+ "audit-file-location",
+ viperutil.Options[string]{
+ FlagName: "audit-file-location",
+ Default: "",
+ Dynamic: false,
+ },
+ )
+
+ auditToBackend = viperutil.Configure(
+ "audit-to-backend",
+ viperutil.Options[bool]{
+ FlagName: "audit-to-backend",
+ Default: false,
+ Dynamic: true,
+ },
+ )
+
+ auditToSyslog = viperutil.Configure(
+ "audit-to-syslog",
+ viperutil.Options[bool]{
+ FlagName: "audit-to-syslog",
+ Default: false,
+ Dynamic: true,
+ },
+ )
+
+ auditPurgeDuration = viperutil.Configure(
+ "audit-purge-duration",
+ viperutil.Options[time.Duration]{
+ FlagName: "audit-purge-duration",
+ Default: 7 * 24 * time.Hour,
+ Dynamic: true,
+ },
+ )
+
+ waitReplicasTimeout = viperutil.Configure(
+ "wait-replicas-timeout",
+ viperutil.Options[time.Duration]{
+ FlagName: "wait-replicas-timeout",
+ Default: 30 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ tolerableReplicationLag = viperutil.Configure(
+ "tolerable-replication-lag",
+ viperutil.Options[time.Duration]{
+ FlagName: "tolerable-replication-lag",
+ Default: 0 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ topoInformationRefreshDuration = viperutil.Configure(
+ "topo-information-refresh-duration",
+ viperutil.Options[time.Duration]{
+ FlagName: "topo-information-refresh-duration",
+ Default: 15 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ recoveryPollDuration = viperutil.Configure(
+ "recovery-poll-duration",
+ viperutil.Options[time.Duration]{
+ FlagName: "recovery-poll-duration",
+ Default: 1 * time.Second,
+ Dynamic: true,
+ },
+ )
+
+ ersEnabled = viperutil.Configure(
+ "allow-emergency-reparent",
+ viperutil.Options[bool]{
+ FlagName: "allow-emergency-reparent",
+ Default: true,
+ Dynamic: true,
+ },
+ )
+
+ convertTabletsWithErrantGTIDs = viperutil.Configure(
+ "change-tablets-with-errant-gtid-to-drained",
+ viperutil.Options[bool]{
+ FlagName: "change-tablets-with-errant-gtid-to-drained",
+ Default: false,
+ Dynamic: true,
+ },
+ )
)
-// RegisterFlags registers the flags required by VTOrc
-func RegisterFlags(fs *pflag.FlagSet) {
- fs.StringVar(&sqliteDataFile, "sqlite-data-file", sqliteDataFile, "SQLite Datafile to use as VTOrc's database")
- fs.DurationVar(&instancePollTime, "instance-poll-time", instancePollTime, "Timer duration on which VTOrc refreshes MySQL information")
- fs.DurationVar(&snapshotTopologyInterval, "snapshot-topology-interval", snapshotTopologyInterval, "Timer duration on which VTOrc takes a snapshot of the current MySQL information it has in the database. Should be in multiple of hours")
- fs.DurationVar(&reasonableReplicationLag, "reasonable-replication-lag", reasonableReplicationLag, "Maximum replication lag on replicas which is deemed to be acceptable")
- fs.StringVar(&auditFileLocation, "audit-file-location", auditFileLocation, "File location where the audit logs are to be stored")
- fs.BoolVar(&auditToBackend, "audit-to-backend", auditToBackend, "Whether to store the audit log in the VTOrc database")
- fs.BoolVar(&auditToSyslog, "audit-to-syslog", auditToSyslog, "Whether to store the audit log in the syslog")
- fs.DurationVar(&auditPurgeDuration, "audit-purge-duration", auditPurgeDuration, "Duration for which audit logs are held before being purged. Should be in multiples of days")
- fs.DurationVar(&recoveryPeriodBlockDuration, "recovery-period-block-duration", recoveryPeriodBlockDuration, "Duration for which a new recovery is blocked on an instance after running a recovery")
- fs.MarkDeprecated("recovery-period-block-duration", "As of v20 this is ignored and will be removed in a future release.")
- fs.BoolVar(&preventCrossCellFailover, "prevent-cross-cell-failover", preventCrossCellFailover, "Prevent VTOrc from promoting a primary in a different cell than the current primary in case of a failover")
- fs.DurationVar(&waitReplicasTimeout, "wait-replicas-timeout", waitReplicasTimeout, "Duration for which to wait for replica's to respond when issuing RPCs")
- fs.DurationVar(&tolerableReplicationLag, "tolerable-replication-lag", tolerableReplicationLag, "Amount of replication lag that is considered acceptable for a tablet to be eligible for promotion when Vitess makes the choice of a new primary in PRS")
- fs.DurationVar(&topoInformationRefreshDuration, "topo-information-refresh-duration", topoInformationRefreshDuration, "Timer duration on which VTOrc refreshes the keyspace and vttablet records from the topology server")
- fs.DurationVar(&recoveryPollDuration, "recovery-poll-duration", recoveryPollDuration, "Timer duration on which VTOrc polls its database to run a recovery")
- fs.BoolVar(&ersEnabled, "allow-emergency-reparent", ersEnabled, "Whether VTOrc should be allowed to run emergency reparent operation when it detects a dead primary")
- fs.BoolVar(&convertTabletsWithErrantGTIDs, "change-tablets-with-errant-gtid-to-drained", convertTabletsWithErrantGTIDs, "Whether VTOrc should be changing the type of tablets with errant GTIDs to DRAINED")
+func init() {
+ servenv.OnParseFor("vtorc", registerFlags)
}
-// Configuration makes for vtorc configuration input, which can be provided by user via JSON formatted file.
-// Some of the parameters have reasonable default values, and some (like database credentials) are
-// strictly expected from user.
-// TODO(sougou): change this to yaml parsing, and possible merge with tabletenv.
-type Configuration struct {
- SQLite3DataFile string // full path to sqlite3 datafile
- InstancePollSeconds uint // Number of seconds between instance reads
- SnapshotTopologiesIntervalHours uint // Interval in hour between snapshot-topologies invocation. Default: 0 (disabled)
- ReasonableReplicationLagSeconds int // Above this value is considered a problem
- AuditLogFile string // Name of log file for audit operations. Disabled when empty.
- AuditToSyslog bool // If true, audit messages are written to syslog
- AuditToBackendDB bool // If true, audit messages are written to the backend DB's `audit` table (default: true)
- AuditPurgeDays uint // Days after which audit entries are purged from the database
- RecoveryPeriodBlockSeconds int // (overrides `RecoveryPeriodBlockMinutes`) The time for which an instance's recovery is kept "active", so as to avoid concurrent recoveries on smae instance as well as flapping
- PreventCrossDataCenterPrimaryFailover bool // When true (default: false), cross-DC primary failover are not allowed, vtorc will do all it can to only fail over within same DC, or else not fail over at all.
- WaitReplicasTimeoutSeconds int // Timeout on amount of time to wait for the replicas in case of ERS. Should be a small value because we should fail-fast. Should not be larger than LockTimeout since that is the total time we use for an ERS.
- TolerableReplicationLagSeconds int // Amount of replication lag that is considered acceptable for a tablet to be eligible for promotion when Vitess makes the choice of a new primary in PRS.
- TopoInformationRefreshSeconds int // Timer duration on which VTOrc refreshes the keyspace and vttablet records from the topo-server.
- RecoveryPollSeconds int // Timer duration on which VTOrc recovery analysis runs
+// registerFlags registers the flags required by VTOrc
+func registerFlags(fs *pflag.FlagSet) {
+ fs.String("sqlite-data-file", sqliteDataFile.Default(), "SQLite Datafile to use as VTOrc's database")
+ fs.Duration("instance-poll-time", instancePollTime.Default(), "Timer duration on which VTOrc refreshes MySQL information")
+ fs.Duration("snapshot-topology-interval", snapshotTopologyInterval.Default(), "Timer duration on which VTOrc takes a snapshot of the current MySQL information it has in the database. Should be in multiple of hours")
+ fs.Duration("reasonable-replication-lag", reasonableReplicationLag.Default(), "Maximum replication lag on replicas which is deemed to be acceptable")
+ fs.String("audit-file-location", auditFileLocation.Default(), "File location where the audit logs are to be stored")
+ fs.Bool("audit-to-backend", auditToBackend.Default(), "Whether to store the audit log in the VTOrc database")
+ fs.Bool("audit-to-syslog", auditToSyslog.Default(), "Whether to store the audit log in the syslog")
+ fs.Duration("audit-purge-duration", auditPurgeDuration.Default(), "Duration for which audit logs are held before being purged. Should be in multiples of days")
+ fs.Bool("prevent-cross-cell-failover", preventCrossCellFailover.Default(), "Prevent VTOrc from promoting a primary in a different cell than the current primary in case of a failover")
+ fs.Duration("wait-replicas-timeout", waitReplicasTimeout.Default(), "Duration for which to wait for replica's to respond when issuing RPCs")
+ fs.Duration("tolerable-replication-lag", tolerableReplicationLag.Default(), "Amount of replication lag that is considered acceptable for a tablet to be eligible for promotion when Vitess makes the choice of a new primary in PRS")
+ fs.Duration("topo-information-refresh-duration", topoInformationRefreshDuration.Default(), "Timer duration on which VTOrc refreshes the keyspace and vttablet records from the topology server")
+ fs.Duration("recovery-poll-duration", recoveryPollDuration.Default(), "Timer duration on which VTOrc polls its database to run a recovery")
+ fs.Bool("allow-emergency-reparent", ersEnabled.Default(), "Whether VTOrc should be allowed to run emergency reparent operation when it detects a dead primary")
+ fs.Bool("change-tablets-with-errant-gtid-to-drained", convertTabletsWithErrantGTIDs.Default(), "Whether VTOrc should be changing the type of tablets with errant GTIDs to DRAINED")
+
+ viperutil.BindFlags(fs,
+ instancePollTime,
+ preventCrossCellFailover,
+ sqliteDataFile,
+ snapshotTopologyInterval,
+ reasonableReplicationLag,
+ auditFileLocation,
+ auditToBackend,
+ auditToSyslog,
+ auditPurgeDuration,
+ waitReplicasTimeout,
+ tolerableReplicationLag,
+ topoInformationRefreshDuration,
+ recoveryPollDuration,
+ ersEnabled,
+ convertTabletsWithErrantGTIDs,
+ )
}
-// ToJSONString will marshal this configuration as JSON
-func (config *Configuration) ToJSONString() string {
- b, _ := json.Marshal(config)
- return string(b)
+// GetInstancePollTime is a getter function.
+func GetInstancePollTime() time.Duration {
+ return instancePollTime.Get()
}
-// Config is *the* configuration instance, used globally to get configuration data
-var Config = newConfiguration()
-var readFileNames []string
-
-// UpdateConfigValuesFromFlags is used to update the config values from the flags defined.
-// This is done before we read any configuration files from the user. So the config files take precedence.
-func UpdateConfigValuesFromFlags() {
- Config.SQLite3DataFile = sqliteDataFile
- Config.InstancePollSeconds = uint(instancePollTime / time.Second)
- Config.InstancePollSeconds = uint(instancePollTime / time.Second)
- Config.SnapshotTopologiesIntervalHours = uint(snapshotTopologyInterval / time.Hour)
- Config.ReasonableReplicationLagSeconds = int(reasonableReplicationLag / time.Second)
- Config.AuditLogFile = auditFileLocation
- Config.AuditToBackendDB = auditToBackend
- Config.AuditToSyslog = auditToSyslog
- Config.AuditPurgeDays = uint(auditPurgeDuration / (time.Hour * 24))
- Config.RecoveryPeriodBlockSeconds = int(recoveryPeriodBlockDuration / time.Second)
- Config.PreventCrossDataCenterPrimaryFailover = preventCrossCellFailover
- Config.WaitReplicasTimeoutSeconds = int(waitReplicasTimeout / time.Second)
- Config.TolerableReplicationLagSeconds = int(tolerableReplicationLag / time.Second)
- Config.TopoInformationRefreshSeconds = int(topoInformationRefreshDuration / time.Second)
- Config.RecoveryPollSeconds = int(recoveryPollDuration / time.Second)
+// SetInstancePollTime is a setter function.
+func SetInstancePollTime(v time.Duration) {
+ instancePollTime.Set(v)
}
-// ERSEnabled reports whether VTOrc is allowed to run ERS or not.
-func ERSEnabled() bool {
- return ersEnabled
+// GetInstancePollSeconds gets the instance poll time but in seconds.
+func GetInstancePollSeconds() uint {
+ return uint(instancePollTime.Get() / time.Second)
}
-// SetERSEnabled sets the value for the ersEnabled variable. This should only be used from tests.
-func SetERSEnabled(val bool) {
- ersEnabled = val
+// GetPreventCrossCellFailover is a getter function.
+func GetPreventCrossCellFailover() bool {
+ return preventCrossCellFailover.Get()
}
-// ConvertTabletWithErrantGTIDs reports whether VTOrc is allowed to change the tablet type of tablets with errant GTIDs to DRAINED.
-func ConvertTabletWithErrantGTIDs() bool {
- return convertTabletsWithErrantGTIDs
+// GetSQLiteDataFile is a getter function.
+func GetSQLiteDataFile() string {
+ return sqliteDataFile.Get()
}
-// SetConvertTabletWithErrantGTIDs sets the value for the convertTabletWithErrantGTIDs variable. This should only be used from tests.
-func SetConvertTabletWithErrantGTIDs(val bool) {
- convertTabletsWithErrantGTIDs = val
+// GetReasonableReplicationLagSeconds gets the reasonable replication lag but in seconds.
+func GetReasonableReplicationLagSeconds() int64 {
+ return int64(reasonableReplicationLag.Get() / time.Second)
+}
+
+// GetSnapshotTopologyInterval is a getter function.
+func GetSnapshotTopologyInterval() time.Duration {
+ return snapshotTopologyInterval.Get()
}
-// LogConfigValues is used to log the config values.
-func LogConfigValues() {
- b, _ := json.MarshalIndent(Config, "", "\t")
- log.Infof("Running with Configuration - %v", string(b))
+// GetAuditFileLocation is a getter function.
+func GetAuditFileLocation() string {
+ return auditFileLocation.Get()
}
-func newConfiguration() *Configuration {
- return &Configuration{
- SQLite3DataFile: "file::memory:?mode=memory&cache=shared",
- InstancePollSeconds: 5,
- SnapshotTopologiesIntervalHours: 0,
- ReasonableReplicationLagSeconds: 10,
- AuditLogFile: "",
- AuditToSyslog: false,
- AuditToBackendDB: false,
- AuditPurgeDays: 7,
- RecoveryPeriodBlockSeconds: 30,
- PreventCrossDataCenterPrimaryFailover: false,
- WaitReplicasTimeoutSeconds: 30,
- TopoInformationRefreshSeconds: 15,
- RecoveryPollSeconds: 1,
- }
+// SetAuditFileLocation is a setter function.
+func SetAuditFileLocation(v string) {
+ auditFileLocation.Set(v)
}
-func (config *Configuration) postReadAdjustments() error {
- if config.SQLite3DataFile == "" {
- return fmt.Errorf("SQLite3DataFile must be set")
- }
+// GetAuditToSyslog is a getter function.
+func GetAuditToSyslog() bool {
+ return auditToSyslog.Get()
+}
+
+// SetAuditToSyslog is a setter function.
+func SetAuditToSyslog(v bool) {
+ auditToSyslog.Set(v)
+}
+
+// GetAuditToBackend is a getter function.
+func GetAuditToBackend() bool {
+ return auditToBackend.Get()
+}
+
+// SetAuditToBackend is a setter function.
+func SetAuditToBackend(v bool) {
+ auditToBackend.Set(v)
+}
- return nil
+// GetAuditPurgeDays gets the audit purge duration but in days.
+func GetAuditPurgeDays() int64 {
+ return int64(auditPurgeDuration.Get() / (24 * time.Hour))
}
-// read reads configuration from given file, or silently skips if the file does not exist.
-// If the file does exist, then it is expected to be in valid JSON format or the function bails out.
-func read(fileName string) (*Configuration, error) {
- if fileName == "" {
- return Config, fmt.Errorf("Empty file name")
- }
- file, err := os.Open(fileName)
- if err != nil {
- return Config, err
- }
- decoder := json.NewDecoder(file)
- err = decoder.Decode(Config)
- if err == nil {
- log.Infof("Read config: %s", fileName)
- } else {
- log.Fatal("Cannot read config file:", fileName, err)
- }
- if err := Config.postReadAdjustments(); err != nil {
- log.Fatal(err)
- }
- return Config, err
+// SetAuditPurgeDays sets the audit purge duration.
+func SetAuditPurgeDays(days int64) {
+ auditPurgeDuration.Set(time.Duration(days) * 24 * time.Hour)
}
-// Read reads configuration from zero, either, some or all given files, in order of input.
-// A file can override configuration provided in previous file.
-func Read(fileNames ...string) *Configuration {
- for _, fileName := range fileNames {
- _, _ = read(fileName)
- }
- readFileNames = fileNames
- return Config
+// GetWaitReplicasTimeout is a getter function.
+func GetWaitReplicasTimeout() time.Duration {
+ return waitReplicasTimeout.Get()
}
-// ForceRead reads configuration from given file name or bails out if it fails
-func ForceRead(fileName string) *Configuration {
- _, err := read(fileName)
- if err != nil {
- log.Fatal("Cannot read config file:", fileName, err)
- }
- readFileNames = []string{fileName}
- return Config
+// GetTolerableReplicationLag is a getter function.
+func GetTolerableReplicationLag() time.Duration {
+ return tolerableReplicationLag.Get()
}
-// Reload re-reads configuration from last used files
-func Reload(extraFileNames ...string) *Configuration {
- for _, fileName := range readFileNames {
- _, _ = read(fileName)
- }
- for _, fileName := range extraFileNames {
- _, _ = read(fileName)
- }
- return Config
+// GetTopoInformationRefreshDuration is a getter function.
+func GetTopoInformationRefreshDuration() time.Duration {
+ return topoInformationRefreshDuration.Get()
+}
+
+// GetRecoveryPollDuration is a getter function.
+func GetRecoveryPollDuration() time.Duration {
+ return recoveryPollDuration.Get()
+}
+
+// ERSEnabled reports whether VTOrc is allowed to run ERS or not.
+func ERSEnabled() bool {
+ return ersEnabled.Get()
+}
+
+// SetERSEnabled sets the value for the ersEnabled variable. This should only be used from tests.
+func SetERSEnabled(val bool) {
+ ersEnabled.Set(val)
+}
+
+// ConvertTabletWithErrantGTIDs reports whether VTOrc is allowed to change the tablet type of tablets with errant GTIDs to DRAINED.
+func ConvertTabletWithErrantGTIDs() bool {
+ return convertTabletsWithErrantGTIDs.Get()
+}
+
+// SetConvertTabletWithErrantGTIDs sets the value for the convertTabletWithErrantGTIDs variable. This should only be used from tests.
+func SetConvertTabletWithErrantGTIDs(val bool) {
+ convertTabletsWithErrantGTIDs.Set(val)
}
// MarkConfigurationLoaded is called once configuration has first been loaded.
diff --git a/go/vt/vtorc/config/config_test.go b/go/vt/vtorc/config/config_test.go
deleted file mode 100644
index 2009b476f1d..00000000000
--- a/go/vt/vtorc/config/config_test.go
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
-Copyright 2022 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 config
-
-import (
- "testing"
- "time"
-
- "github.com/stretchr/testify/require"
-)
-
-func TestUpdateConfigValuesFromFlags(t *testing.T) {
- t.Run("defaults", func(t *testing.T) {
- // Restore the changes we make to the Config parameter
- defer func() {
- Config = newConfiguration()
- }()
- defaultConfig := newConfiguration()
- UpdateConfigValuesFromFlags()
- require.Equal(t, defaultConfig, Config)
- })
-
- t.Run("override auditPurgeDuration", func(t *testing.T) {
- oldAuditPurgeDuration := auditPurgeDuration
- auditPurgeDuration = 8 * time.Hour * 24
- auditPurgeDuration += time.Second + 4*time.Minute
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- auditPurgeDuration = oldAuditPurgeDuration
- }()
-
- testConfig := newConfiguration()
- // auditPurgeDuration is supposed to be in multiples of days.
- // If it is not, then we round down to the nearest number of days.
- testConfig.AuditPurgeDays = 8
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override sqliteDataFile", func(t *testing.T) {
- oldSqliteDataFile := sqliteDataFile
- sqliteDataFile = "newVal"
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- sqliteDataFile = oldSqliteDataFile
- }()
-
- testConfig := newConfiguration()
- testConfig.SQLite3DataFile = "newVal"
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override instancePollTime", func(t *testing.T) {
- oldInstancePollTime := instancePollTime
- instancePollTime = 7 * time.Second
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- instancePollTime = oldInstancePollTime
- }()
-
- testConfig := newConfiguration()
- testConfig.InstancePollSeconds = 7
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override snapshotTopologyInterval", func(t *testing.T) {
- oldSnapshotTopologyInterval := snapshotTopologyInterval
- snapshotTopologyInterval = 1 * time.Hour
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- snapshotTopologyInterval = oldSnapshotTopologyInterval
- }()
-
- testConfig := newConfiguration()
- testConfig.SnapshotTopologiesIntervalHours = 1
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override reasonableReplicationLag", func(t *testing.T) {
- oldReasonableReplicationLag := reasonableReplicationLag
- reasonableReplicationLag = 15 * time.Second
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- reasonableReplicationLag = oldReasonableReplicationLag
- }()
-
- testConfig := newConfiguration()
- testConfig.ReasonableReplicationLagSeconds = 15
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override auditFileLocation", func(t *testing.T) {
- oldAuditFileLocation := auditFileLocation
- auditFileLocation = "newFile"
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- auditFileLocation = oldAuditFileLocation
- }()
-
- testConfig := newConfiguration()
- testConfig.AuditLogFile = "newFile"
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override auditToBackend", func(t *testing.T) {
- oldAuditToBackend := auditToBackend
- auditToBackend = true
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- auditToBackend = oldAuditToBackend
- }()
-
- testConfig := newConfiguration()
- testConfig.AuditToBackendDB = true
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override auditToSyslog", func(t *testing.T) {
- oldAuditToSyslog := auditToSyslog
- auditToSyslog = true
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- auditToSyslog = oldAuditToSyslog
- }()
-
- testConfig := newConfiguration()
- testConfig.AuditToSyslog = true
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override recoveryPeriodBlockDuration", func(t *testing.T) {
- oldRecoveryPeriodBlockDuration := recoveryPeriodBlockDuration
- recoveryPeriodBlockDuration = 5 * time.Minute
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- recoveryPeriodBlockDuration = oldRecoveryPeriodBlockDuration
- }()
-
- testConfig := newConfiguration()
- testConfig.RecoveryPeriodBlockSeconds = 300
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override preventCrossCellFailover", func(t *testing.T) {
- oldPreventCrossCellFailover := preventCrossCellFailover
- preventCrossCellFailover = true
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- preventCrossCellFailover = oldPreventCrossCellFailover
- }()
-
- testConfig := newConfiguration()
- testConfig.PreventCrossDataCenterPrimaryFailover = true
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override waitReplicasTimeout", func(t *testing.T) {
- oldWaitReplicasTimeout := waitReplicasTimeout
- waitReplicasTimeout = 3*time.Minute + 4*time.Second
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- waitReplicasTimeout = oldWaitReplicasTimeout
- }()
-
- testConfig := newConfiguration()
- testConfig.WaitReplicasTimeoutSeconds = 184
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override topoInformationRefreshDuration", func(t *testing.T) {
- oldTopoInformationRefreshDuration := topoInformationRefreshDuration
- topoInformationRefreshDuration = 1 * time.Second
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- topoInformationRefreshDuration = oldTopoInformationRefreshDuration
- }()
-
- testConfig := newConfiguration()
- testConfig.TopoInformationRefreshSeconds = 1
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-
- t.Run("override recoveryPollDuration", func(t *testing.T) {
- oldRecoveryPollDuration := recoveryPollDuration
- recoveryPollDuration = 15 * time.Second
- // Restore the changes we make
- defer func() {
- Config = newConfiguration()
- recoveryPollDuration = oldRecoveryPollDuration
- }()
-
- testConfig := newConfiguration()
- testConfig.RecoveryPollSeconds = 15
- UpdateConfigValuesFromFlags()
- require.Equal(t, testConfig, Config)
- })
-}
diff --git a/go/vt/vtorc/db/db.go b/go/vt/vtorc/db/db.go
index 00f5b5b2550..870a3d15949 100644
--- a/go/vt/vtorc/db/db.go
+++ b/go/vt/vtorc/db/db.go
@@ -18,7 +18,6 @@ package db
import (
"database/sql"
- "strings"
"vitess.io/vitess/go/vt/external/golib/sqlutils"
"vitess.io/vitess/go/vt/log"
@@ -45,10 +44,12 @@ func (m *vtorcDB) QueryVTOrc(query string, argsArray []any, onRow func(sqlutils.
// OpenTopology returns the DB instance for the vtorc backed database
func OpenVTOrc() (db *sql.DB, err error) {
var fromCache bool
- db, fromCache, err = sqlutils.GetSQLiteDB(config.Config.SQLite3DataFile)
+ db, fromCache, err = sqlutils.GetSQLiteDB(config.GetSQLiteDataFile())
if err == nil && !fromCache {
- log.Infof("Connected to vtorc backend: sqlite on %v", config.Config.SQLite3DataFile)
- _ = initVTOrcDB(db)
+ log.Infof("Connected to vtorc backend: sqlite on %v", config.GetSQLiteDataFile())
+ if err := initVTOrcDB(db); err != nil {
+ log.Fatalf("Cannot initiate vtorc: %+v", err)
+ }
}
if db != nil {
db.SetMaxOpenConns(1)
@@ -57,19 +58,15 @@ func OpenVTOrc() (db *sql.DB, err error) {
return db, err
}
-func translateStatement(statement string) string {
- return sqlutils.ToSqlite3Dialect(statement)
-}
-
// registerVTOrcDeployment updates the vtorc_db_deployments table upon successful deployment
func registerVTOrcDeployment(db *sql.DB) error {
- query := `
- replace into vtorc_db_deployments (
- deployed_version, deployed_timestamp
- ) values (
- ?, NOW()
- )
- `
+ query := `REPLACE INTO vtorc_db_deployments (
+ deployed_version,
+ deployed_timestamp
+ ) VALUES (
+ ?,
+ DATETIME('now')
+ )`
if _, err := execInternal(db, query, ""); err != nil {
log.Fatalf("Unable to write to vtorc_db_deployments: %+v", err)
}
@@ -81,40 +78,24 @@ func registerVTOrcDeployment(db *sql.DB) error {
func deployStatements(db *sql.DB, queries []string) error {
tx, err := db.Begin()
if err != nil {
- log.Fatal(err.Error())
+ return err
}
for _, query := range queries {
- query = translateStatement(query)
if _, err := tx.Exec(query); err != nil {
- if strings.Contains(err.Error(), "syntax error") {
- log.Fatalf("Cannot initiate vtorc: %+v; query=%+v", err, query)
- return err
- }
- if !sqlutils.IsAlterTable(query) && !sqlutils.IsCreateIndex(query) && !sqlutils.IsDropIndex(query) {
- log.Fatalf("Cannot initiate vtorc: %+v; query=%+v", err, query)
- return err
- }
- if !strings.Contains(err.Error(), "duplicate column name") &&
- !strings.Contains(err.Error(), "Duplicate column name") &&
- !strings.Contains(err.Error(), "check that column/key exists") &&
- !strings.Contains(err.Error(), "already exists") &&
- !strings.Contains(err.Error(), "Duplicate key name") {
- log.Errorf("Error initiating vtorc: %+v; query=%+v", err, query)
- }
+ return err
}
}
- if err := tx.Commit(); err != nil {
- log.Fatal(err.Error())
- }
- return nil
+ return tx.Commit()
}
// ClearVTOrcDatabase is used to clear the VTOrc database. This function is meant to be used by tests to clear the
// database to get a clean slate without starting a new one.
func ClearVTOrcDatabase() {
- db, _, _ := sqlutils.GetSQLiteDB(config.Config.SQLite3DataFile)
+ db, _, _ := sqlutils.GetSQLiteDB(config.GetSQLiteDataFile())
if db != nil {
- _ = initVTOrcDB(db)
+ if err := initVTOrcDB(db); err != nil {
+ log.Fatalf("Cannot re-initiate vtorc: %+v", err)
+ }
}
}
@@ -123,21 +104,24 @@ func ClearVTOrcDatabase() {
func initVTOrcDB(db *sql.DB) error {
log.Info("Initializing vtorc")
log.Info("Migrating database schema")
- _ = deployStatements(db, vtorcBackend)
- _ = registerVTOrcDeployment(db)
-
- _, _ = ExecVTOrc(`PRAGMA journal_mode = WAL`)
- _, _ = ExecVTOrc(`PRAGMA synchronous = NORMAL`)
-
+ if err := deployStatements(db, vtorcBackend); err != nil {
+ return err
+ }
+ if err := registerVTOrcDeployment(db); err != nil {
+ return err
+ }
+ if _, err := ExecVTOrc(`PRAGMA journal_mode = WAL`); err != nil {
+ return err
+ }
+ if _, err := ExecVTOrc(`PRAGMA synchronous = NORMAL`); err != nil {
+ return err
+ }
return nil
}
// execInternal
func execInternal(db *sql.DB, query string, args ...any) (sql.Result, error) {
- var err error
- query = translateStatement(query)
- res, err := sqlutils.ExecNoPrepare(db, query, args...)
- return res, err
+ return sqlutils.ExecNoPrepare(db, query, args...)
}
// ExecVTOrc will execute given query on the vtorc backend database.
@@ -151,7 +135,6 @@ func ExecVTOrc(query string, args ...any) (sql.Result, error) {
// QueryVTOrcRowsMap
func QueryVTOrcRowsMap(query string, onRow func(sqlutils.RowMap) error) error {
- query = translateStatement(query)
db, err := OpenVTOrc()
if err != nil {
return err
@@ -162,7 +145,6 @@ func QueryVTOrcRowsMap(query string, onRow func(sqlutils.RowMap) error) error {
// QueryVTOrc
func QueryVTOrc(query string, argsArray []any, onRow func(sqlutils.RowMap) error) error {
- query = translateStatement(query)
db, err := OpenVTOrc()
if err != nil {
return err
diff --git a/go/vt/vtorc/discovery/queue.go b/go/vt/vtorc/discovery/queue.go
index 95751c6ae25..4b18303959b 100644
--- a/go/vt/vtorc/discovery/queue.go
+++ b/go/vt/vtorc/discovery/queue.go
@@ -153,7 +153,7 @@ func (q *Queue) Consume() string {
// alarm if have been waiting for too long
timeOnQueue := time.Since(q.queuedKeys[key])
- if timeOnQueue > time.Duration(config.Config.InstancePollSeconds)*time.Second {
+ if timeOnQueue > config.GetInstancePollTime() {
log.Warningf("key %v spent %.4fs waiting on a discoveryQueue", key, timeOnQueue.Seconds())
}
diff --git a/go/vt/vtorc/inst/analysis.go b/go/vt/vtorc/inst/analysis.go
index 66d6c6dd9ce..3e9e81c5c9f 100644
--- a/go/vt/vtorc/inst/analysis.go
+++ b/go/vt/vtorc/inst/analysis.go
@@ -144,5 +144,5 @@ func (replicationAnalysis *ReplicationAnalysis) MarshalJSON() ([]byte, error) {
// ValidSecondsFromSeenToLastAttemptedCheck returns the maximum allowed elapsed time
// between last_attempted_check to last_checked before we consider the instance as invalid.
func ValidSecondsFromSeenToLastAttemptedCheck() uint {
- return config.Config.InstancePollSeconds + 1
+ return config.GetInstancePollSeconds()
}
diff --git a/go/vt/vtorc/inst/analysis_dao.go b/go/vt/vtorc/inst/analysis_dao.go
index 0268a8b183a..07830bf7dda 100644
--- a/go/vt/vtorc/inst/analysis_dao.go
+++ b/go/vt/vtorc/inst/analysis_dao.go
@@ -47,7 +47,7 @@ func init() {
func initializeAnalysisDaoPostConfiguration() {
config.WaitForConfigurationToBeLoaded()
- recentInstantAnalysis = cache.New(time.Duration(config.Config.RecoveryPollSeconds*2)*time.Second, time.Second)
+ recentInstantAnalysis = cache.New(config.GetRecoveryPollDuration()*2, time.Second)
}
type clusterAnalysis struct {
@@ -68,9 +68,8 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna
}
// TODO(sougou); deprecate ReduceReplicationAnalysisCount
- args := sqlutils.Args(config.Config.ReasonableReplicationLagSeconds, ValidSecondsFromSeenToLastAttemptedCheck(), config.Config.ReasonableReplicationLagSeconds, keyspace, shard)
- query := `
- SELECT
+ args := sqlutils.Args(config.GetReasonableReplicationLagSeconds(), ValidSecondsFromSeenToLastAttemptedCheck(), config.GetReasonableReplicationLagSeconds(), keyspace, shard)
+ query := `SELECT
vitess_tablet.info AS tablet_info,
vitess_tablet.tablet_type,
vitess_tablet.primary_timestamp,
@@ -91,13 +90,13 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna
IFNULL(
primary_instance.binary_log_file = database_instance_stale_binlog_coordinates.binary_log_file
AND primary_instance.binary_log_pos = database_instance_stale_binlog_coordinates.binary_log_pos
- AND database_instance_stale_binlog_coordinates.first_seen < NOW() - interval ? second,
+ AND database_instance_stale_binlog_coordinates.first_seen < DATETIME('now', PRINTF('-%d SECOND', ?)),
0
)
) AS is_stale_binlog_coordinates,
MIN(
primary_instance.last_checked <= primary_instance.last_seen
- and primary_instance.last_attempted_check <= primary_instance.last_seen + interval ? second
+ and primary_instance.last_attempted_check <= DATETIME(primary_instance.last_seen, PRINTF('+%d SECOND', ?))
) = 1 AS is_last_check_valid,
/* To be considered a primary, traditional async replication must not be present/valid AND the host should either */
/* not be a replication group member OR be the primary of the replication group */
@@ -294,6 +293,11 @@ func GetReplicationAnalysis(keyspace string, shard string, hints *ReplicationAna
return nil
}
+ // We don't want to run any fixes on any non-replica type tablet.
+ if tablet.Type != topodatapb.TabletType_PRIMARY && !topo.IsReplicaType(tablet.Type) {
+ return nil
+ }
+
primaryTablet := &topodatapb.Tablet{}
if str := m.GetString("primary_tablet_info"); str != "" {
if err := opts.Unmarshal([]byte(str), primaryTablet); err != nil {
@@ -650,13 +654,13 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC
// Find if the lastAnalysisHasChanged or not while updating the row if it has.
lastAnalysisChanged := false
{
- sqlResult, err := db.ExecVTOrc(`
- update database_instance_last_analysis set
+ sqlResult, err := db.ExecVTOrc(`UPDATE database_instance_last_analysis
+ SET
analysis = ?,
- analysis_timestamp = now()
- where
+ analysis_timestamp = DATETIME('now')
+ WHERE
alias = ?
- and analysis != ?
+ AND analysis != ?
`,
string(analysisCode), tabletAlias, string(analysisCode),
)
@@ -677,13 +681,16 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC
firstInsertion := false
if !lastAnalysisChanged {
// The insert only returns more than 1 row changed if this is the first insertion.
- sqlResult, err := db.ExecVTOrc(`
- insert ignore into database_instance_last_analysis (
- alias, analysis_timestamp, analysis
- ) values (
- ?, now(), ?
- )
- `,
+ sqlResult, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO database_instance_last_analysis (
+ alias,
+ analysis_timestamp,
+ analysis
+ ) VALUES (
+ ?,
+ DATETIME('now'),
+ ?
+ )`,
tabletAlias, string(analysisCode),
)
if err != nil {
@@ -703,13 +710,16 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC
return nil
}
- _, err := db.ExecVTOrc(`
- insert into database_instance_analysis_changelog (
- alias, analysis_timestamp, analysis
- ) values (
- ?, now(), ?
- )
- `,
+ _, err := db.ExecVTOrc(`INSERT
+ INTO database_instance_analysis_changelog (
+ alias,
+ analysis_timestamp,
+ analysis
+ ) VALUES (
+ ?,
+ DATETIME('now'),
+ ?
+ )`,
tabletAlias, string(analysisCode),
)
if err == nil {
@@ -722,12 +732,11 @@ func auditInstanceAnalysisInChangelog(tabletAlias string, analysisCode AnalysisC
// ExpireInstanceAnalysisChangelog removes old-enough analysis entries from the changelog
func ExpireInstanceAnalysisChangelog() error {
- _, err := db.ExecVTOrc(`
- delete
- from database_instance_analysis_changelog
- where
- analysis_timestamp < now() - interval ? hour
- `,
+ _, err := db.ExecVTOrc(`DELETE
+ FROM database_instance_analysis_changelog
+ WHERE
+ analysis_timestamp < DATETIME('now', PRINTF('-%d HOUR', ?))
+ `,
config.UnseenInstanceForgetHours,
)
if err != nil {
diff --git a/go/vt/vtorc/inst/analysis_dao_test.go b/go/vt/vtorc/inst/analysis_dao_test.go
index a83e975c747..c061d54ebb3 100644
--- a/go/vt/vtorc/inst/analysis_dao_test.go
+++ b/go/vt/vtorc/inst/analysis_dao_test.go
@@ -424,6 +424,47 @@ func TestGetReplicationAnalysisDecision(t *testing.T) {
keyspaceWanted: "ks",
shardWanted: "0",
codeWanted: ReplicationStopped,
+ }, {
+ name: "No recoveries on drained tablets",
+ info: []*test.InfoForRecoveryAnalysis{{
+ TabletInfo: &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 101},
+ Hostname: "localhost",
+ Keyspace: "ks",
+ Shard: "0",
+ Type: topodatapb.TabletType_PRIMARY,
+ MysqlHostname: "localhost",
+ MysqlPort: 6708,
+ },
+ DurabilityPolicy: "none",
+ LastCheckValid: 1,
+ CountReplicas: 4,
+ CountValidReplicas: 4,
+ CountValidReplicatingReplicas: 3,
+ CountValidOracleGTIDReplicas: 4,
+ CountLoggingReplicas: 2,
+ IsPrimary: 1,
+ }, {
+ TabletInfo: &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 100},
+ Hostname: "localhost",
+ Keyspace: "ks",
+ Shard: "0",
+ Type: topodatapb.TabletType_DRAINED,
+ MysqlHostname: "localhost",
+ MysqlPort: 6709,
+ },
+ DurabilityPolicy: "none",
+ PrimaryTabletInfo: &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{Cell: "zon1", Uid: 101},
+ },
+ LastCheckValid: 1,
+ ReadOnly: 1,
+ ReplicationStopped: 1,
+ }},
+ keyspaceWanted: "ks",
+ shardWanted: "0",
+ codeWanted: NoProblem,
}, {
name: "ReplicaMisconfigured",
info: []*test.InfoForRecoveryAnalysis{{
diff --git a/go/vt/vtorc/inst/audit_dao.go b/go/vt/vtorc/inst/audit_dao.go
index d048f300faf..7ae60fba927 100644
--- a/go/vt/vtorc/inst/audit_dao.go
+++ b/go/vt/vtorc/inst/audit_dao.go
@@ -38,10 +38,10 @@ func AuditOperation(auditType string, tabletAlias string, message string) error
}
auditWrittenToFile := false
- if config.Config.AuditLogFile != "" {
+ if config.GetAuditFileLocation() != "" {
auditWrittenToFile = true
go func() {
- f, err := os.OpenFile(config.Config.AuditLogFile, os.O_RDWR|os.O_CREATE|os.O_APPEND, 0640)
+ f, err := os.OpenFile(config.GetAuditFileLocation(), os.O_RDWR|os.O_CREATE|os.O_APPEND, 0640)
if err != nil {
log.Error(err)
return
@@ -54,15 +54,23 @@ func AuditOperation(auditType string, tabletAlias string, message string) error
}
}()
}
- if config.Config.AuditToBackendDB {
- _, err := db.ExecVTOrc(`
- insert
- into audit (
- audit_timestamp, audit_type, alias, keyspace, shard, message
- ) VALUES (
- NOW(), ?, ?, ?, ?, ?
- )
- `,
+ if config.GetAuditToBackend() {
+ _, err := db.ExecVTOrc(`INSERT
+ INTO audit (
+ audit_timestamp,
+ audit_type,
+ alias,
+ keyspace,
+ shard,
+ message
+ ) VALUES (
+ DATETIME('now'),
+ ?,
+ ?,
+ ?,
+ ?,
+ ?
+ )`,
auditType,
tabletAlias,
keyspace,
diff --git a/go/vt/vtorc/inst/audit_dao_test.go b/go/vt/vtorc/inst/audit_dao_test.go
index 1d50de4c146..d22e9177dc3 100644
--- a/go/vt/vtorc/inst/audit_dao_test.go
+++ b/go/vt/vtorc/inst/audit_dao_test.go
@@ -35,13 +35,13 @@ import (
// This test also verifies that we are able to read the recent audits that are written to the databaes.
func TestAuditOperation(t *testing.T) {
// Restore original configurations
- originalAuditSysLog := config.Config.AuditToSyslog
- originalAuditLogFile := config.Config.AuditLogFile
- originalAuditBackend := config.Config.AuditToBackendDB
+ originalAuditSysLog := config.GetAuditToSyslog()
+ originalAuditLogFile := config.GetAuditFileLocation()
+ originalAuditBackend := config.GetAuditToBackend()
defer func() {
- config.Config.AuditToSyslog = originalAuditSysLog
- config.Config.AuditLogFile = originalAuditLogFile
- config.Config.AuditToBackendDB = originalAuditBackend
+ config.SetAuditToSyslog(originalAuditSysLog)
+ config.SetAuditFileLocation(originalAuditLogFile)
+ config.SetAuditToBackend(originalAuditBackend)
}()
orcDb, err := db.OpenVTOrc()
@@ -78,9 +78,9 @@ func TestAuditOperation(t *testing.T) {
message := "test-message"
t.Run("audit to backend", func(t *testing.T) {
- config.Config.AuditLogFile = ""
- config.Config.AuditToSyslog = false
- config.Config.AuditToBackendDB = true
+ config.SetAuditFileLocation("")
+ config.SetAuditToSyslog(false)
+ config.SetAuditToBackend(true)
// Auditing should succeed as expected
err = AuditOperation(auditType, tab100Alias, message)
@@ -106,13 +106,13 @@ func TestAuditOperation(t *testing.T) {
})
t.Run("audit to File", func(t *testing.T) {
- config.Config.AuditToBackendDB = false
- config.Config.AuditToSyslog = false
+ config.SetAuditToBackend(false)
+ config.SetAuditToSyslog(false)
file, err := os.CreateTemp("", "test-auditing-*")
require.NoError(t, err)
defer os.Remove(file.Name())
- config.Config.AuditLogFile = file.Name()
+ config.SetAuditFileLocation(file.Name())
err = AuditOperation(auditType, tab100Alias, message)
require.NoError(t, err)
diff --git a/go/vt/vtorc/inst/instance_dao.go b/go/vt/vtorc/inst/instance_dao.go
index 0615cbc0cde..d1421dbc91d 100644
--- a/go/vt/vtorc/inst/instance_dao.go
+++ b/go/vt/vtorc/inst/instance_dao.go
@@ -61,6 +61,7 @@ var forgetAliases *cache.Cache
var (
readTopologyInstanceCounter = stats.NewCounter("InstanceReadTopology", "Number of times an instance was read from the topology")
readInstanceCounter = stats.NewCounter("InstanceRead", "Number of times an instance was read")
+ currentErrantGTIDCount = stats.NewGaugesWithSingleLabel("CurrentErrantGTIDCount", "Number of errant GTIDs a vttablet currently has", "TabletAlias")
backendWrites = collection.CreateOrReturnCollection("BACKEND_WRITES")
writeBufferLatency = stopwatch.NewNamedStopwatch()
)
@@ -79,7 +80,7 @@ func init() {
func initializeInstanceDao() {
config.WaitForConfigurationToBeLoaded()
- forgetAliases = cache.New(time.Duration(config.Config.InstancePollSeconds*3)*time.Second, time.Second)
+ forgetAliases = cache.New(config.GetInstancePollTime()*3, time.Second)
cacheInitializationCompleted.Store(true)
}
@@ -113,10 +114,15 @@ func ExecDBWriteFunc(f func() error) error {
func ExpireTableData(tableName string, timestampColumn string) error {
writeFunc := func() error {
- _, err := db.ExecVTOrc(
- fmt.Sprintf("delete from %s where %s < NOW() - INTERVAL ? DAY", tableName, timestampColumn),
- config.Config.AuditPurgeDays,
+ query := fmt.Sprintf(`DELETE
+ FROM %s
+ WHERE
+ %s < DATETIME('now', PRINTF('-%%d DAY', ?))
+ `,
+ tableName,
+ timestampColumn,
)
+ _, err := db.ExecVTOrc(query, config.GetAuditPurgeDays())
return err
}
return ExecDBWriteFunc(writeFunc)
@@ -141,8 +147,8 @@ func logReadTopologyInstanceError(tabletAlias string, hint string, err error) er
strings.Replace(hint, "%", "%%", -1), // escape %
err)
}
- log.Errorf(msg)
- return fmt.Errorf(msg)
+ log.Error(msg)
+ return errors.New(msg)
}
// RegisterStats registers stats from the inst package
@@ -356,30 +362,7 @@ Cleanup:
// Add replication group ancestry UUID as well. Otherwise, VTOrc thinks there are errant GTIDs in group
// members and its replicas, even though they are not.
instance.AncestryUUID = strings.Trim(instance.AncestryUUID, ",")
- if instance.ExecutedGtidSet != "" && instance.primaryExecutedGtidSet != "" {
- // Compare primary & replica GTID sets, but ignore the sets that present the primary's UUID.
- // This is because vtorc may pool primary and replica at an inconvenient timing,
- // such that the replica may _seems_ to have more entries than the primary, when in fact
- // it's just that the primary's probing is stale.
- redactedExecutedGtidSet, _ := NewOracleGtidSet(instance.ExecutedGtidSet)
- for _, uuid := range strings.Split(instance.AncestryUUID, ",") {
- if uuid != instance.ServerUUID {
- redactedExecutedGtidSet.RemoveUUID(uuid)
- }
- if instance.IsCoPrimary && uuid == instance.ServerUUID {
- // If this is a co-primary, then this server is likely to show its own generated GTIDs as errant,
- // because its co-primary has not applied them yet
- redactedExecutedGtidSet.RemoveUUID(uuid)
- }
- }
- // Avoid querying the database if there's no point:
- if !redactedExecutedGtidSet.IsEmpty() {
- redactedPrimaryExecutedGtidSet, _ := NewOracleGtidSet(instance.primaryExecutedGtidSet)
- redactedPrimaryExecutedGtidSet.RemoveUUID(instance.SourceUUID)
-
- instance.GtidErrant, err = replication.Subtract(redactedExecutedGtidSet.String(), redactedPrimaryExecutedGtidSet.String())
- }
- }
+ err = detectErrantGTIDs(instance, tablet)
}
latency.Stop("instance")
@@ -406,6 +389,63 @@ Cleanup:
return nil, err
}
+// detectErrantGTIDs detects the errant GTIDs on an instance.
+func detectErrantGTIDs(instance *Instance, tablet *topodatapb.Tablet) (err error) {
+ // If the tablet is not replicating from anyone, then it could be the previous primary.
+ // We should check for errant GTIDs by finding the difference with the shard's current primary.
+ if instance.primaryExecutedGtidSet == "" && instance.SourceHost == "" {
+ var primaryInstance *Instance
+ primaryAlias, _, _ := ReadShardPrimaryInformation(tablet.Keyspace, tablet.Shard)
+ if primaryAlias != "" {
+ // Check if the current tablet is the primary.
+ // If it is, then we don't need to run errant gtid detection on it.
+ if primaryAlias == instance.InstanceAlias {
+ return nil
+ }
+ primaryInstance, _, _ = ReadInstance(primaryAlias)
+ }
+ // Only run errant GTID detection, if we are sure that the data read of the current primary
+ // is up-to-date enough to reflect that it has been promoted. This is needed to prevent
+ // flagging incorrect errant GTIDs. If we were to use old data, we could have some GTIDs
+ // accepted by the old primary (this tablet) that don't show in the new primary's set.
+ if primaryInstance != nil {
+ if primaryInstance.SourceHost == "" {
+ instance.primaryExecutedGtidSet = primaryInstance.ExecutedGtidSet
+ }
+ }
+ }
+ if instance.ExecutedGtidSet != "" && instance.primaryExecutedGtidSet != "" {
+ // Compare primary & replica GTID sets, but ignore the sets that present the primary's UUID.
+ // This is because vtorc may pool primary and replica at an inconvenient timing,
+ // such that the replica may _seems_ to have more entries than the primary, when in fact
+ // it's just that the primary's probing is stale.
+ redactedExecutedGtidSet, _ := NewOracleGtidSet(instance.ExecutedGtidSet)
+ for _, uuid := range strings.Split(instance.AncestryUUID, ",") {
+ if uuid != instance.ServerUUID {
+ redactedExecutedGtidSet.RemoveUUID(uuid)
+ }
+ if instance.IsCoPrimary && uuid == instance.ServerUUID {
+ // If this is a co-primary, then this server is likely to show its own generated GTIDs as errant,
+ // because its co-primary has not applied them yet
+ redactedExecutedGtidSet.RemoveUUID(uuid)
+ }
+ }
+ // Avoid querying the database if there's no point:
+ if !redactedExecutedGtidSet.IsEmpty() {
+ redactedPrimaryExecutedGtidSet, _ := NewOracleGtidSet(instance.primaryExecutedGtidSet)
+ redactedPrimaryExecutedGtidSet.RemoveUUID(instance.SourceUUID)
+
+ instance.GtidErrant, err = replication.Subtract(redactedExecutedGtidSet.String(), redactedPrimaryExecutedGtidSet.String())
+ if err == nil {
+ var gtidCount int64
+ gtidCount, err = replication.GTIDCount(instance.GtidErrant)
+ currentErrantGTIDCount.Set(instance.InstanceAlias, gtidCount)
+ }
+ }
+ }
+ return err
+}
+
// getKeyspaceShardName returns a single string having both the keyspace and shard
func getKeyspaceShardName(keyspace, shard string) string {
return fmt.Sprintf("%v:%v", keyspace, shard)
@@ -433,16 +473,16 @@ func ReadInstanceClusterAttributes(instance *Instance) (err error) {
var primaryExecutedGtidSet string
primaryDataFound := false
- query := `
- select
- replication_depth,
- source_host,
- source_port,
- ancestry_uuid,
- executed_gtid_set
- from database_instance
- where hostname=? and port=?
- `
+ query := `SELECT
+ replication_depth,
+ source_host,
+ source_port,
+ ancestry_uuid,
+ executed_gtid_set
+ FROM database_instance
+ WHERE
+ hostname = ?
+ AND port = ?`
primaryHostname := instance.SourceHost
primaryPort := instance.SourcePort
args := sqlutils.Args(primaryHostname, primaryPort)
@@ -538,8 +578,8 @@ func readInstanceRow(m sqlutils.RowMap) *Instance {
instance.ReplicationDepth = m.GetUint("replication_depth")
instance.IsCoPrimary = m.GetBool("is_co_primary")
instance.HasReplicationCredentials = m.GetBool("has_replication_credentials")
- instance.IsUpToDate = (m.GetUint("seconds_since_last_checked") <= config.Config.InstancePollSeconds)
- instance.IsRecentlyChecked = (m.GetUint("seconds_since_last_checked") <= config.Config.InstancePollSeconds*5)
+ instance.IsUpToDate = m.GetUint("seconds_since_last_checked") <= config.GetInstancePollSeconds()
+ instance.IsRecentlyChecked = m.GetUint("seconds_since_last_checked") <= config.GetInstancePollSeconds()*5
instance.LastSeenTimestamp = m.GetString("last_seen")
instance.IsLastCheckValid = m.GetBool("is_last_check_valid")
instance.SecondsSinceLastSeen = m.GetNullInt64("seconds_since_last_seen")
@@ -556,7 +596,7 @@ func readInstanceRow(m sqlutils.RowMap) *Instance {
instance.Problems = append(instance.Problems, "not_recently_checked")
} else if instance.ReplicationThreadsExist() && !instance.ReplicaRunning() {
instance.Problems = append(instance.Problems, "not_replicating")
- } else if instance.ReplicationLagSeconds.Valid && util.AbsInt64(instance.ReplicationLagSeconds.Int64-int64(instance.SQLDelay)) > int64(config.Config.ReasonableReplicationLagSeconds) {
+ } else if instance.ReplicationLagSeconds.Valid && util.AbsInt64(instance.ReplicationLagSeconds.Int64-int64(instance.SQLDelay)) > int64(config.GetReasonableReplicationLagSeconds()) {
instance.Problems = append(instance.Problems, "replication_lag")
}
if instance.GtidErrant != "" {
@@ -574,20 +614,22 @@ func readInstancesByCondition(condition string, args []any, sort string) ([](*In
if sort == "" {
sort = `alias`
}
- query := fmt.Sprintf(`
- select
- *,
- unix_timestamp() - unix_timestamp(last_checked) as seconds_since_last_checked,
- ifnull(last_checked <= last_seen, 0) as is_last_check_valid,
- unix_timestamp() - unix_timestamp(last_seen) as seconds_since_last_seen
- from
- vitess_tablet
- left join database_instance using (alias, hostname, port)
- where
- %s
- order by
- %s
- `, condition, sort)
+ query := fmt.Sprintf(`SELECT
+ *,
+ STRFTIME('%%s', 'now') - STRFTIME('%%s', last_checked) AS seconds_since_last_checked,
+ IFNULL(last_checked <= last_seen, 0) AS is_last_check_valid,
+ STRFTIME('%%s', 'now') - STRFTIME('%%s', last_seen) AS seconds_since_last_seen
+ FROM
+ vitess_tablet
+ LEFT JOIN database_instance USING (alias, hostname, port)
+ WHERE
+ %s
+ ORDER BY
+ %s
+ `,
+ condition,
+ sort,
+ )
err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error {
instance := readInstanceRow(m)
@@ -608,9 +650,7 @@ func readInstancesByCondition(condition string, args []any, sort string) ([](*In
// ReadInstance reads an instance from the vtorc backend database
func ReadInstance(tabletAlias string) (*Instance, bool, error) {
- condition := `
- alias = ?
- `
+ condition := `alias = ?`
instances, err := readInstancesByCondition(condition, sqlutils.Args(tabletAlias), "")
// We know there will be at most one (alias is the PK).
// And we expect to find one.
@@ -627,30 +667,28 @@ func ReadInstance(tabletAlias string) (*Instance, bool, error) {
// ReadProblemInstances reads all instances with problems
func ReadProblemInstances(keyspace string, shard string) ([](*Instance), error) {
condition := `
- keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
- and shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
- and (
- (last_seen < last_checked)
- or (unix_timestamp() - unix_timestamp(last_checked) > ?)
- or (replication_sql_thread_state not in (-1 ,1))
- or (replication_io_thread_state not in (-1 ,1))
- or (abs(cast(replication_lag_seconds as signed) - cast(sql_delay as signed)) > ?)
- or (abs(cast(replica_lag_seconds as signed) - cast(sql_delay as signed)) > ?)
- or (gtid_errant != '')
- )
- `
-
- args := sqlutils.Args(keyspace, keyspace, shard, shard, config.Config.InstancePollSeconds*5, config.Config.ReasonableReplicationLagSeconds, config.Config.ReasonableReplicationLagSeconds)
+ keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
+ AND shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
+ AND (
+ (last_seen < last_checked)
+ OR (STRFTIME('%%s', 'now') - STRFTIME('%%s', last_checked) > ?)
+ OR (replication_sql_thread_state NOT IN (-1 ,1))
+ OR (replication_io_thread_state NOT IN (-1 ,1))
+ OR (ABS(CAST(replication_lag_seconds AS integer) - CAST(sql_delay AS integer)) > ?)
+ OR (ABS(CAST(replica_lag_seconds AS integer) - CAST(sql_delay AS integer)) > ?)
+ OR (gtid_errant != '')
+ )`
+
+ args := sqlutils.Args(keyspace, keyspace, shard, shard, config.GetInstancePollSeconds()*5, config.GetReasonableReplicationLagSeconds(), config.GetReasonableReplicationLagSeconds())
return readInstancesByCondition(condition, args, "")
}
// ReadInstancesWithErrantGTIds reads all instances with errant GTIDs
func ReadInstancesWithErrantGTIds(keyspace string, shard string) ([]*Instance, error) {
condition := `
- keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
- and shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
- and gtid_errant != ''
- `
+ keyspace LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
+ AND shard LIKE (CASE WHEN ? = '' THEN '%' ELSE ? END)
+ AND gtid_errant != ''`
args := sqlutils.Args(keyspace, keyspace, shard, shard)
return readInstancesByCondition(condition, args, "")
@@ -658,15 +696,14 @@ func ReadInstancesWithErrantGTIds(keyspace string, shard string) ([]*Instance, e
// GetKeyspaceShardName gets the keyspace shard name for the given instance key
func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, err error) {
- query := `
- select
- keyspace,
- shard
- from
- vitess_tablet
- where
- alias = ?
- `
+ query := `SELECT
+ keyspace,
+ shard
+ FROM
+ vitess_tablet
+ WHERE
+ alias = ?
+ `
err = db.QueryVTOrc(query, sqlutils.Args(tabletAlias), func(m sqlutils.RowMap) error {
keyspace = m.GetString("keyspace")
shard = m.GetString("shard")
@@ -689,28 +726,27 @@ func GetKeyspaceShardName(tabletAlias string) (keyspace string, shard string, er
// the instance.
func ReadOutdatedInstanceKeys() ([]string, error) {
var res []string
- query := `
- SELECT
- alias
- FROM
- database_instance
- WHERE
- CASE
- WHEN last_attempted_check <= last_checked
- THEN last_checked < now() - interval ? second
- ELSE last_checked < now() - interval ? second
- END
- UNION
- SELECT
- vitess_tablet.alias
- FROM
- vitess_tablet LEFT JOIN database_instance ON (
- vitess_tablet.alias = database_instance.alias
- )
- WHERE
- database_instance.alias IS NULL
- `
- args := sqlutils.Args(config.Config.InstancePollSeconds, 2*config.Config.InstancePollSeconds)
+ query := `SELECT
+ alias
+ FROM
+ database_instance
+ WHERE
+ CASE
+ WHEN last_attempted_check <= last_checked
+ THEN last_checked < DATETIME('now', PRINTF('-%d SECOND', ?))
+ ELSE last_checked < DATETIME('now', PRINTF('-%d SECOND', ?))
+ END
+ UNION
+ SELECT
+ vitess_tablet.alias
+ FROM
+ vitess_tablet LEFT JOIN database_instance ON (
+ vitess_tablet.alias = database_instance.alias
+ )
+ WHERE
+ database_instance.alias IS NULL
+ `
+ args := sqlutils.Args(config.GetInstancePollSeconds(), 2*config.GetInstancePollSeconds())
err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error {
tabletAlias := m.GetString("alias")
@@ -727,7 +763,7 @@ func ReadOutdatedInstanceKeys() ([]string, error) {
return res, err
}
-func mkInsertOdku(table string, columns []string, values []string, nrRows int, insertIgnore bool) (string, error) {
+func mkInsert(table string, columns []string, values []string, nrRows int, insertIgnore bool) (string, error) {
if len(columns) == 0 {
return "", errors.New("Column list cannot be empty")
}
@@ -739,9 +775,9 @@ func mkInsertOdku(table string, columns []string, values []string, nrRows int, i
}
var q strings.Builder
- var ignore string
+ insertStr := "REPLACE INTO"
if insertIgnore {
- ignore = "ignore"
+ insertStr = "INSERT OR IGNORE INTO"
}
valRow := fmt.Sprintf("(%s)", strings.Join(values, ", "))
var val strings.Builder
@@ -752,26 +788,22 @@ func mkInsertOdku(table string, columns []string, values []string, nrRows int, i
}
col := strings.Join(columns, ", ")
- var odku strings.Builder
- odku.WriteString(fmt.Sprintf("%s=VALUES(%s)", columns[0], columns[0]))
- for _, c := range columns[1:] {
- odku.WriteString(", ")
- odku.WriteString(fmt.Sprintf("%s=VALUES(%s)", c, c))
- }
-
- q.WriteString(fmt.Sprintf(`INSERT %s INTO %s
- (%s)
- VALUES
- %s
- ON DUPLICATE KEY UPDATE
- %s
- `,
- ignore, table, col, val.String(), odku.String()))
+ query := fmt.Sprintf(`%s %s
+ (%s)
+ VALUES
+ %s
+ `,
+ insertStr,
+ table,
+ col,
+ val.String(),
+ )
+ q.WriteString(query)
return q.String(), nil
}
-func mkInsertOdkuForInstances(instances []*Instance, instanceWasActuallyFound bool, updateLastSeen bool) (string, []any, error) {
+func mkInsertForInstances(instances []*Instance, instanceWasActuallyFound bool, updateLastSeen bool) (string, []any, error) {
if len(instances) == 0 {
return "", nil, nil
}
@@ -852,13 +884,13 @@ func mkInsertOdkuForInstances(instances []*Instance, instanceWasActuallyFound bo
for i := range columns {
values[i] = "?"
}
- values[3] = "NOW()" // last_checked
- values[4] = "NOW()" // last_attempted_check
- values[5] = "1" // last_check_partial_success
+ values[3] = "DATETIME('now')" // last_checked
+ values[4] = "DATETIME('now')" // last_attempted_check
+ values[5] = "1" // last_check_partial_success
if updateLastSeen {
columns = append(columns, "last_seen")
- values = append(values, "NOW()")
+ values = append(values, "DATETIME('now')")
}
var args []any
@@ -929,11 +961,11 @@ func mkInsertOdkuForInstances(instances []*Instance, instanceWasActuallyFound bo
args = append(args, instance.LastDiscoveryLatency.Nanoseconds())
}
- sql, err := mkInsertOdku("database_instance", columns, values, len(instances), insertIgnore)
+ sql, err := mkInsert("database_instance", columns, values, len(instances), insertIgnore)
if err != nil {
errMsg := fmt.Sprintf("Failed to build query: %v", err)
log.Errorf(errMsg)
- return sql, args, fmt.Errorf(errMsg)
+ return sql, args, errors.New(errMsg)
}
return sql, args, nil
@@ -951,7 +983,7 @@ func writeManyInstances(instances []*Instance, instanceWasActuallyFound bool, up
if len(writeInstances) == 0 {
return nil // nothing to write
}
- sql, args, err := mkInsertOdkuForInstances(writeInstances, instanceWasActuallyFound, updateLastSeen)
+ sql, args, err := mkInsertForInstances(writeInstances, instanceWasActuallyFound, updateLastSeen)
if err != nil {
return err
}
@@ -974,14 +1006,13 @@ func WriteInstance(instance *Instance, instanceWasActuallyFound bool, lastError
// for a given instance
func UpdateInstanceLastChecked(tabletAlias string, partialSuccess bool) error {
writeFunc := func() error {
- _, err := db.ExecVTOrc(`
- update
- database_instance
- set
- last_checked = NOW(),
- last_check_partial_success = ?
- where
- alias = ?`,
+ _, err := db.ExecVTOrc(`UPDATE database_instance
+ SET
+ last_checked = DATETIME('now'),
+ last_check_partial_success = ?
+ WHERE
+ alias = ?
+ `,
partialSuccess,
tabletAlias,
)
@@ -1003,13 +1034,12 @@ func UpdateInstanceLastChecked(tabletAlias string, partialSuccess bool) error {
// we have a "hanging" issue.
func UpdateInstanceLastAttemptedCheck(tabletAlias string) error {
writeFunc := func() error {
- _, err := db.ExecVTOrc(`
- update
- database_instance
- set
- last_attempted_check = NOW()
- where
- alias = ?`,
+ _, err := db.ExecVTOrc(`UPDATE database_instance
+ SET
+ last_attempted_check = DATETIME('now')
+ WHERE
+ alias = ?
+ `,
tabletAlias,
)
if err != nil {
@@ -1031,17 +1061,20 @@ func ForgetInstance(tabletAlias string) error {
if tabletAlias == "" {
errMsg := "ForgetInstance(): empty tabletAlias"
log.Errorf(errMsg)
- return fmt.Errorf(errMsg)
+ return errors.New(errMsg)
}
forgetAliases.Set(tabletAlias, true, cache.DefaultExpiration)
log.Infof("Forgetting: %v", tabletAlias)
+ // Remove this tablet from errant GTID count metric.
+ currentErrantGTIDCount.Reset(tabletAlias)
+
// Delete from the 'vitess_tablet' table.
- _, err := db.ExecVTOrc(`
- delete
- from vitess_tablet
- where
- alias = ?`,
+ _, err := db.ExecVTOrc(`DELETE
+ FROM vitess_tablet
+ WHERE
+ alias = ?
+ `,
tabletAlias,
)
if err != nil {
@@ -1050,11 +1083,11 @@ func ForgetInstance(tabletAlias string) error {
}
// Also delete from the 'database_instance' table.
- sqlResult, err := db.ExecVTOrc(`
- delete
- from database_instance
- where
- alias = ?`,
+ sqlResult, err := db.ExecVTOrc(`DELETE
+ FROM database_instance
+ WHERE
+ alias = ?
+ `,
tabletAlias,
)
if err != nil {
@@ -1069,8 +1102,8 @@ func ForgetInstance(tabletAlias string) error {
}
if rows == 0 {
errMsg := fmt.Sprintf("ForgetInstance(): tablet %+v not found", tabletAlias)
- log.Errorf(errMsg)
- return fmt.Errorf(errMsg)
+ log.Error(errMsg)
+ return errors.New(errMsg)
}
_ = AuditOperation("forget", tabletAlias, "")
return nil
@@ -1078,11 +1111,11 @@ func ForgetInstance(tabletAlias string) error {
// ForgetLongUnseenInstances will remove entries of all instances that have long since been last seen.
func ForgetLongUnseenInstances() error {
- sqlResult, err := db.ExecVTOrc(`
- delete
- from database_instance
- where
- last_seen < NOW() - interval ? hour`,
+ sqlResult, err := db.ExecVTOrc(`DELETE
+ FROM database_instance
+ WHERE
+ last_seen < DATETIME('now', PRINTF('-%d HOUR', ?))
+ `,
config.UnseenInstanceForgetHours,
)
if err != nil {
@@ -1103,18 +1136,26 @@ func ForgetLongUnseenInstances() error {
// SnapshotTopologies records topology graph for all existing topologies
func SnapshotTopologies() error {
writeFunc := func() error {
- _, err := db.ExecVTOrc(`
- insert ignore into
- database_instance_topology_history (snapshot_unix_timestamp,
- alias, hostname, port, source_host, source_port, keyspace, shard, version)
- select
- UNIX_TIMESTAMP(NOW()),
- vitess_tablet.alias, vitess_tablet.hostname, vitess_tablet.port,
- database_instance.source_host, database_instance.source_port,
+ _, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO database_instance_topology_history (
+ snapshot_unix_timestamp,
+ alias,
+ hostname,
+ port,
+ source_host,
+ source_port,
+ keyspace,
+ shard,
+ version
+ )
+ SELECT
+ STRFTIME('%s', 'now'),
+ vitess_tablet.alias, vitess_tablet.hostname, vitess_tablet.port,
+ database_instance.source_host, database_instance.source_port,
vitess_tablet.keyspace, vitess_tablet.shard, database_instance.version
- from
- vitess_tablet left join database_instance using (alias, hostname, port)
- `,
+ FROM
+ vitess_tablet LEFT JOIN database_instance USING (alias, hostname, port)
+ `,
)
if err != nil {
log.Error(err)
@@ -1127,15 +1168,17 @@ func SnapshotTopologies() error {
}
func ExpireStaleInstanceBinlogCoordinates() error {
- expireSeconds := config.Config.ReasonableReplicationLagSeconds * 2
+ expireSeconds := config.GetReasonableReplicationLagSeconds() * 2
if expireSeconds < config.StaleInstanceCoordinatesExpireSeconds {
expireSeconds = config.StaleInstanceCoordinatesExpireSeconds
}
writeFunc := func() error {
- _, err := db.ExecVTOrc(`
- delete from database_instance_stale_binlog_coordinates
- where first_seen < NOW() - INTERVAL ? SECOND
- `, expireSeconds,
+ _, err := db.ExecVTOrc(`DELETE
+ FROM database_instance_stale_binlog_coordinates
+ WHERE
+ first_seen < DATETIME('now', PRINTF('-%d SECOND', ?))
+ `,
+ expireSeconds,
)
if err != nil {
log.Error(err)
@@ -1157,7 +1200,7 @@ func GetDatabaseState() (string, error) {
ts := tableState{
TableName: tableName,
}
- err := db.QueryVTOrc("select * from "+tableName, nil, func(rowMap sqlutils.RowMap) error {
+ err := db.QueryVTOrc("SELECT * FROM "+tableName, nil, func(rowMap sqlutils.RowMap) error {
ts.Rows = append(ts.Rows, rowMap)
return nil
})
diff --git a/go/vt/vtorc/inst/instance_dao_test.go b/go/vt/vtorc/inst/instance_dao_test.go
index 741fc48bca9..cc3217442ed 100644
--- a/go/vt/vtorc/inst/instance_dao_test.go
+++ b/go/vt/vtorc/inst/instance_dao_test.go
@@ -14,6 +14,7 @@ import (
"vitess.io/vitess/go/vt/external/golib/sqlutils"
"vitess.io/vitess/go/vt/log"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vtorc/config"
"vitess.io/vitess/go/vt/vtorc/db"
@@ -49,57 +50,48 @@ func mkTestInstances() []*Instance {
return instances
}
-func TestMkInsertOdkuSingle(t *testing.T) {
+func TestMkInsertSingle(t *testing.T) {
instances := mkTestInstances()
- sql, args, err := mkInsertOdkuForInstances(nil, true, true)
+ sql, args, err := mkInsertForInstances(nil, true, true)
require.NoError(t, err)
require.Equal(t, sql, "")
require.Equal(t, len(args), 0)
// one instance
- s1 := `INSERT ignore INTO database_instance
+ s1 := `INSERT OR IGNORE INTO database_instance
(alias, hostname, port, last_checked, last_attempted_check, last_check_partial_success, server_id, server_uuid,
version, major_version, version_comment, binlog_server, read_only, binlog_format,
binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval,
replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, mariadb_gtid, pseudo_gtid,
source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, last_discovery_latency, last_seen)
VALUES
- (?, ?, ?, NOW(), NOW(), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, NOW())
- ON DUPLICATE KEY UPDATE
- alias=VALUES(alias), hostname=VALUES(hostname), port=VALUES(port), last_checked=VALUES(last_checked), last_attempted_check=VALUES(last_attempted_check), last_check_partial_success=VALUES(last_check_partial_success), server_id=VALUES(server_id), server_uuid=VALUES(server_uuid), version=VALUES(version), major_version=VALUES(major_version), version_comment=VALUES(version_comment), binlog_server=VALUES(binlog_server), read_only=VALUES(read_only), binlog_format=VALUES(binlog_format), binlog_row_image=VALUES(binlog_row_image), log_bin=VALUES(log_bin), log_replica_updates=VALUES(log_replica_updates), binary_log_file=VALUES(binary_log_file), binary_log_pos=VALUES(binary_log_pos), source_host=VALUES(source_host), source_port=VALUES(source_port), replica_net_timeout=VALUES(replica_net_timeout), heartbeat_interval=VALUES(heartbeat_interval), replica_sql_running=VALUES(replica_sql_running), replica_io_running=VALUES(replica_io_running), replication_sql_thread_state=VALUES(replication_sql_thread_state), replication_io_thread_state=VALUES(replication_io_thread_state), has_replication_filters=VALUES(has_replication_filters), supports_oracle_gtid=VALUES(supports_oracle_gtid), oracle_gtid=VALUES(oracle_gtid), source_uuid=VALUES(source_uuid), ancestry_uuid=VALUES(ancestry_uuid), executed_gtid_set=VALUES(executed_gtid_set), gtid_mode=VALUES(gtid_mode), gtid_purged=VALUES(gtid_purged), gtid_errant=VALUES(gtid_errant), mariadb_gtid=VALUES(mariadb_gtid), pseudo_gtid=VALUES(pseudo_gtid), source_log_file=VALUES(source_log_file), read_source_log_pos=VALUES(read_source_log_pos), relay_source_log_file=VALUES(relay_source_log_file), exec_source_log_pos=VALUES(exec_source_log_pos), relay_log_file=VALUES(relay_log_file), relay_log_pos=VALUES(relay_log_pos), last_sql_error=VALUES(last_sql_error), last_io_error=VALUES(last_io_error), replication_lag_seconds=VALUES(replication_lag_seconds), replica_lag_seconds=VALUES(replica_lag_seconds), sql_delay=VALUES(sql_delay), data_center=VALUES(data_center), region=VALUES(region), physical_environment=VALUES(physical_environment), replication_depth=VALUES(replication_depth), is_co_primary=VALUES(is_co_primary), has_replication_credentials=VALUES(has_replication_credentials), allow_tls=VALUES(allow_tls),
- semi_sync_enforced=VALUES(semi_sync_enforced), semi_sync_primary_enabled=VALUES(semi_sync_primary_enabled), semi_sync_primary_timeout=VALUES(semi_sync_primary_timeout), semi_sync_primary_wait_for_replica_count=VALUES(semi_sync_primary_wait_for_replica_count), semi_sync_replica_enabled=VALUES(semi_sync_replica_enabled), semi_sync_primary_status=VALUES(semi_sync_primary_status), semi_sync_primary_clients=VALUES(semi_sync_primary_clients), semi_sync_replica_status=VALUES(semi_sync_replica_status),
- last_discovery_latency=VALUES(last_discovery_latency), last_seen=VALUES(last_seen)
+ (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now'))
`
a1 := `zone1-i710, i710, 3306, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT,
FULL, false, false, , 0, , 0, 0, 0,
false, false, 0, 0, false, false, false, , , , , , , false, false, , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0,`
- sql1, args1, err := mkInsertOdkuForInstances(instances[:1], false, true)
+ sql1, args1, err := mkInsertForInstances(instances[:1], false, true)
require.NoError(t, err)
require.Equal(t, normalizeQuery(sql1), normalizeQuery(s1))
require.Equal(t, stripSpaces(fmtArgs(args1)), stripSpaces(a1))
}
-func TestMkInsertOdkuThree(t *testing.T) {
+func TestMkInsertThree(t *testing.T) {
instances := mkTestInstances()
// three instances
- s3 := `INSERT INTO database_instance
+ s3 := `REPLACE INTO database_instance
(alias, hostname, port, last_checked, last_attempted_check, last_check_partial_success, server_id, server_uuid,
version, major_version, version_comment, binlog_server, read_only, binlog_format,
binlog_row_image, log_bin, log_replica_updates, binary_log_file, binary_log_pos, source_host, source_port, replica_net_timeout, heartbeat_interval,
replica_sql_running, replica_io_running, replication_sql_thread_state, replication_io_thread_state, has_replication_filters, supports_oracle_gtid, oracle_gtid, source_uuid, ancestry_uuid, executed_gtid_set, gtid_mode, gtid_purged, gtid_errant, mariadb_gtid, pseudo_gtid,
source_log_file, read_source_log_pos, relay_source_log_file, exec_source_log_pos, relay_log_file, relay_log_pos, last_sql_error, last_io_error, replication_lag_seconds, replica_lag_seconds, sql_delay, data_center, region, physical_environment, replication_depth, is_co_primary, has_replication_credentials, allow_tls, semi_sync_enforced, semi_sync_primary_enabled, semi_sync_primary_timeout, semi_sync_primary_wait_for_replica_count, semi_sync_replica_enabled, semi_sync_primary_status, semi_sync_primary_clients, semi_sync_replica_status, last_discovery_latency, last_seen)
VALUES
- (?, ?, ?, NOW(), NOW(), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, NOW()),
- (?, ?, ?, NOW(), NOW(), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, NOW()),
- (?, ?, ?, NOW(), NOW(), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, NOW())
- ON DUPLICATE KEY UPDATE
- alias=VALUES(alias), hostname=VALUES(hostname), port=VALUES(port), last_checked=VALUES(last_checked), last_attempted_check=VALUES(last_attempted_check), last_check_partial_success=VALUES(last_check_partial_success), server_id=VALUES(server_id), server_uuid=VALUES(server_uuid), version=VALUES(version), major_version=VALUES(major_version), version_comment=VALUES(version_comment), binlog_server=VALUES(binlog_server), read_only=VALUES(read_only), binlog_format=VALUES(binlog_format), binlog_row_image=VALUES(binlog_row_image), log_bin=VALUES(log_bin), log_replica_updates=VALUES(log_replica_updates), binary_log_file=VALUES(binary_log_file), binary_log_pos=VALUES(binary_log_pos), source_host=VALUES(source_host), source_port=VALUES(source_port), replica_net_timeout=VALUES(replica_net_timeout), heartbeat_interval=VALUES(heartbeat_interval), replica_sql_running=VALUES(replica_sql_running), replica_io_running=VALUES(replica_io_running), replication_sql_thread_state=VALUES(replication_sql_thread_state), replication_io_thread_state=VALUES(replication_io_thread_state), has_replication_filters=VALUES(has_replication_filters), supports_oracle_gtid=VALUES(supports_oracle_gtid), oracle_gtid=VALUES(oracle_gtid), source_uuid=VALUES(source_uuid), ancestry_uuid=VALUES(ancestry_uuid), executed_gtid_set=VALUES(executed_gtid_set), gtid_mode=VALUES(gtid_mode), gtid_purged=VALUES(gtid_purged), gtid_errant=VALUES(gtid_errant), mariadb_gtid=VALUES(mariadb_gtid), pseudo_gtid=VALUES(pseudo_gtid), source_log_file=VALUES(source_log_file), read_source_log_pos=VALUES(read_source_log_pos), relay_source_log_file=VALUES(relay_source_log_file), exec_source_log_pos=VALUES(exec_source_log_pos), relay_log_file=VALUES(relay_log_file), relay_log_pos=VALUES(relay_log_pos), last_sql_error=VALUES(last_sql_error), last_io_error=VALUES(last_io_error), replication_lag_seconds=VALUES(replication_lag_seconds), replica_lag_seconds=VALUES(replica_lag_seconds), sql_delay=VALUES(sql_delay), data_center=VALUES(data_center), region=VALUES(region),
- physical_environment=VALUES(physical_environment), replication_depth=VALUES(replication_depth), is_co_primary=VALUES(is_co_primary), has_replication_credentials=VALUES(has_replication_credentials), allow_tls=VALUES(allow_tls), semi_sync_enforced=VALUES(semi_sync_enforced),
- semi_sync_primary_enabled=VALUES(semi_sync_primary_enabled), semi_sync_primary_timeout=VALUES(semi_sync_primary_timeout), semi_sync_primary_wait_for_replica_count=VALUES(semi_sync_primary_wait_for_replica_count), semi_sync_replica_enabled=VALUES(semi_sync_replica_enabled), semi_sync_primary_status=VALUES(semi_sync_primary_status), semi_sync_primary_clients=VALUES(semi_sync_primary_clients), semi_sync_replica_status=VALUES(semi_sync_replica_status),
- last_discovery_latency=VALUES(last_discovery_latency), last_seen=VALUES(last_seen)
+ (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')),
+ (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now')),
+ (?, ?, ?, DATETIME('now'), DATETIME('now'), 1, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, DATETIME('now'))
`
a3 := `
zone1-i710, i710, 3306, 710, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , false, false, , 0, mysql.000007, 10, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0,
@@ -107,7 +99,7 @@ func TestMkInsertOdkuThree(t *testing.T) {
zone1-i730, i730, 3306, 730, , 5.6.7, 5.6, MySQL, false, false, STATEMENT, FULL, false, false, , 0, , 0, 0, 0, false, false, 0, 0, false, false, false, , , , , , , false, false, , 0, mysql.000007, 30, , 0, , , {0 false}, {0 false}, 0, , , , 0, false, false, false, false, false, 0, 0, false, false, 0, false, 0,
`
- sql3, args3, err := mkInsertOdkuForInstances(instances[:3], true, true)
+ sql3, args3, err := mkInsertForInstances(instances[:3], true, true)
require.NoError(t, err)
require.Equal(t, normalizeQuery(sql3), normalizeQuery(s3))
require.Equal(t, stripSpaces(fmtArgs(args3)), stripSpaces(a3))
@@ -250,11 +242,11 @@ func TestReadProblemInstances(t *testing.T) {
// We need to set InstancePollSeconds to a large value otherwise all the instances are reported as having problems since their last_checked is very old.
// Setting this value to a hundred years, we ensure that this test doesn't fail with this issue for the next hundred years.
- oldVal := config.Config.InstancePollSeconds
+ oldVal := config.GetInstancePollTime()
defer func() {
- config.Config.InstancePollSeconds = oldVal
+ config.SetInstancePollTime(oldVal)
}()
- config.Config.InstancePollSeconds = 60 * 60 * 24 * 365 * 100
+ config.SetInstancePollTime(60 * 60 * 24 * 365 * 100 * time.Second)
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
@@ -334,11 +326,11 @@ func TestReadInstancesWithErrantGTIds(t *testing.T) {
// We need to set InstancePollSeconds to a large value otherwise all the instances are reported as having problems since their last_checked is very old.
// Setting this value to a hundred years, we ensure that this test doesn't fail with this issue for the next hundred years.
- oldVal := config.Config.InstancePollSeconds
+ oldVal := config.GetInstancePollTime()
defer func() {
- config.Config.InstancePollSeconds = oldVal
+ config.SetInstancePollTime(oldVal)
}()
- config.Config.InstancePollSeconds = 60 * 60 * 24 * 365 * 100
+ config.SetInstancePollTime(60 * 60 * 24 * 365 * 100 * time.Second)
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
@@ -437,27 +429,27 @@ func TestReadOutdatedInstanceKeys(t *testing.T) {
}{
{
name: "No problems",
- sql: []string{"update database_instance set last_checked = now()"},
+ sql: []string{"update database_instance set last_checked = DATETIME('now')"},
instancesRequired: nil,
}, {
name: "One instance is outdated",
sql: []string{
- "update database_instance set last_checked = now()",
- "update database_instance set last_checked = datetime(now(), '-1 hour') where alias = 'zone1-0000000100'",
+ "update database_instance set last_checked = DATETIME('now')",
+ "update database_instance set last_checked = DATETIME('now', '-1 hour') where alias = 'zone1-0000000100'",
},
instancesRequired: []string{"zone1-0000000100"},
}, {
name: "One instance doesn't have myql data",
sql: []string{
- "update database_instance set last_checked = now()",
+ "update database_instance set last_checked = DATETIME('now')",
`INSERT INTO vitess_tablet VALUES('zone1-0000000103','localhost',7706,'ks','0','zone1',2,'0001-01-01 00:00:00+00:00','');`,
},
instancesRequired: []string{"zone1-0000000103"},
}, {
name: "One instance doesn't have myql data and one is outdated",
sql: []string{
- "update database_instance set last_checked = now()",
- "update database_instance set last_checked = datetime(now(), '-1 hour') where alias = 'zone1-0000000100'",
+ "update database_instance set last_checked = DATETIME('now')",
+ "update database_instance set last_checked = DATETIME('now', '-1 hour') where alias = 'zone1-0000000100'",
`INSERT INTO vitess_tablet VALUES('zone1-0000000103','localhost',7706,'ks','0','zone1',2,'0001-01-01 00:00:00+00:00','');`,
},
instancesRequired: []string{"zone1-0000000103", "zone1-0000000100"},
@@ -468,13 +460,13 @@ func TestReadOutdatedInstanceKeys(t *testing.T) {
waitForCacheInitialization()
// We are setting InstancePollSeconds to 59 minutes, just for the test.
- oldVal := config.Config.InstancePollSeconds
+ oldVal := config.GetInstancePollTime()
oldCache := forgetAliases
defer func() {
forgetAliases = oldCache
- config.Config.InstancePollSeconds = oldVal
+ config.SetInstancePollTime(oldVal)
}()
- config.Config.InstancePollSeconds = 60 * 25
+ config.SetInstancePollTime(60 * 25 * time.Second)
forgetAliases = cache.New(time.Minute, time.Minute)
for _, tt := range tests {
@@ -494,10 +486,10 @@ func TestReadOutdatedInstanceKeys(t *testing.T) {
errInDataCollection := db.QueryVTOrcRowsMap(`select alias,
last_checked,
last_attempted_check,
-ROUND((JULIANDAY(now()) - JULIANDAY(last_checked)) * 86400) AS difference,
+ROUND((JULIANDAY(DATETIME('now')) - JULIANDAY(last_checked)) * 86400) AS difference,
last_attempted_check <= last_checked as use1,
-last_checked < now() - interval 1500 second as is_outdated1,
-last_checked < now() - interval 3000 second as is_outdated2
+last_checked < DATETIME('now', '-1500 second') as is_outdated1,
+last_checked < DATETIME('now', '-3000 second') as is_outdated2
from database_instance`, func(rowMap sqlutils.RowMap) error {
log.Errorf("Row in database_instance - %+v", rowMap)
return nil
@@ -521,12 +513,12 @@ func TestUpdateInstanceLastChecked(t *testing.T) {
name: "Verify updated last checked",
tabletAlias: "zone1-0000000100",
partialSuccess: false,
- conditionToCheck: "last_checked >= now() - interval 30 second and last_check_partial_success = false",
+ conditionToCheck: "last_checked >= DATETIME('now', '-30 second') and last_check_partial_success = false",
}, {
name: "Verify partial success",
tabletAlias: "zone1-0000000100",
partialSuccess: true,
- conditionToCheck: "last_checked >= now() - interval 30 second and last_check_partial_success = true",
+ conditionToCheck: "last_checked >= DATETIME('now', '-30 second') and last_check_partial_success = true",
}, {
name: "Verify no error on unknown tablet",
tabletAlias: "unknown tablet",
@@ -572,7 +564,7 @@ func TestUpdateInstanceLastAttemptedCheck(t *testing.T) {
{
name: "Verify updated last checked",
tabletAlias: "zone1-0000000100",
- conditionToCheck: "last_attempted_check >= now() - interval 30 second",
+ conditionToCheck: "last_attempted_check >= DATETIME('now', '-30 second')",
}, {
name: "Verify no error on unknown tablet",
tabletAlias: "unknown tablet",
@@ -727,10 +719,10 @@ func TestGetDatabaseState(t *testing.T) {
}
func TestExpireTableData(t *testing.T) {
- oldVal := config.Config.AuditPurgeDays
- config.Config.AuditPurgeDays = 10
+ oldVal := config.GetAuditPurgeDays()
+ config.SetAuditPurgeDays(10)
defer func() {
- config.Config.AuditPurgeDays = oldVal
+ config.SetAuditPurgeDays(oldVal)
}()
tests := []struct {
@@ -745,19 +737,19 @@ func TestExpireTableData(t *testing.T) {
tableName: "audit",
timestampColumn: "audit_timestamp",
expectedRowCount: 1,
- insertQuery: `insert into audit (audit_id, audit_timestamp, audit_type, alias, message, keyspace, shard) values
-(1, NOW() - INTERVAL 50 DAY, 'a','a','a','a','a'),
-(2, NOW() - INTERVAL 5 DAY, 'a','a','a','a','a')`,
+ insertQuery: `INSERT INTO audit (audit_id, audit_timestamp, audit_type, alias, message, keyspace, shard) VALUES
+(1, DATETIME('now', '-50 DAY'), 'a','a','a','a','a'),
+(2, DATETIME('now', '-5 DAY'), 'a','a','a','a','a')`,
},
{
name: "ExpireRecoveryDetectionHistory",
tableName: "recovery_detection",
timestampColumn: "detection_timestamp",
expectedRowCount: 2,
- insertQuery: `insert into recovery_detection (detection_id, detection_timestamp, alias, analysis, keyspace, shard) values
-(1, NOW() - INTERVAL 3 DAY,'a','a','a','a'),
-(2, NOW() - INTERVAL 5 DAY,'a','a','a','a'),
-(3, NOW() - INTERVAL 15 DAY,'a','a','a','a')`,
+ insertQuery: `INSERT INTO recovery_detection (detection_id, detection_timestamp, alias, analysis, keyspace, shard) VALUES
+(1, DATETIME('now', '-3 DAY'),'a','a','a','a'),
+(2, DATETIME('now', '-5 DAY'),'a','a','a','a'),
+(3, DATETIME('now', '-15 DAY'),'a','a','a','a')`,
},
}
for _, tt := range tests {
@@ -782,3 +774,165 @@ func TestExpireTableData(t *testing.T) {
})
}
}
+
+func TestDetectErrantGTIDs(t *testing.T) {
+ tests := []struct {
+ name string
+ instance *Instance
+ primaryInstance *Instance
+ wantErr bool
+ wantErrantGTID string
+ }{
+ {
+ name: "No errant GTIDs",
+ instance: &Instance{
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34",
+ primaryExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10591,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34",
+ AncestryUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff,230ea8ea-81e3-11e4-972a-e25ec4bd140a",
+ ServerUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ SourceUUID: "230ea8ea-81e3-11e4-972a-e25ec4bd140a",
+ },
+ }, {
+ name: "Errant GTIDs on replica",
+ instance: &Instance{
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:34",
+ primaryExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10591,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34",
+ AncestryUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff,230ea8ea-81e3-11e4-972a-e25ec4bd140a",
+ ServerUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ SourceUUID: "230ea8ea-81e3-11e4-972a-e25ec4bd140a",
+ },
+ wantErrantGTID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff:34",
+ },
+ {
+ name: "No errant GTIDs on old primary",
+ instance: &Instance{
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-341",
+ AncestryUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ ServerUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ },
+ primaryInstance: &Instance{
+ SourceHost: "",
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10589,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-341",
+ },
+ },
+ {
+ name: "Errant GTIDs on old primary",
+ instance: &Instance{
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-342",
+ AncestryUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ ServerUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ },
+ primaryInstance: &Instance{
+ SourceHost: "",
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10589,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-341",
+ },
+ wantErrantGTID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff:342",
+ }, {
+ name: "Old information for new primary",
+ instance: &Instance{
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-342",
+ AncestryUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ ServerUUID: "316d193c-70e5-11e5-adb2-ecf4bb2262ff",
+ },
+ primaryInstance: &Instance{
+ SourceHost: "localhost",
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10539,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-311",
+ },
+ },
+ }
+
+ keyspaceName := "ks"
+ shardName := "0"
+ tablet := &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{
+ Cell: "zone-1",
+ Uid: 100,
+ },
+ Keyspace: keyspaceName,
+ Shard: shardName,
+ }
+ primaryTablet := &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{
+ Cell: "zone-1",
+ Uid: 101,
+ },
+ Keyspace: keyspaceName,
+ Shard: shardName,
+ }
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ // Clear the database after the test. The easiest way to do that is to run all the initialization commands again.
+ defer func() {
+ db.ClearVTOrcDatabase()
+ }()
+ db.ClearVTOrcDatabase()
+
+ // Save shard record for the primary tablet.
+ err := SaveShard(topo.NewShardInfo(keyspaceName, shardName, &topodatapb.Shard{
+ PrimaryAlias: primaryTablet.Alias,
+ }, nil))
+ require.NoError(t, err)
+
+ if tt.primaryInstance != nil {
+ tt.primaryInstance.InstanceAlias = topoproto.TabletAliasString(primaryTablet.Alias)
+ err = SaveTablet(primaryTablet)
+ require.NoError(t, err)
+ err = WriteInstance(tt.primaryInstance, true, nil)
+ require.NoError(t, err)
+ }
+
+ tt.instance.InstanceAlias = topoproto.TabletAliasString(tablet.Alias)
+ err = detectErrantGTIDs(tt.instance, tablet)
+ if tt.wantErr {
+ require.Error(t, err)
+ return
+ }
+ require.NoError(t, err)
+ require.EqualValues(t, tt.wantErrantGTID, tt.instance.GtidErrant)
+ })
+ }
+}
+
+// TestPrimaryErrantGTIDs tests that we don't run Errant GTID detection on the primary tablet itself!
+func TestPrimaryErrantGTIDs(t *testing.T) {
+ // Clear the database after the test. The easiest way to do that is to run all the initialization commands again.
+ defer func() {
+ db.ClearVTOrcDatabase()
+ }()
+ db.ClearVTOrcDatabase()
+ keyspaceName := "ks"
+ shardName := "0"
+ tablet := &topodatapb.Tablet{
+ Alias: &topodatapb.TabletAlias{
+ Cell: "zone-1",
+ Uid: 100,
+ },
+ Keyspace: keyspaceName,
+ Shard: shardName,
+ }
+ instance := &Instance{
+ SourceHost: "",
+ ExecutedGtidSet: "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10589,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-341",
+ InstanceAlias: topoproto.TabletAliasString(tablet.Alias),
+ }
+
+ // Save shard record for the primary tablet.
+ err := SaveShard(topo.NewShardInfo(keyspaceName, shardName, &topodatapb.Shard{
+ PrimaryAlias: tablet.Alias,
+ }, nil))
+ require.NoError(t, err)
+
+ // Store the tablet record and the instance.
+ err = SaveTablet(tablet)
+ require.NoError(t, err)
+ err = WriteInstance(instance, true, nil)
+ require.NoError(t, err)
+
+ // After this if we read a new information for the record that updates its
+ // gtid set further, we shouldn't be detecting errant GTIDs on it since it is the primary!
+ // We shouldn't be comparing it with a previous version of itself!
+ instance.ExecutedGtidSet = "230ea8ea-81e3-11e4-972a-e25ec4bd140a:1-10589,8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-34,316d193c-70e5-11e5-adb2-ecf4bb2262ff:1-351"
+ err = detectErrantGTIDs(instance, tablet)
+ require.NoError(t, err)
+ require.EqualValues(t, "", instance.GtidErrant)
+}
diff --git a/go/vt/vtorc/inst/tablet_dao.go b/go/vt/vtorc/inst/tablet_dao.go
index af304292a70..f48f2b97370 100644
--- a/go/vt/vtorc/inst/tablet_dao.go
+++ b/go/vt/vtorc/inst/tablet_dao.go
@@ -56,13 +56,13 @@ func fullStatus(tabletAlias string) (*replicationdatapb.FullStatus, error) {
// ReadTablet reads the vitess tablet record.
func ReadTablet(tabletAlias string) (*topodatapb.Tablet, error) {
- query := `
- select
- info
- from
- vitess_tablet
- where alias = ?
- `
+ query := `SELECT
+ info
+ FROM
+ vitess_tablet
+ WHERE
+ alias = ?
+ `
args := sqlutils.Args(tabletAlias)
tablet := &topodatapb.Tablet{}
opts := prototext.UnmarshalOptions{DiscardUnknown: true}
@@ -84,14 +84,28 @@ func SaveTablet(tablet *topodatapb.Tablet) error {
if err != nil {
return err
}
- _, err = db.ExecVTOrc(`
- replace
- into vitess_tablet (
- alias, hostname, port, cell, keyspace, shard, tablet_type, primary_timestamp, info
- ) values (
- ?, ?, ?, ?, ?, ?, ?, ?, ?
- )
- `,
+ _, err = db.ExecVTOrc(`REPLACE
+ INTO vitess_tablet (
+ alias,
+ hostname,
+ port,
+ cell,
+ keyspace,
+ shard,
+ tablet_type,
+ primary_timestamp,
+ info
+ ) VALUES (
+ ?,
+ ?,
+ ?,
+ ?,
+ ?,
+ ?,
+ ?,
+ ?,
+ ?
+ )`,
topoproto.TabletAliasString(tablet.Alias),
tablet.MysqlHostname,
int(tablet.MysqlPort),
diff --git a/go/vt/vtorc/logic/disable_recovery.go b/go/vt/vtorc/logic/disable_recovery.go
index 4a3766055d2..c5446eeb9ff 100644
--- a/go/vt/vtorc/logic/disable_recovery.go
+++ b/go/vt/vtorc/logic/disable_recovery.go
@@ -30,6 +30,7 @@ package logic
// go to the database each time.
import (
+ "errors"
"fmt"
"vitess.io/vitess/go/vt/external/golib/sqlutils"
@@ -39,14 +40,13 @@ import (
// IsRecoveryDisabled returns true if Recoveries are disabled globally
func IsRecoveryDisabled() (disabled bool, err error) {
- query := `
- SELECT
- COUNT(*) as mycount
- FROM
- global_recovery_disable
- WHERE
- disable_recovery=?
- `
+ query := `SELECT
+ COUNT(*) AS mycount
+ FROM
+ global_recovery_disable
+ WHERE
+ disable_recovery = ?
+ `
err = db.QueryVTOrc(query, sqlutils.Args(1), func(m sqlutils.RowMap) error {
mycount := m.GetInt("mycount")
disabled = (mycount > 0)
@@ -55,28 +55,26 @@ func IsRecoveryDisabled() (disabled bool, err error) {
if err != nil {
errMsg := fmt.Sprintf("recovery.IsRecoveryDisabled(): %v", err)
log.Errorf(errMsg)
- err = fmt.Errorf(errMsg)
+ err = errors.New(errMsg)
}
return disabled, err
}
// DisableRecovery ensures recoveries are disabled globally
func DisableRecovery() error {
- _, err := db.ExecVTOrc(`
- INSERT IGNORE INTO global_recovery_disable
- (disable_recovery)
- VALUES (1)
- `,
- )
+ _, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO global_recovery_disable (
+ disable_recovery
+ ) VALUES (1)`)
return err
}
// EnableRecovery ensures recoveries are enabled globally
func EnableRecovery() error {
// The "WHERE" clause is just to avoid full-scan reports by monitoring tools
- _, err := db.ExecVTOrc(`
- DELETE FROM global_recovery_disable WHERE disable_recovery >= 0
- `,
- )
+ _, err := db.ExecVTOrc(`DELETE
+ FROM global_recovery_disable
+ WHERE
+ disable_recovery >= 0`)
return err
}
diff --git a/go/vt/vtorc/logic/keyspace_shard_discovery.go b/go/vt/vtorc/logic/keyspace_shard_discovery.go
index b1e93fe2a01..0dd17cb65fd 100644
--- a/go/vt/vtorc/logic/keyspace_shard_discovery.go
+++ b/go/vt/vtorc/logic/keyspace_shard_discovery.go
@@ -29,17 +29,16 @@ import (
)
// RefreshAllKeyspacesAndShards reloads the keyspace and shard information for the keyspaces that vtorc is concerned with.
-func RefreshAllKeyspacesAndShards() {
+func RefreshAllKeyspacesAndShards(ctx context.Context) error {
var keyspaces []string
if len(clustersToWatch) == 0 { // all known keyspaces
- ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer cancel()
var err error
// Get all the keyspaces
keyspaces, err = ts.GetKeyspaces(ctx)
if err != nil {
- log.Error(err)
- return
+ return err
}
} else {
// Parse input and build list of keyspaces
@@ -55,14 +54,14 @@ func RefreshAllKeyspacesAndShards() {
}
if len(keyspaces) == 0 {
log.Errorf("Found no keyspaces for input: %+v", clustersToWatch)
- return
+ return nil
}
}
// Sort the list of keyspaces.
// The list can have duplicates because the input to clusters to watch may have multiple shards of the same keyspace
sort.Strings(keyspaces)
- refreshCtx, refreshCancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ refreshCtx, refreshCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer refreshCancel()
var wg sync.WaitGroup
for idx, keyspace := range keyspaces {
@@ -83,6 +82,8 @@ func RefreshAllKeyspacesAndShards() {
}(keyspace)
}
wg.Wait()
+
+ return nil
}
// RefreshKeyspaceAndShard refreshes the keyspace record and shard record for the given keyspace and shard.
diff --git a/go/vt/vtorc/logic/keyspace_shard_discovery_test.go b/go/vt/vtorc/logic/keyspace_shard_discovery_test.go
index 097865db84a..5cbe139728b 100644
--- a/go/vt/vtorc/logic/keyspace_shard_discovery_test.go
+++ b/go/vt/vtorc/logic/keyspace_shard_discovery_test.go
@@ -92,7 +92,7 @@ func TestRefreshAllKeyspaces(t *testing.T) {
// Set clusters to watch to only watch ks1 and ks3
onlyKs1and3 := []string{"ks1/-80", "ks3/-80", "ks3/80-"}
clustersToWatch = onlyKs1and3
- RefreshAllKeyspacesAndShards()
+ require.NoError(t, RefreshAllKeyspacesAndShards(context.Background()))
// Verify that we only have ks1 and ks3 in vtorc's db.
verifyKeyspaceInfo(t, "ks1", keyspaceDurabilityNone, "")
@@ -107,7 +107,7 @@ func TestRefreshAllKeyspaces(t *testing.T) {
clustersToWatch = nil
// Change the durability policy of ks1
reparenttestutil.SetKeyspaceDurability(ctx, t, ts, "ks1", "semi_sync")
- RefreshAllKeyspacesAndShards()
+ require.NoError(t, RefreshAllKeyspacesAndShards(context.Background()))
// Verify that all the keyspaces are correctly reloaded
verifyKeyspaceInfo(t, "ks1", keyspaceDurabilitySemiSync, "")
diff --git a/go/vt/vtorc/logic/tablet_discovery.go b/go/vt/vtorc/logic/tablet_discovery.go
index 593b846a72e..7066229ab06 100644
--- a/go/vt/vtorc/logic/tablet_discovery.go
+++ b/go/vt/vtorc/logic/tablet_discovery.go
@@ -27,7 +27,6 @@ import (
"time"
"github.com/spf13/pflag"
-
"google.golang.org/protobuf/encoding/prototext"
"google.golang.org/protobuf/proto"
@@ -35,11 +34,9 @@ import (
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
- "vitess.io/vitess/go/vt/topotools"
"vitess.io/vitess/go/vt/vtorc/config"
"vitess.io/vitess/go/vt/vtorc/db"
"vitess.io/vitess/go/vt/vtorc/inst"
- "vitess.io/vitess/go/vt/vtorc/process"
"vitess.io/vitess/go/vt/vttablet/tmclient"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
@@ -67,39 +64,36 @@ func OpenTabletDiscovery() <-chan time.Time {
ts = topo.Open()
tmc = inst.InitializeTMC()
// Clear existing cache and perform a new refresh.
- if _, err := db.ExecVTOrc("delete from vitess_tablet"); err != nil {
+ if _, err := db.ExecVTOrc("DELETE FROM vitess_tablet"); err != nil {
log.Error(err)
}
- // We refresh all information from the topo once before we start the ticks to do it on a timer.
- populateAllInformation()
- return time.Tick(time.Second * time.Duration(config.Config.TopoInformationRefreshSeconds)) //nolint SA1015: using time.Tick leaks the underlying ticker
-}
-
-// populateAllInformation initializes all the information for VTOrc to function.
-func populateAllInformation() {
- refreshAllInformation()
- // We have completed one full discovery cycle. We should update the process health.
- process.FirstDiscoveryCycleComplete.Store(true)
+ // We refresh all information from the topo once before we start the ticks to do
+ // it on a timer.
+ ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ defer cancel()
+ if err := refreshAllInformation(ctx); err != nil {
+ log.Errorf("failed to initialize topo information: %+v", err)
+ }
+ return time.Tick(config.GetTopoInformationRefreshDuration()) //nolint SA1015: using time.Tick leaks the underlying ticker
}
// refreshAllTablets reloads the tablets from topo and discovers the ones which haven't been refreshed in a while
-func refreshAllTablets() {
- refreshTabletsUsing(func(tabletAlias string) {
+func refreshAllTablets(ctx context.Context) error {
+ return refreshTabletsUsing(ctx, func(tabletAlias string) {
DiscoverInstance(tabletAlias, false /* forceDiscovery */)
}, false /* forceRefresh */)
}
-func refreshTabletsUsing(loader func(tabletAlias string), forceRefresh bool) {
+func refreshTabletsUsing(ctx context.Context, loader func(tabletAlias string), forceRefresh bool) error {
if len(clustersToWatch) == 0 { // all known clusters
- ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer cancel()
cells, err := ts.GetKnownCells(ctx)
if err != nil {
- log.Error(err)
- return
+ return err
}
- refreshCtx, refreshCancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ refreshCtx, refreshCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer refreshCancel()
var wg sync.WaitGroup
for _, cell := range cells {
@@ -120,7 +114,7 @@ func refreshTabletsUsing(loader func(tabletAlias string), forceRefresh bool) {
keyspaceShards = append(keyspaceShards, &topo.KeyspaceShard{Keyspace: input[0], Shard: input[1]})
} else {
// Assume this is a keyspace and find all shards in keyspace
- ctx, cancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer cancel()
shards, err := ts.GetShardNames(ctx, ks)
if err != nil {
@@ -139,9 +133,9 @@ func refreshTabletsUsing(loader func(tabletAlias string), forceRefresh bool) {
}
if len(keyspaceShards) == 0 {
log.Errorf("Found no keyspaceShards for input: %+v", clustersToWatch)
- return
+ return nil
}
- refreshCtx, refreshCancel := context.WithTimeout(context.Background(), topo.RemoteOperationTimeout)
+ refreshCtx, refreshCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
defer refreshCancel()
var wg sync.WaitGroup
for _, ks := range keyspaceShards {
@@ -153,10 +147,11 @@ func refreshTabletsUsing(loader func(tabletAlias string), forceRefresh bool) {
}
wg.Wait()
}
+ return nil
}
func refreshTabletsInCell(ctx context.Context, cell string, loader func(tabletAlias string), forceRefresh bool) {
- tablets, err := topotools.GetTabletMapForCell(ctx, ts, cell)
+ tablets, err := ts.GetTabletsByCell(ctx, cell, nil)
if err != nil {
log.Errorf("Error fetching topo info for cell %v: %v", cell, err)
return
@@ -188,7 +183,7 @@ func refreshTabletInfoOfShard(ctx context.Context, keyspace, shard string) {
}
func refreshTabletsInKeyspaceShard(ctx context.Context, keyspace, shard string, loader func(tabletAlias string), forceRefresh bool, tabletsToIgnore []string) {
- tablets, err := ts.GetTabletMapForShard(ctx, keyspace, shard)
+ tablets, err := ts.GetTabletsByShard(ctx, keyspace, shard)
if err != nil {
log.Errorf("Error fetching tablets for keyspace/shard %v/%v: %v", keyspace, shard, err)
return
@@ -198,15 +193,12 @@ func refreshTabletsInKeyspaceShard(ctx context.Context, keyspace, shard string,
refreshTablets(tablets, query, args, loader, forceRefresh, tabletsToIgnore)
}
-func refreshTablets(tablets map[string]*topo.TabletInfo, query string, args []any, loader func(tabletAlias string), forceRefresh bool, tabletsToIgnore []string) {
+func refreshTablets(tablets []*topo.TabletInfo, query string, args []any, loader func(tabletAlias string), forceRefresh bool, tabletsToIgnore []string) {
// Discover new tablets.
latestInstances := make(map[string]bool)
var wg sync.WaitGroup
for _, tabletInfo := range tablets {
tablet := tabletInfo.Tablet
- if tablet.Type != topodatapb.TabletType_PRIMARY && !topo.IsReplicaType(tablet.Type) {
- continue
- }
tabletAliasString := topoproto.TabletAliasString(tablet.Alias)
latestInstances[tabletAliasString] = true
old, err := inst.ReadTablet(tabletAliasString)
diff --git a/go/vt/vtorc/logic/tablet_discovery_test.go b/go/vt/vtorc/logic/tablet_discovery_test.go
index 7acb29dcc5b..f6a7af38382 100644
--- a/go/vt/vtorc/logic/tablet_discovery_test.go
+++ b/go/vt/vtorc/logic/tablet_discovery_test.go
@@ -37,7 +37,6 @@ import (
"vitess.io/vitess/go/vt/vtctl/grpcvtctldserver/testutil"
"vitess.io/vitess/go/vt/vtorc/db"
"vitess.io/vitess/go/vt/vtorc/inst"
- "vitess.io/vitess/go/vt/vtorc/process"
)
var (
@@ -200,6 +199,8 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) {
return nil
})
tab100.MysqlPort = 100
+ // We refresh once more to ensure we don't affect the next tests since we've made a change again.
+ refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias string) {}, false, nil)
}()
// Let's assume tab100 restarted on a different pod. This would change its tablet hostname and port
_, err = ts.UpdateTabletFields(context.Background(), tab100.Alias, func(tablet *topodatapb.Tablet) error {
@@ -212,6 +213,26 @@ func TestRefreshTabletsInKeyspaceShard(t *testing.T) {
// Also the old tablet should be forgotten
verifyRefreshTabletsInKeyspaceShard(t, false, 1, tablets, nil)
})
+
+ t.Run("Replica becomes a drained tablet", func(t *testing.T) {
+ defer func() {
+ _, err = ts.UpdateTabletFields(context.Background(), tab101.Alias, func(tablet *topodatapb.Tablet) error {
+ tablet.Type = topodatapb.TabletType_REPLICA
+ return nil
+ })
+ tab101.Type = topodatapb.TabletType_REPLICA
+ // We refresh once more to ensure we don't affect the next tests since we've made a change again.
+ refreshTabletsInKeyspaceShard(ctx, keyspace, shard, func(tabletAlias string) {}, false, nil)
+ }()
+ // A replica tablet can be converted to drained type if it has an errant GTID.
+ _, err = ts.UpdateTabletFields(context.Background(), tab101.Alias, func(tablet *topodatapb.Tablet) error {
+ tablet.Type = topodatapb.TabletType_DRAINED
+ return nil
+ })
+ tab101.Type = topodatapb.TabletType_DRAINED
+ // We expect 1 tablet to be refreshed since its type has been changed.
+ verifyRefreshTabletsInKeyspaceShard(t, false, 1, tablets, nil)
+ })
}
func TestShardPrimary(t *testing.T) {
@@ -347,25 +368,6 @@ func TestGetLockAction(t *testing.T) {
}
}
-// TestProcessHealth tests that the health of the process reflects that we have run the first discovery once correctly.
-func TestProcessHealth(t *testing.T) {
- require.False(t, process.FirstDiscoveryCycleComplete.Load())
- originalTs := ts
- defer func() {
- ts = originalTs
- process.FirstDiscoveryCycleComplete.Store(false)
- }()
- // Verify in the beginning, we have the first DiscoveredOnce field false.
- _, discoveredOnce := process.HealthTest()
- require.False(t, discoveredOnce)
- ts = memorytopo.NewServer(context.Background(), cell1)
- populateAllInformation()
- require.True(t, process.FirstDiscoveryCycleComplete.Load())
- // Verify after we populate all information, we have the first DiscoveredOnce field true.
- _, discoveredOnce = process.HealthTest()
- require.True(t, discoveredOnce)
-}
-
func TestSetReadOnly(t *testing.T) {
tests := []struct {
name string
diff --git a/go/vt/vtorc/logic/topology_recovery.go b/go/vt/vtorc/logic/topology_recovery.go
index aec137a45b4..f14eca624c9 100644
--- a/go/vt/vtorc/logic/topology_recovery.go
+++ b/go/vt/vtorc/logic/topology_recovery.go
@@ -21,7 +21,6 @@ import (
"encoding/json"
"fmt"
"math/rand/v2"
- "time"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/vt/log"
@@ -235,8 +234,8 @@ func runEmergencyReparentOp(ctx context.Context, analysisEntry *inst.Replication
tablet.Shard,
reparentutil.EmergencyReparentOptions{
IgnoreReplicas: nil,
- WaitReplicasTimeout: time.Duration(config.Config.WaitReplicasTimeoutSeconds) * time.Second,
- PreventCrossCellPromotion: config.Config.PreventCrossDataCenterPrimaryFailover,
+ WaitReplicasTimeout: config.GetWaitReplicasTimeout(),
+ PreventCrossCellPromotion: config.GetPreventCrossCellFailover(),
WaitAllTablets: waitForAllTablets,
},
)
@@ -703,8 +702,8 @@ func electNewPrimary(ctx context.Context, analysisEntry *inst.ReplicationAnalysi
analyzedTablet.Keyspace,
analyzedTablet.Shard,
reparentutil.PlannedReparentOptions{
- WaitReplicasTimeout: time.Duration(config.Config.WaitReplicasTimeoutSeconds) * time.Second,
- TolerableReplLag: time.Duration(config.Config.TolerableReplicationLagSeconds) * time.Second,
+ WaitReplicasTimeout: config.GetWaitReplicasTimeout(),
+ TolerableReplLag: config.GetTolerableReplicationLag(),
},
)
diff --git a/go/vt/vtorc/logic/topology_recovery_dao.go b/go/vt/vtorc/logic/topology_recovery_dao.go
index e8af34bdad4..137251c4fc8 100644
--- a/go/vt/vtorc/logic/topology_recovery_dao.go
+++ b/go/vt/vtorc/logic/topology_recovery_dao.go
@@ -17,6 +17,7 @@
package logic
import (
+ "errors"
"fmt"
"strings"
@@ -29,21 +30,20 @@ import (
// InsertRecoveryDetection inserts the recovery analysis that has been detected.
func InsertRecoveryDetection(analysisEntry *inst.ReplicationAnalysis) error {
- sqlResult, err := db.ExecVTOrc(`
- insert ignore
- into recovery_detection (
- alias,
- analysis,
- keyspace,
- shard,
- detection_timestamp
- ) values (
- ?,
- ?,
- ?,
- ?,
- now()
- )`,
+ sqlResult, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO recovery_detection (
+ alias,
+ analysis,
+ keyspace,
+ shard,
+ detection_timestamp
+ ) VALUES (
+ ?,
+ ?,
+ ?,
+ ?,
+ DATETIME('now')
+ )`,
analysisEntry.AnalyzedInstanceAlias,
string(analysisEntry.Analysis),
analysisEntry.ClusterDetails.Keyspace,
@@ -64,26 +64,24 @@ func InsertRecoveryDetection(analysisEntry *inst.ReplicationAnalysis) error {
func writeTopologyRecovery(topologyRecovery *TopologyRecovery) (*TopologyRecovery, error) {
analysisEntry := topologyRecovery.AnalysisEntry
- sqlResult, err := db.ExecVTOrc(`
- insert ignore
- into topology_recovery (
- recovery_id,
- alias,
- start_recovery,
- analysis,
- keyspace,
- shard,
- detection_id
- ) values (
- ?,
- ?,
- NOW(),
- ?,
- ?,
- ?,
- ?
- )
- `,
+ sqlResult, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO topology_recovery (
+ recovery_id,
+ alias,
+ start_recovery,
+ analysis,
+ keyspace,
+ shard,
+ detection_id
+ ) VALUES (
+ ?,
+ ?,
+ DATETIME('now'),
+ ?,
+ ?,
+ ?,
+ ?
+ )`,
sqlutils.NilIfZero(topologyRecovery.ID),
analysisEntry.AnalyzedInstanceAlias,
string(analysisEntry.Analysis),
@@ -121,7 +119,7 @@ func AttemptRecoveryRegistration(analysisEntry *inst.ReplicationAnalysis) (*Topo
if len(recoveries) > 0 {
errMsg := fmt.Sprintf("AttemptRecoveryRegistration: Active recovery (id:%v) in the cluster %s:%s for %s", recoveries[0].ID, analysisEntry.ClusterDetails.Keyspace, analysisEntry.ClusterDetails.Shard, recoveries[0].AnalysisEntry.Analysis)
log.Errorf(errMsg)
- return nil, fmt.Errorf(errMsg)
+ return nil, errors.New(errMsg)
}
topologyRecovery := NewTopologyRecovery(*analysisEntry)
@@ -137,15 +135,16 @@ func AttemptRecoveryRegistration(analysisEntry *inst.ReplicationAnalysis) (*Topo
// ResolveRecovery is called on completion of a recovery process and updates the recovery status.
// It does not clear the "active period" as this still takes place in order to avoid flapping.
func writeResolveRecovery(topologyRecovery *TopologyRecovery) error {
- _, err := db.ExecVTOrc(`
- update topology_recovery set
- is_successful = ?,
- successor_alias = ?,
- all_errors = ?,
- end_recovery = NOW()
- where
- recovery_id = ?
- `, topologyRecovery.IsSuccessful,
+ _, err := db.ExecVTOrc(`UPDATE topology_recovery
+ SET
+ is_successful = ?,
+ successor_alias = ?,
+ all_errors = ?,
+ end_recovery = DATETIME('now')
+ WHERE
+ recovery_id = ?
+ `,
+ topologyRecovery.IsSuccessful,
topologyRecovery.SuccessorAlias,
strings.Join(topologyRecovery.AllErrors, "\n"),
topologyRecovery.ID,
@@ -159,26 +158,27 @@ func writeResolveRecovery(topologyRecovery *TopologyRecovery) error {
// readRecoveries reads recovery entry/audit entries from topology_recovery
func readRecoveries(whereCondition string, limit string, args []any) ([]*TopologyRecovery, error) {
res := []*TopologyRecovery{}
- query := fmt.Sprintf(`
- select
- recovery_id,
- alias,
- start_recovery,
- IFNULL(end_recovery, '') AS end_recovery,
- is_successful,
- ifnull(successor_alias, '') as successor_alias,
- analysis,
- keyspace,
- shard,
- all_errors,
- detection_id
- from
+ query := fmt.Sprintf(`SELECT
+ recovery_id,
+ alias,
+ start_recovery,
+ IFNULL(end_recovery, '') AS end_recovery,
+ is_successful,
+ IFNULL(successor_alias, '') AS successor_alias,
+ analysis,
+ keyspace,
+ shard,
+ all_errors,
+ detection_id
+ FROM
topology_recovery
%s
- order by
- recovery_id desc
+ ORDER BY recovery_id DESC
%s
- `, whereCondition, limit)
+ `,
+ whereCondition,
+ limit,
+ )
err := db.QueryVTOrc(query, args, func(m sqlutils.RowMap) error {
topologyRecovery := *NewTopologyRecovery(inst.ReplicationAnalysis{})
topologyRecovery.ID = m.GetInt64("recovery_id")
@@ -210,11 +210,10 @@ func readRecoveries(whereCondition string, limit string, args []any) ([]*Topolog
// ReadActiveClusterRecoveries reads recoveries that are ongoing for the given cluster.
func ReadActiveClusterRecoveries(keyspace string, shard string) ([]*TopologyRecovery, error) {
- whereClause := `
- where
- end_recovery IS NULL
- and keyspace=?
- and shard=?`
+ whereClause := `WHERE
+ end_recovery IS NULL
+ AND keyspace = ?
+ AND shard = ?`
return readRecoveries(whereClause, ``, sqlutils.Args(keyspace, shard))
}
@@ -224,23 +223,30 @@ func ReadRecentRecoveries(page int) ([]*TopologyRecovery, error) {
whereClause := ""
var args []any
if len(whereConditions) > 0 {
- whereClause = fmt.Sprintf("where %s", strings.Join(whereConditions, " and "))
+ whereClause = fmt.Sprintf("WHERE %s", strings.Join(whereConditions, " AND "))
}
- limit := `
- limit ?
- offset ?`
+ limit := `LIMIT ? OFFSET ?`
args = append(args, config.AuditPageSize, page*config.AuditPageSize)
return readRecoveries(whereClause, limit, args)
}
// writeTopologyRecoveryStep writes down a single step in a recovery process
func writeTopologyRecoveryStep(topologyRecoveryStep *TopologyRecoveryStep) error {
- sqlResult, err := db.ExecVTOrc(`
- insert ignore
- into topology_recovery_steps (
- recovery_step_id, recovery_id, audit_at, message
- ) values (?, ?, now(), ?)
- `, sqlutils.NilIfZero(topologyRecoveryStep.ID), topologyRecoveryStep.RecoveryID, topologyRecoveryStep.Message,
+ sqlResult, err := db.ExecVTOrc(`INSERT OR IGNORE
+ INTO topology_recovery_steps (
+ recovery_step_id,
+ recovery_id,
+ audit_at,
+ message
+ ) VALUES (
+ ?,
+ ?,
+ DATETIME('now'),
+ ?
+ )`,
+ sqlutils.NilIfZero(topologyRecoveryStep.ID),
+ topologyRecoveryStep.RecoveryID,
+ topologyRecoveryStep.Message,
)
if err != nil {
log.Error(err)
diff --git a/go/vt/vtorc/logic/topology_recovery_dao_test.go b/go/vt/vtorc/logic/topology_recovery_dao_test.go
index 354af82e2b3..6a1d7c4c48f 100644
--- a/go/vt/vtorc/logic/topology_recovery_dao_test.go
+++ b/go/vt/vtorc/logic/topology_recovery_dao_test.go
@@ -70,10 +70,10 @@ func TestTopologyRecovery(t *testing.T) {
}
func TestExpireTableData(t *testing.T) {
- oldVal := config.Config.AuditPurgeDays
- config.Config.AuditPurgeDays = 10
+ oldVal := config.GetAuditPurgeDays()
+ config.SetAuditPurgeDays(10)
defer func() {
- config.Config.AuditPurgeDays = oldVal
+ config.SetAuditPurgeDays(oldVal)
}()
tests := []struct {
@@ -88,9 +88,9 @@ func TestExpireTableData(t *testing.T) {
tableName: "recovery_detection",
expectedRowCount: 2,
insertQuery: `insert into recovery_detection (detection_id, detection_timestamp, alias, analysis, keyspace, shard) values
-(1, NOW() - INTERVAL 3 DAY,'a','a','a','a'),
-(2, NOW() - INTERVAL 5 DAY,'a','a','a','a'),
-(3, NOW() - INTERVAL 15 DAY,'a','a','a','a')`,
+(1, datetime('now', '-3 DAY'),'a','a','a','a'),
+(2, datetime('now', '-5 DAY'),'a','a','a','a'),
+(3, datetime('now', '-15 DAY'),'a','a','a','a')`,
expireFunc: ExpireRecoveryDetectionHistory,
},
{
@@ -98,9 +98,9 @@ func TestExpireTableData(t *testing.T) {
tableName: "topology_recovery",
expectedRowCount: 1,
insertQuery: `insert into topology_recovery (recovery_id, start_recovery, alias, analysis, keyspace, shard) values
-(1, NOW() - INTERVAL 13 DAY,'a','a','a','a'),
-(2, NOW() - INTERVAL 5 DAY,'a','a','a','a'),
-(3, NOW() - INTERVAL 15 DAY,'a','a','a','a')`,
+(1, datetime('now', '-13 DAY'),'a','a','a','a'),
+(2, datetime('now', '-5 DAY'),'a','a','a','a'),
+(3, datetime('now', '-15 DAY'),'a','a','a','a')`,
expireFunc: ExpireTopologyRecoveryHistory,
},
{
@@ -108,9 +108,9 @@ func TestExpireTableData(t *testing.T) {
tableName: "topology_recovery_steps",
expectedRowCount: 1,
insertQuery: `insert into topology_recovery_steps (recovery_step_id, audit_at, recovery_id, message) values
-(1, NOW() - INTERVAL 13 DAY, 1, 'a'),
-(2, NOW() - INTERVAL 5 DAY, 2, 'a'),
-(3, NOW() - INTERVAL 15 DAY, 3, 'a')`,
+(1, datetime('now', '-13 DAY'), 1, 'a'),
+(2, datetime('now', '-5 DAY'), 2, 'a'),
+(3, datetime('now', '-15 DAY'), 3, 'a')`,
expireFunc: ExpireTopologyRecoveryStepsHistory,
},
}
diff --git a/go/vt/vtorc/logic/vtorc.go b/go/vt/vtorc/logic/vtorc.go
index 9a468d1508a..39326525ce2 100644
--- a/go/vt/vtorc/logic/vtorc.go
+++ b/go/vt/vtorc/logic/vtorc.go
@@ -17,15 +17,14 @@
package logic
import (
- "os"
- "os/signal"
+ "context"
"sync"
"sync/atomic"
- "syscall"
"time"
"github.com/patrickmn/go-cache"
"github.com/sjmudd/stopwatch"
+ "golang.org/x/sync/errgroup"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/vt/log"
@@ -35,6 +34,7 @@ import (
"vitess.io/vitess/go/vt/vtorc/discovery"
"vitess.io/vitess/go/vt/vtorc/inst"
ometrics "vitess.io/vitess/go/vt/vtorc/metrics"
+ "vitess.io/vitess/go/vt/vtorc/process"
"vitess.io/vitess/go/vt/vtorc/util"
)
@@ -73,26 +73,6 @@ func init() {
})
}
-// used in several places
-func instancePollSecondsDuration() time.Duration {
- return time.Duration(config.Config.InstancePollSeconds) * time.Second
-}
-
-// acceptSighupSignal registers for SIGHUP signal from the OS to reload the configuration files.
-func acceptSighupSignal() {
- c := make(chan os.Signal, 1)
-
- signal.Notify(c, syscall.SIGHUP)
- go func() {
- for range c {
- log.Infof("Received SIGHUP. Reloading configuration")
- _ = inst.AuditOperation("reload-configuration", "", "Triggered via SIGHUP")
- config.Reload()
- discoveryMetrics.SetExpirePeriod(time.Duration(config.DiscoveryCollectionRetentionSeconds) * time.Second)
- }
- }()
-}
-
// closeVTOrc runs all the operations required to cleanly shutdown VTOrc
func closeVTOrc() {
log.Infof("Starting VTOrc shutdown")
@@ -161,7 +141,7 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) {
defer func() {
latency.Stop("total")
discoveryTime := latency.Elapsed("total")
- if discoveryTime > instancePollSecondsDuration() {
+ if discoveryTime > config.GetInstancePollTime() {
instancePollSecondsExceededCounter.Add(1)
log.Warningf("discoverInstance exceeded InstancePollSeconds for %+v, took %.4fs", tabletAlias, discoveryTime.Seconds())
if metric != nil {
@@ -177,7 +157,7 @@ func DiscoverInstance(tabletAlias string, forceDiscovery bool) {
// Calculate the expiry period each time as InstancePollSeconds
// _may_ change during the run of the process (via SIGHUP) and
// it is not possible to change the cache's default expiry..
- if existsInCacheError := recentDiscoveryOperationKeys.Add(tabletAlias, true, instancePollSecondsDuration()); existsInCacheError != nil && !forceDiscovery {
+ if existsInCacheError := recentDiscoveryOperationKeys.Add(tabletAlias, true, config.GetInstancePollTime()); existsInCacheError != nil && !forceDiscovery {
// Just recently attempted
return
}
@@ -271,24 +251,23 @@ func onHealthTick() {
// nolint SA1015: using time.Tick leaks the underlying ticker
func ContinuousDiscovery() {
log.Infof("continuous discovery: setting up")
- recentDiscoveryOperationKeys = cache.New(instancePollSecondsDuration(), time.Second)
+ recentDiscoveryOperationKeys = cache.New(config.GetInstancePollTime(), time.Second)
go handleDiscoveryRequests()
healthTick := time.Tick(config.HealthPollSeconds * time.Second)
caretakingTick := time.Tick(time.Minute)
- recoveryTick := time.Tick(time.Duration(config.Config.RecoveryPollSeconds) * time.Second)
+ recoveryTick := time.Tick(config.GetRecoveryPollDuration())
tabletTopoTick := OpenTabletDiscovery()
var recoveryEntrance int64
var snapshotTopologiesTick <-chan time.Time
- if config.Config.SnapshotTopologiesIntervalHours > 0 {
- snapshotTopologiesTick = time.Tick(time.Duration(config.Config.SnapshotTopologiesIntervalHours) * time.Hour)
+ if config.GetSnapshotTopologyInterval() > 0 {
+ snapshotTopologiesTick = time.Tick(config.GetSnapshotTopologyInterval())
}
go func() {
_ = ometrics.InitMetrics()
}()
- go acceptSighupSignal()
// On termination of the server, we should close VTOrc cleanly
servenv.OnTermSync(closeVTOrc)
@@ -328,30 +307,34 @@ func ContinuousDiscovery() {
go inst.SnapshotTopologies()
}()
case <-tabletTopoTick:
- refreshAllInformation()
+ ctx, cancel := context.WithTimeout(context.Background(), config.GetTopoInformationRefreshDuration())
+ if err := refreshAllInformation(ctx); err != nil {
+ log.Errorf("failed to refresh topo information: %+v", err)
+ }
+ cancel()
}
}
}
// refreshAllInformation refreshes both shard and tablet information. This is meant to be run on tablet topo ticks.
-func refreshAllInformation() {
- // Create a wait group
- var wg sync.WaitGroup
+func refreshAllInformation(ctx context.Context) error {
+ // Create an errgroup
+ eg, ctx := errgroup.WithContext(ctx)
// Refresh all keyspace information.
- wg.Add(1)
- go func() {
- defer wg.Done()
- RefreshAllKeyspacesAndShards()
- }()
+ eg.Go(func() error {
+ return RefreshAllKeyspacesAndShards(ctx)
+ })
// Refresh all tablets.
- wg.Add(1)
- go func() {
- defer wg.Done()
- refreshAllTablets()
- }()
+ eg.Go(func() error {
+ return refreshAllTablets(ctx)
+ })
// Wait for both the refreshes to complete
- wg.Wait()
+ err := eg.Wait()
+ if err == nil {
+ process.FirstDiscoveryCycleComplete.Store(true)
+ }
+ return err
}
diff --git a/go/vt/vtorc/logic/vtorc_test.go b/go/vt/vtorc/logic/vtorc_test.go
index c8f2ac3bfdc..edd8141e8b7 100644
--- a/go/vt/vtorc/logic/vtorc_test.go
+++ b/go/vt/vtorc/logic/vtorc_test.go
@@ -1,11 +1,17 @@
package logic
import (
+ "context"
"sync/atomic"
"testing"
"time"
"github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/vt/topo/memorytopo"
+ "vitess.io/vitess/go/vt/vtorc/db"
+ "vitess.io/vitess/go/vt/vtorc/process"
)
func TestWaitForLocksRelease(t *testing.T) {
@@ -54,3 +60,41 @@ func waitForLocksReleaseAndGetTimeWaitedFor() time.Duration {
waitForLocksRelease()
return time.Since(start)
}
+
+func TestRefreshAllInformation(t *testing.T) {
+ // Store the old flags and restore on test completion
+ oldTs := ts
+ defer func() {
+ ts = oldTs
+ }()
+
+ // Clear the database after the test. The easiest way to do that is to run all the initialization commands again.
+ defer func() {
+ db.ClearVTOrcDatabase()
+ }()
+
+ // Verify in the beginning, we have the first DiscoveredOnce field false.
+ _, discoveredOnce := process.HealthTest()
+ require.False(t, discoveredOnce)
+
+ // Create a memory topo-server and create the keyspace and shard records
+ ts = memorytopo.NewServer(context.Background(), cell1)
+ _, err := ts.GetOrCreateShard(context.Background(), keyspace, shard)
+ require.NoError(t, err)
+
+ // Test error
+ ctx, cancel := context.WithCancel(context.Background())
+ cancel() // cancel context to simulate timeout
+ require.Error(t, refreshAllInformation(ctx))
+ require.False(t, process.FirstDiscoveryCycleComplete.Load())
+ _, discoveredOnce = process.HealthTest()
+ require.False(t, discoveredOnce)
+
+ // Test success
+ ctx2, cancel2 := context.WithCancel(context.Background())
+ defer cancel2()
+ require.NoError(t, refreshAllInformation(ctx2))
+ require.True(t, process.FirstDiscoveryCycleComplete.Load())
+ _, discoveredOnce = process.HealthTest()
+ require.True(t, discoveredOnce)
+}
diff --git a/go/vt/vtorc/process/health.go b/go/vt/vtorc/process/health.go
index 86101d6c5c0..d448f03bb83 100644
--- a/go/vt/vtorc/process/health.go
+++ b/go/vt/vtorc/process/health.go
@@ -35,12 +35,17 @@ var ThisNodeHealth = &NodeHealth{}
// writeHealthToDatabase writes to the database and returns if it was successful.
func writeHealthToDatabase() bool {
- _, err := db.ExecVTOrc("delete from node_health")
+ _, err := db.ExecVTOrc("DELETE FROM node_health")
if err != nil {
log.Error(err)
return false
}
- sqlResult, err := db.ExecVTOrc(`insert into node_health (last_seen_active) values (now())`)
+ sqlResult, err := db.ExecVTOrc(`INSERT
+ INTO node_health (
+ last_seen_active
+ ) VALUES (
+ DATETIME('now')
+ )`)
if err != nil {
log.Error(err)
return false
@@ -57,7 +62,7 @@ func writeHealthToDatabase() bool {
func HealthTest() (health *NodeHealth, discoveredOnce bool) {
ThisNodeHealth.LastReported = time.Now()
discoveredOnce = FirstDiscoveryCycleComplete.Load()
- ThisNodeHealth.Healthy = writeHealthToDatabase()
+ ThisNodeHealth.Healthy = discoveredOnce && writeHealthToDatabase()
return ThisNodeHealth, discoveredOnce
}
diff --git a/go/vt/vtorc/process/health_test.go b/go/vt/vtorc/process/health_test.go
new file mode 100644
index 00000000000..c198deda4e4
--- /dev/null
+++ b/go/vt/vtorc/process/health_test.go
@@ -0,0 +1,46 @@
+/*
+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 process
+
+import (
+ "testing"
+
+ "github.com/stretchr/testify/require"
+)
+
+func TestHealthTest(t *testing.T) {
+ defer func() {
+ FirstDiscoveryCycleComplete.Store(false)
+ ThisNodeHealth = &NodeHealth{}
+ }()
+
+ require.Zero(t, ThisNodeHealth.LastReported)
+ require.False(t, ThisNodeHealth.Healthy)
+
+ ThisNodeHealth = &NodeHealth{}
+ health, discoveredOnce := HealthTest()
+ require.False(t, health.Healthy)
+ require.False(t, discoveredOnce)
+ require.NotZero(t, ThisNodeHealth.LastReported)
+
+ ThisNodeHealth = &NodeHealth{}
+ FirstDiscoveryCycleComplete.Store(true)
+ health, discoveredOnce = HealthTest()
+ require.True(t, health.Healthy)
+ require.True(t, discoveredOnce)
+ require.NotZero(t, ThisNodeHealth.LastReported)
+}
diff --git a/go/vt/vtorc/server/api.go b/go/vt/vtorc/server/api.go
index 5e9a84c0a29..177f2c80333 100644
--- a/go/vt/vtorc/server/api.go
+++ b/go/vt/vtorc/server/api.go
@@ -25,6 +25,7 @@ import (
"time"
"vitess.io/vitess/go/acl"
+ "vitess.io/vitess/go/viperutil/debug"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/vtorc/collection"
"vitess.io/vitess/go/vt/vtorc/discovery"
@@ -46,6 +47,7 @@ const (
enableGlobalRecoveriesAPI = "/api/enable-global-recoveries"
replicationAnalysisAPI = "/api/replication-analysis"
databaseStateAPI = "/api/database-state"
+ configAPI = "/api/config"
healthAPI = "/debug/health"
AggregatedDiscoveryMetricsAPI = "/api/aggregated-discovery-metrics"
@@ -62,6 +64,7 @@ var (
enableGlobalRecoveriesAPI,
replicationAnalysisAPI,
databaseStateAPI,
+ configAPI,
healthAPI,
AggregatedDiscoveryMetricsAPI,
}
@@ -90,6 +93,8 @@ func (v *vtorcAPI) ServeHTTP(response http.ResponseWriter, request *http.Request
replicationAnalysisAPIHandler(response, request)
case databaseStateAPI:
databaseStateAPIHandler(response)
+ case configAPI:
+ configAPIHandler(response)
case AggregatedDiscoveryMetricsAPI:
AggregatedDiscoveryMetricsAPIHandler(response, request)
default:
@@ -106,7 +111,7 @@ func getACLPermissionLevelForAPI(apiEndpoint string) string {
return acl.MONITORING
case disableGlobalRecoveriesAPI, enableGlobalRecoveriesAPI:
return acl.ADMIN
- case replicationAnalysisAPI:
+ case replicationAnalysisAPI, configAPI:
return acl.MONITORING
case healthAPI, databaseStateAPI:
return acl.MONITORING
@@ -180,6 +185,17 @@ func databaseStateAPIHandler(response http.ResponseWriter) {
writePlainTextResponse(response, ds, http.StatusOK)
}
+// configAPIHandler is the handler for the configAPI endpoint
+func configAPIHandler(response http.ResponseWriter) {
+ settingsMap := debug.AllSettings()
+ jsonOut, err := json.MarshalIndent(settingsMap, "", "\t")
+ if err != nil {
+ http.Error(response, err.Error(), http.StatusInternalServerError)
+ return
+ }
+ writePlainTextResponse(response, string(jsonOut), http.StatusOK)
+}
+
// AggregatedDiscoveryMetricsAPIHandler is the handler for the discovery metrics endpoint
func AggregatedDiscoveryMetricsAPIHandler(response http.ResponseWriter, request *http.Request) {
// return metrics for last x seconds
diff --git a/go/vt/vtorc/server/api_test.go b/go/vt/vtorc/server/api_test.go
index c352d1e600f..ab6b9eed9af 100644
--- a/go/vt/vtorc/server/api_test.go
+++ b/go/vt/vtorc/server/api_test.go
@@ -31,6 +31,9 @@ func TestGetACLPermissionLevelForAPI(t *testing.T) {
}, {
apiEndpoint: healthAPI,
want: acl.MONITORING,
+ }, {
+ apiEndpoint: configAPI,
+ want: acl.MONITORING,
}, {
apiEndpoint: "gibberish",
want: acl.ADMIN,
diff --git a/go/vt/vttablet/common/config.go b/go/vt/vttablet/common/config.go
new file mode 100644
index 00000000000..72047ce4580
--- /dev/null
+++ b/go/vt/vttablet/common/config.go
@@ -0,0 +1,281 @@
+/*
+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 vttablet
+
+import (
+ "encoding/json"
+ "fmt"
+ "maps"
+ "strconv"
+ "strings"
+ "sync"
+ "time"
+)
+
+/*
+ This file contains the model for all the configuration parameters for VReplication workflows. It also provides methods to
+ initialize the default configuration and to override the default configuration with user-provided values. The overrides
+ are stored in the `config` sub-document of the `options` attribute in `_vt.vreplication` and merged with the defaults
+ when the workflow is initialized.
+*/
+
+// VReplicationConfig has the all the configuration parameters for VReplication workflows, both applicable on the
+// target (vreplication)and the source (vstreamer) side.
+type VReplicationConfig struct {
+ // Config parameters applicable to the target side (vreplication)
+ ExperimentalFlags int64
+ NetReadTimeout int
+ NetWriteTimeout int
+ CopyPhaseDuration time.Duration
+ RetryDelay time.Duration
+ MaxTimeToRetryError time.Duration
+ RelayLogMaxSize int
+ RelayLogMaxItems int
+ ReplicaLagTolerance time.Duration
+ HeartbeatUpdateInterval int
+ StoreCompressedGTID bool
+ ParallelInsertWorkers int
+ TabletTypesStr string
+
+ // Config parameters applicable to the source side (vstreamer)
+ // The coresponding Override fields are used to determine if the user has provided a value for the parameter so
+ // that they can be sent in the VStreamer API calls to the source.
+ VStreamPacketSize int
+ VStreamPacketSizeOverride bool
+ VStreamDynamicPacketSize bool
+ VStreamDynamicPacketSizeOverride bool
+ VStreamBinlogRotationThreshold int64
+ VStreamBinlogRotationThresholdOverride bool
+
+ // Overrides is a map of user-provided configuration values that override the default configuration.
+ Overrides map[string]string
+}
+
+var configMutex sync.Mutex
+
+// DefaultVReplicationConfig has the default values for VReplicationConfig initialized from the vttablet flags
+// when the workflow is initialized.
+var DefaultVReplicationConfig *VReplicationConfig
+
+// GetVReplicationConfigDefaults returns the default VReplicationConfig. If `useCached` is true, it returns the previously
+// loaded configuration. Otherwise it reloads the configuration from the vttablet flags. useCached is set to false
+// when the vttablet flags are updated in unit tests.
+func GetVReplicationConfigDefaults(useCached bool) *VReplicationConfig {
+ configMutex.Lock()
+ defer configMutex.Unlock()
+ if useCached && DefaultVReplicationConfig != nil {
+ return DefaultVReplicationConfig
+ }
+ DefaultVReplicationConfig = &VReplicationConfig{
+ ExperimentalFlags: vreplicationExperimentalFlags,
+ NetReadTimeout: vreplicationNetReadTimeout,
+ NetWriteTimeout: vreplicationNetWriteTimeout,
+ CopyPhaseDuration: vreplicationCopyPhaseDuration,
+ RetryDelay: vreplicationRetryDelay,
+ MaxTimeToRetryError: vreplicationMaxTimeToRetryError,
+ RelayLogMaxSize: vreplicationRelayLogMaxSize,
+ RelayLogMaxItems: vreplicationRelayLogMaxItems,
+ ReplicaLagTolerance: vreplicationReplicaLagTolerance,
+ HeartbeatUpdateInterval: vreplicationHeartbeatUpdateInterval,
+ StoreCompressedGTID: vreplicationStoreCompressedGTID,
+ ParallelInsertWorkers: vreplicationParallelInsertWorkers,
+ TabletTypesStr: vreplicationTabletTypesStr,
+
+ VStreamPacketSizeOverride: false,
+ VStreamPacketSize: VStreamerDefaultPacketSize,
+ VStreamDynamicPacketSizeOverride: false,
+ VStreamDynamicPacketSize: VStreamerUseDynamicPacketSize,
+ VStreamBinlogRotationThresholdOverride: false,
+ VStreamBinlogRotationThreshold: VStreamerBinlogRotationThreshold,
+
+ Overrides: make(map[string]string),
+ }
+ return DefaultVReplicationConfig
+}
+
+// InitVReplicationConfigDefaults initializes the default VReplicationConfig in an idempotent way.
+func InitVReplicationConfigDefaults() *VReplicationConfig {
+ return GetVReplicationConfigDefaults(true)
+}
+
+// GetDefaultVReplicationConfig returns a copy of the default VReplicationConfig.
+func GetDefaultVReplicationConfig() *VReplicationConfig {
+ c := &VReplicationConfig{}
+ defaultConfig := GetVReplicationConfigDefaults(true)
+ *c = *defaultConfig
+ return c
+}
+
+// NewVReplicationConfig creates a new VReplicationConfig by merging the default configuration with the user-provided
+// overrides. It returns an error if the user-provided values are invalid.
+func NewVReplicationConfig(overrides map[string]string) (*VReplicationConfig, error) {
+ c := GetDefaultVReplicationConfig()
+ c.Overrides = maps.Clone(overrides)
+ var errors []string
+ getError := func(k, v string) string {
+ return fmt.Sprintf("invalid value for %s: %s", k, v)
+ }
+ for k, v := range overrides {
+ if v == "" {
+ continue
+ }
+ switch k {
+ case "vreplication_experimental_flags":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.ExperimentalFlags = value
+ }
+ case "vreplication_net_read_timeout":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.NetReadTimeout = int(value)
+ }
+ case "vreplication_net_write_timeout":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.NetWriteTimeout = int(value)
+ }
+ case "vreplication_copy_phase_duration":
+ value, err := time.ParseDuration(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.CopyPhaseDuration = value
+ }
+ case "vreplication_retry_delay":
+ value, err := time.ParseDuration(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.RetryDelay = value
+ }
+ case "vreplication_max_time_to_retry_on_error":
+ value, err := time.ParseDuration(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.MaxTimeToRetryError = value
+ }
+ case "relay_log_max_size":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.RelayLogMaxSize = int(value)
+ }
+ case "relay_log_max_items":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.RelayLogMaxItems = int(value)
+ }
+ case "vreplication_replica_lag_tolerance":
+ value, err := time.ParseDuration(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.ReplicaLagTolerance = value
+ }
+ case "vreplication_heartbeat_update_interval":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.HeartbeatUpdateInterval = int(value)
+ }
+ case "vreplication_store_compressed_gtid":
+ value, err := strconv.ParseBool(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.StoreCompressedGTID = value
+ }
+ case "vreplication-parallel-insert-workers":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.ParallelInsertWorkers = int(value)
+ }
+ case "vstream_packet_size":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.VStreamPacketSizeOverride = true
+ c.VStreamPacketSize = int(value)
+ }
+ case "vstream_dynamic_packet_size":
+ value, err := strconv.ParseBool(v)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.VStreamDynamicPacketSizeOverride = true
+ c.VStreamDynamicPacketSize = value
+ }
+ case "vstream_binlog_rotation_threshold":
+ value, err := strconv.ParseInt(v, 10, 64)
+ if err != nil {
+ errors = append(errors, getError(k, v))
+ } else {
+ c.VStreamBinlogRotationThresholdOverride = true
+ c.VStreamBinlogRotationThreshold = value
+ }
+ default:
+ errors = append(errors, fmt.Sprintf("unknown vreplication config flag: %s", k))
+ }
+ }
+ if len(errors) > 0 {
+ return c, fmt.Errorf("%s", strings.Join(errors, ", "))
+ }
+ return c, nil
+}
+
+// Map returns a map of the VReplicationConfig: the keys are the flag names and the values are string representations.
+// Used in tests to compare the expected and actual configuration values and in validations to check if the user-provided
+// keys are one of those that are supported.
+func (c VReplicationConfig) Map() map[string]string {
+ return map[string]string{
+ "vreplication_experimental_flags": strconv.FormatInt(c.ExperimentalFlags, 10),
+ "vreplication_net_read_timeout": strconv.Itoa(c.NetReadTimeout),
+ "vreplication_net_write_timeout": strconv.Itoa(c.NetWriteTimeout),
+ "vreplication_copy_phase_duration": c.CopyPhaseDuration.String(),
+ "vreplication_retry_delay": c.RetryDelay.String(),
+ "vreplication_max_time_to_retry_on_error": c.MaxTimeToRetryError.String(),
+ "relay_log_max_size": strconv.Itoa(c.RelayLogMaxSize),
+ "relay_log_max_items": strconv.Itoa(c.RelayLogMaxItems),
+ "vreplication_replica_lag_tolerance": c.ReplicaLagTolerance.String(),
+ "vreplication_heartbeat_update_interval": strconv.Itoa(c.HeartbeatUpdateInterval),
+ "vreplication_store_compressed_gtid": strconv.FormatBool(c.StoreCompressedGTID),
+ "vreplication-parallel-insert-workers": strconv.Itoa(c.ParallelInsertWorkers),
+ "vstream_packet_size": strconv.Itoa(c.VStreamPacketSize),
+ "vstream_dynamic_packet_size": strconv.FormatBool(c.VStreamDynamicPacketSize),
+ "vstream_binlog_rotation_threshold": strconv.FormatInt(c.VStreamBinlogRotationThreshold, 10),
+ }
+}
+
+func (c VReplicationConfig) String() string {
+ s, _ := json.Marshal(c.Map())
+ return string(s)
+}
diff --git a/go/vt/vttablet/common/config_test.go b/go/vt/vttablet/common/config_test.go
new file mode 100644
index 00000000000..c73259036f2
--- /dev/null
+++ b/go/vt/vttablet/common/config_test.go
@@ -0,0 +1,159 @@
+/*
+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 vttablet
+
+import (
+ "strconv"
+ "strings"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/require"
+)
+
+func TestNewVReplicationConfig(t *testing.T) {
+ InitVReplicationConfigDefaults()
+ tests := []struct {
+ name string
+ config map[string]string
+ wantErr int
+ want *VReplicationConfig
+ }{
+ {
+ name: "Valid values",
+ config: map[string]string{
+ "vreplication_experimental_flags": "3",
+ "vreplication_net_read_timeout": "100",
+ "vreplication_net_write_timeout": "200",
+ "vreplication_copy_phase_duration": "2h",
+ "vreplication_retry_delay": "10s",
+ "vreplication_max_time_to_retry_on_error": "1h",
+ "relay_log_max_size": "500000",
+ "relay_log_max_items": "10000",
+ "vreplication_replica_lag_tolerance": "2m",
+ "vreplication_heartbeat_update_interval": "2",
+ "vreplication_store_compressed_gtid": "true",
+ "vreplication-parallel-insert-workers": "4",
+ "vstream_packet_size": "1024",
+ "vstream_dynamic_packet_size": "false",
+ "vstream_binlog_rotation_threshold": "2048",
+ },
+ wantErr: 0,
+ want: &VReplicationConfig{
+ ExperimentalFlags: 3,
+ NetReadTimeout: 100,
+ NetWriteTimeout: 200,
+ CopyPhaseDuration: 2 * time.Hour,
+ RetryDelay: 10 * time.Second,
+ MaxTimeToRetryError: 1 * time.Hour,
+ RelayLogMaxSize: 500000,
+ RelayLogMaxItems: 10000,
+ ReplicaLagTolerance: 2 * time.Minute,
+ HeartbeatUpdateInterval: 2,
+ StoreCompressedGTID: true,
+ ParallelInsertWorkers: 4,
+ VStreamPacketSize: 1024,
+ VStreamDynamicPacketSize: false,
+ VStreamBinlogRotationThreshold: 2048,
+ TabletTypesStr: DefaultVReplicationConfig.TabletTypesStr,
+ VStreamPacketSizeOverride: true,
+ VStreamDynamicPacketSizeOverride: true,
+ VStreamBinlogRotationThresholdOverride: true,
+ },
+ },
+ {
+ name: "Invalid values",
+ config: map[string]string{
+ "vreplication_experimental_flags": "invalid",
+ "vreplication_net_read_timeout": "100.0",
+ "vreplication_net_write_timeout": "invalid",
+ "vreplication_copy_phase_duration": "invalid",
+ "vreplication_retry_delay": "invalid",
+ "vreplication_max_time_to_retry_on_error": "invalid",
+ "relay_log_max_size": "invalid",
+ "relay_log_max_items": "invalid",
+ "vreplication_replica_lag_tolerance": "invalid",
+ "vreplication_heartbeat_update_interval": "invalid",
+ "vreplication_store_compressed_gtid": "nottrue",
+ "vreplication-parallel-insert-workers": "invalid",
+ "vstream_packet_size": "invalid",
+ "vstream_dynamic_packet_size": "waar",
+ "vstream_binlog_rotation_threshold": "invalid",
+ },
+ wantErr: 15,
+ },
+ {
+ name: "Partial values",
+ config: map[string]string{
+ "vreplication_experimental_flags": "5",
+ "vreplication_net_read_timeout": "150",
+ "vstream_dynamic_packet_size": strconv.FormatBool(!DefaultVReplicationConfig.VStreamDynamicPacketSize),
+ "vreplication_store_compressed_gtid": strconv.FormatBool(!DefaultVReplicationConfig.StoreCompressedGTID),
+ },
+ wantErr: 0,
+ want: &VReplicationConfig{
+ ExperimentalFlags: 5,
+ NetReadTimeout: 150,
+ NetWriteTimeout: DefaultVReplicationConfig.NetWriteTimeout,
+ CopyPhaseDuration: DefaultVReplicationConfig.CopyPhaseDuration,
+ RetryDelay: DefaultVReplicationConfig.RetryDelay,
+ MaxTimeToRetryError: DefaultVReplicationConfig.MaxTimeToRetryError,
+ RelayLogMaxSize: DefaultVReplicationConfig.RelayLogMaxSize,
+ RelayLogMaxItems: DefaultVReplicationConfig.RelayLogMaxItems,
+ ReplicaLagTolerance: DefaultVReplicationConfig.ReplicaLagTolerance,
+ HeartbeatUpdateInterval: DefaultVReplicationConfig.HeartbeatUpdateInterval,
+ StoreCompressedGTID: !DefaultVReplicationConfig.StoreCompressedGTID,
+ ParallelInsertWorkers: DefaultVReplicationConfig.ParallelInsertWorkers,
+ VStreamPacketSize: DefaultVReplicationConfig.VStreamPacketSize,
+ VStreamDynamicPacketSize: !DefaultVReplicationConfig.VStreamDynamicPacketSize,
+ VStreamBinlogRotationThreshold: DefaultVReplicationConfig.VStreamBinlogRotationThreshold,
+ VStreamDynamicPacketSizeOverride: true,
+ TabletTypesStr: DefaultVReplicationConfig.TabletTypesStr,
+ },
+ },
+ }
+
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ InitVReplicationConfigDefaults()
+ got, err := NewVReplicationConfig(tt.config)
+ if tt.wantErr == 0 {
+ require.EqualValuesf(t, tt.config, got.Overrides,
+ "NewVReplicationConfig() overrides got = %v, want %v", got.Overrides, tt.config)
+
+ }
+ if tt.wantErr > 0 && err == nil || tt.wantErr == 0 && err != nil {
+ t.Errorf("NewVReplicationConfig() got num errors = %v, want %v", err, tt.wantErr)
+ }
+ if tt.wantErr > 0 && err != nil {
+ errors := strings.Split(err.Error(), ", ")
+ if len(errors) != tt.wantErr {
+ t.Errorf("NewVReplicationConfig() got num errors = %v, want %v", len(errors), tt.wantErr)
+ }
+ }
+ if tt.want == nil {
+ require.EqualValuesf(t, DefaultVReplicationConfig.Map(), got.Map(),
+ "NewVReplicationConfig() Map got = %v, want %v", got.Map(), DefaultVReplicationConfig.Map())
+ } else {
+ tt.want.Overrides = tt.config
+ require.EqualValues(t, tt.want.Map(), got.Map(),
+ "NewVReplicationConfig() Map got = %v, want %v", got.Map(), tt.want.Map())
+ }
+
+ })
+ }
+}
diff --git a/go/vt/vttablet/common/flags.go b/go/vt/vttablet/common/flags.go
new file mode 100644
index 00000000000..75e8e58982f
--- /dev/null
+++ b/go/vt/vttablet/common/flags.go
@@ -0,0 +1,99 @@
+/*
+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 vttablet
+
+import (
+ "time"
+
+ "github.com/spf13/pflag"
+
+ "vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/vt/servenv"
+)
+
+const (
+ // VReplicationExperimentalFlags is a bitmask of experimental features in vreplication.
+ VReplicationExperimentalFlagOptimizeInserts = int64(1)
+ VReplicationExperimentalFlagAllowNoBlobBinlogRowImage = int64(2)
+ VReplicationExperimentalFlagVPlayerBatching = int64(4)
+)
+
+var (
+ vreplicationExperimentalFlags = VReplicationExperimentalFlagOptimizeInserts | VReplicationExperimentalFlagAllowNoBlobBinlogRowImage | VReplicationExperimentalFlagVPlayerBatching
+ vreplicationNetReadTimeout = 300
+ vreplicationNetWriteTimeout = 600
+ vreplicationCopyPhaseDuration = 1 * time.Hour
+ vreplicationRetryDelay = 5 * time.Second
+ vreplicationMaxTimeToRetryError = 0 * time.Second // Default behavior is to keep retrying, for backward compatibility
+
+ vreplicationTabletTypesStr = "in_order:REPLICA,PRIMARY" // Default value
+
+ vreplicationRelayLogMaxSize = 250000
+ vreplicationRelayLogMaxItems = 5000
+
+ vreplicationReplicaLagTolerance = 1 * time.Minute
+
+ vreplicationHeartbeatUpdateInterval = 1
+
+ vreplicationStoreCompressedGTID = false
+ vreplicationParallelInsertWorkers = 1
+
+ // VStreamerBinlogRotationThreshold is the threshold, above which we rotate binlogs, before taking a GTID snapshot
+ VStreamerBinlogRotationThreshold = int64(64 * 1024 * 1024) // 64MiB
+ VStreamerDefaultPacketSize = 250000
+ VStreamerUseDynamicPacketSize = true
+)
+
+func GetVReplicationNetReadTimeout() int {
+ return vreplicationNetReadTimeout
+}
+func GetVReplicationNetWriteTimeout() int {
+ return vreplicationNetWriteTimeout
+}
+
+func init() {
+ servenv.OnParseFor("vttablet", registerFlags)
+ servenv.OnParseFor("vtcombo", registerFlags)
+}
+
+func registerFlags(fs *pflag.FlagSet) {
+ fs.Int64Var(&vreplicationExperimentalFlags, "vreplication_experimental_flags", vreplicationExperimentalFlags,
+ "(Bitmask) of experimental features in vreplication to enable")
+ fs.IntVar(&vreplicationNetReadTimeout, "vreplication_net_read_timeout", vreplicationNetReadTimeout, "Session value of net_read_timeout for vreplication, in seconds")
+ fs.IntVar(&vreplicationNetWriteTimeout, "vreplication_net_write_timeout", vreplicationNetWriteTimeout, "Session value of net_write_timeout for vreplication, in seconds")
+ fs.DurationVar(&vreplicationCopyPhaseDuration, "vreplication_copy_phase_duration", vreplicationCopyPhaseDuration, "Duration for each copy phase loop (before running the next catchup: default 1h)")
+ fs.DurationVar(&vreplicationRetryDelay, "vreplication_retry_delay", vreplicationRetryDelay, "delay before retrying a failed workflow event in the replication phase")
+ fs.DurationVar(&vreplicationMaxTimeToRetryError, "vreplication_max_time_to_retry_on_error", vreplicationMaxTimeToRetryError, "stop automatically retrying when we've had consecutive failures with the same error for this long after the first occurrence")
+
+ fs.IntVar(&vreplicationRelayLogMaxSize, "relay_log_max_size", vreplicationRelayLogMaxSize, "Maximum buffer size (in bytes) for vreplication target buffering. If single rows are larger than this, a single row is buffered at a time.")
+ fs.IntVar(&vreplicationRelayLogMaxItems, "relay_log_max_items", vreplicationRelayLogMaxItems, "Maximum number of rows for vreplication target buffering.")
+
+ fs.DurationVar(&vreplicationReplicaLagTolerance, "vreplication_replica_lag_tolerance", vreplicationReplicaLagTolerance, "Replica lag threshold duration: once lag is below this we switch from copy phase to the replication (streaming) phase")
+
+ // vreplicationHeartbeatUpdateInterval determines how often the time_updated column is updated if there are no
+ // real events on the source and the source vstream is only sending heartbeats for this long. Keep this low if you
+ // expect high QPS and are monitoring this column to alert about potential outages. Keep this high if
+ // * you have too many streams the extra write qps or cpu load due to these updates are unacceptable
+ // * you have too many streams and/or a large source field (lot of participating tables) which generates
+ // unacceptable increase in your binlog size
+ fs.IntVar(&vreplicationHeartbeatUpdateInterval, "vreplication_heartbeat_update_interval", vreplicationHeartbeatUpdateInterval, "Frequency (in seconds, default 1, max 60) at which the time_updated column of a vreplication stream when idling")
+ fs.BoolVar(&vreplicationStoreCompressedGTID, "vreplication_store_compressed_gtid", vreplicationStoreCompressedGTID, "Store compressed gtids in the pos column of the sidecar database's vreplication table")
+
+ fs.IntVar(&vreplicationParallelInsertWorkers, "vreplication-parallel-insert-workers", vreplicationParallelInsertWorkers, "Number of parallel insertion workers to use during copy phase. Set <= 1 to disable parallelism, or > 1 to enable concurrent insertion during copy phase.")
+
+ fs.Uint64Var(&mysql.ZstdInMemoryDecompressorMaxSize, "binlog-in-memory-decompressor-max-size", mysql.ZstdInMemoryDecompressorMaxSize, "This value sets the uncompressed transaction payload size at which we switch from in-memory buffer based decompression to the slower streaming mode.")
+}
diff --git a/go/vt/vttablet/endtoend/call_test.go b/go/vt/vttablet/endtoend/call_test.go
index a1a2eae792a..95171e1523e 100644
--- a/go/vt/vttablet/endtoend/call_test.go
+++ b/go/vt/vttablet/endtoend/call_test.go
@@ -20,7 +20,6 @@ import (
"testing"
"github.com/stretchr/testify/assert"
-
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/vt/vttablet/endtoend/framework"
@@ -136,6 +135,7 @@ func TestCallProcedureLeakTx(t *testing.T) {
func TestCallProcedureChangedTx(t *testing.T) {
client := framework.NewClient()
+ defer client.Release()
_, err := client.Execute(`call proc_tx_begin()`, nil)
require.EqualError(t, err, "Transaction not concluded inside the stored procedure, leaking transaction from stored procedure is not allowed (CallerID: dev)")
diff --git a/go/vt/vttablet/endtoend/config_test.go b/go/vt/vttablet/endtoend/config_test.go
index 3902113f354..c3ad5f8a9db 100644
--- a/go/vt/vttablet/endtoend/config_test.go
+++ b/go/vt/vttablet/endtoend/config_test.go
@@ -23,7 +23,6 @@ import (
"time"
"github.com/stretchr/testify/assert"
-
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/sqltypes"
@@ -37,7 +36,7 @@ import (
)
func TestPoolSize(t *testing.T) {
- revert := changeVar(t, "PoolSize", "1")
+ revert := changeVar(t, "ReadPoolSize", "1")
defer revert()
vstart := framework.DebugVars()
@@ -73,6 +72,48 @@ func TestStreamPoolSize(t *testing.T) {
verifyIntValue(t, vstart, "StreamConnPoolCapacity", 1)
}
+// TestTxPoolSize starts 2 transactions, one in normal pool and one in found rows pool of transaction pool.
+// Changing the pool size to 1, we verify that the pool size is updated and the pool is full when we try to acquire next transaction.
+func TestTxPoolSize(t *testing.T) {
+ vstart := framework.DebugVars()
+
+ verifyIntValue(t, vstart, "TransactionPoolCapacity", 20)
+ verifyIntValue(t, vstart, "FoundRowsPoolCapacity", 20)
+
+ client1 := framework.NewClient()
+ err := client1.Begin( /* found rows pool*/ false)
+ require.NoError(t, err)
+ defer client1.Rollback()
+ verifyIntValue(t, framework.DebugVars(), "TransactionPoolAvailable", framework.FetchInt(vstart, "TransactionPoolAvailable")-1)
+
+ client2 := framework.NewClient()
+ err = client2.Begin( /* found rows pool*/ true)
+ require.NoError(t, err)
+ defer client2.Rollback()
+ verifyIntValue(t, framework.DebugVars(), "FoundRowsPoolAvailable", framework.FetchInt(vstart, "FoundRowsPoolAvailable")-1)
+
+ revert := changeVar(t, "TransactionPoolSize", "1")
+ defer revert()
+ vend := framework.DebugVars()
+ verifyIntValue(t, vend, "TransactionPoolAvailable", 0)
+ verifyIntValue(t, vend, "TransactionPoolCapacity", 1)
+ verifyIntValue(t, vend, "FoundRowsPoolAvailable", 0)
+ verifyIntValue(t, vend, "FoundRowsPoolCapacity", 1)
+ assert.Equal(t, 1, framework.Server.TxPoolSize())
+
+ client3 := framework.NewClient()
+
+ // tx pool - normal
+ err = client3.Begin( /* found rows pool*/ false)
+ require.ErrorContains(t, err, "connection limit exceeded")
+ compareIntDiff(t, framework.DebugVars(), "Errors/RESOURCE_EXHAUSTED", vstart, 1)
+
+ // tx pool - found rows
+ err = client3.Begin( /* found rows pool*/ true)
+ require.ErrorContains(t, err, "connection limit exceeded")
+ compareIntDiff(t, framework.DebugVars(), "Errors/RESOURCE_EXHAUSTED", vstart, 2)
+}
+
func TestDisableConsolidator(t *testing.T) {
totalConsolidationsTag := "Waits/Histograms/Consolidations/Count"
initial := framework.FetchInt(framework.DebugVars(), totalConsolidationsTag)
diff --git a/go/vt/vttablet/endtoend/connecttcp/main_test.go b/go/vt/vttablet/endtoend/connecttcp/main_test.go
new file mode 100644
index 00000000000..43be05893cc
--- /dev/null
+++ b/go/vt/vttablet/endtoend/connecttcp/main_test.go
@@ -0,0 +1,119 @@
+/*
+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 connecttcp
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "os"
+ "testing"
+ "time"
+
+ "vitess.io/vitess/go/mysql"
+ vttestpb "vitess.io/vitess/go/vt/proto/vttest"
+ "vitess.io/vitess/go/vt/vttablet/endtoend/framework"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
+ "vitess.io/vitess/go/vt/vttest"
+)
+
+var (
+ connParams mysql.ConnParams
+ connAppDebugParams mysql.ConnParams
+)
+
+func TestMain(m *testing.M) {
+ flag.Parse() // Do not remove this comment, import into google3 depends on it
+ tabletenv.Init()
+
+ exitCode := func() int {
+ // Launch MySQL.
+ // We need a Keyspace in the topology, so the DbName is set.
+ // We need a Shard too, so the database 'vttest' is created.
+ cfg := vttest.Config{
+ Topology: &vttestpb.VTTestTopology{
+ Keyspaces: []*vttestpb.Keyspace{
+ {
+ Name: "vttest",
+ Shards: []*vttestpb.Shard{
+ {
+ Name: "0",
+ DbNameOverride: "vttest",
+ },
+ },
+ },
+ },
+ },
+ OnlyMySQL: true,
+ Charset: "utf8mb4_general_ci",
+ }
+ if err := cfg.InitSchemas("vttest", testSchema, nil); err != nil {
+ fmt.Fprintf(os.Stderr, "InitSchemas failed: %v\n", err)
+ return 1
+ }
+ defer os.RemoveAll(cfg.SchemaDir)
+ cluster := vttest.LocalCluster{
+ Config: cfg,
+ }
+ if err := cluster.Setup(); err != nil {
+ fmt.Fprintf(os.Stderr, "could not launch mysql: %v\n", err)
+ return 1
+ }
+ defer cluster.TearDown()
+
+ if err := allowConnectOnTCP(cluster); err != nil {
+ fmt.Fprintf(os.Stderr, "failed to allow tcp priviliges: %v", err)
+ return 1
+ }
+
+ connParams = cluster.MySQLTCPConnParams()
+ connAppDebugParams = cluster.MySQLAppDebugConnParams()
+
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ config := tabletenv.NewDefaultConfig()
+ config.TwoPCAbandonAge = 1 * time.Second
+
+ if err := framework.StartCustomServer(ctx, connParams, connAppDebugParams, cluster.DbName(), config); err != nil {
+ fmt.Fprintf(os.Stderr, "%v", err)
+ return 1
+ }
+ defer framework.StopServer()
+
+ return m.Run()
+ }()
+ os.Exit(exitCode)
+}
+
+func allowConnectOnTCP(cluster vttest.LocalCluster) error {
+ connParams = cluster.MySQLConnParams()
+ conn, err := mysql.Connect(context.Background(), &connParams)
+ if err != nil {
+ return err
+ }
+ if _, err = conn.ExecuteFetch("UPDATE mysql.user SET Host = '%' WHERE User = 'vt_dba';", 0, false); err != nil {
+ return err
+ }
+ if _, err = conn.ExecuteFetch("FLUSH PRIVILEGES;", 0, false); err != nil {
+ return err
+ }
+ conn.Close()
+ return nil
+}
+
+var testSchema = `create table vitess_test(intval int primary key);`
diff --git a/go/vt/vttablet/tabletserver/tabletenv/seconds_test.go b/go/vt/vttablet/endtoend/connecttcp/prepare_test.go
similarity index 50%
rename from go/vt/vttablet/tabletserver/tabletenv/seconds_test.go
rename to go/vt/vttablet/endtoend/connecttcp/prepare_test.go
index dc09a3f419f..524b4a3dcc8 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/seconds_test.go
+++ b/go/vt/vttablet/endtoend/connecttcp/prepare_test.go
@@ -1,5 +1,5 @@
/*
-Copyright 2020 The Vitess Authors.
+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.
@@ -14,40 +14,28 @@ See the License for the specific language governing permissions and
limitations under the License.
*/
-package tabletenv
+package connecttcp
import (
"testing"
- "time"
- "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
- "vitess.io/vitess/go/yaml2"
+ "vitess.io/vitess/go/vt/vttablet/endtoend/framework"
)
-func TestSecondsYaml(t *testing.T) {
- type testSecond struct {
- Value Seconds `json:"value"`
- }
+// TestPrepareOnTCP tests that a prepare statement is not allowed on a network connection.
+func TestPrepareOnTCP(t *testing.T) {
+ client := framework.NewClient()
- ts := testSecond{
- Value: 1,
- }
- gotBytes, err := yaml2.Marshal(&ts)
+ query := "insert into vitess_test (intval) values(4)"
+
+ err := client.Begin(false)
require.NoError(t, err)
- wantBytes := "value: 1\n"
- assert.Equal(t, wantBytes, string(gotBytes))
- var gotts testSecond
- err = yaml2.Unmarshal([]byte(wantBytes), &gotts)
+ _, err = client.Execute(query, nil)
require.NoError(t, err)
- assert.Equal(t, ts, gotts)
-}
-func TestSecondsGetSet(t *testing.T) {
- var val Seconds
- val.Set(2 * time.Second)
- assert.Equal(t, Seconds(2), val)
- assert.Equal(t, 2*time.Second, val.Get())
+ err = client.Prepare("aa")
+ require.ErrorContains(t, err, "VT10002: atomic distributed transaction not allowed: cannot prepare the transaction on a network connection")
}
diff --git a/go/vt/vttablet/endtoend/framework/client.go b/go/vt/vttablet/endtoend/framework/client.go
index 1cbff71dc25..59def25ab61 100644
--- a/go/vt/vttablet/endtoend/framework/client.go
+++ b/go/vt/vttablet/endtoend/framework/client.go
@@ -158,7 +158,7 @@ func (client *QueryClient) CreateTransaction(dtid string, participants []*queryp
}
// StartCommit issues a StartCommit to TabletServer for the current transaction.
-func (client *QueryClient) StartCommit(dtid string) error {
+func (client *QueryClient) StartCommit(dtid string) (querypb.StartCommitState, error) {
defer func() { client.transactionID = 0 }()
return client.server.StartCommit(client.ctx, client.target, client.transactionID, dtid)
}
@@ -180,15 +180,13 @@ func (client *QueryClient) ReadTransaction(dtid string) (*querypb.TransactionMet
// UnresolvedTransactions invokes the UnresolvedTransactions API of TabletServer.
func (client *QueryClient) UnresolvedTransactions() ([]*querypb.TransactionMetadata, error) {
- return client.server.UnresolvedTransactions(client.ctx, client.target)
+ return client.server.UnresolvedTransactions(client.ctx, client.target, 0 /* abandonAgeSeconds */)
}
// SetServingType is for testing transitions.
// It currently supports only primary->replica and back.
func (client *QueryClient) SetServingType(tabletType topodatapb.TabletType) error {
err := client.server.SetServingType(tabletType, time.Time{}, true /* serving */, "" /* reason */)
- // Wait for TwoPC transition, if necessary
- client.server.TwoPCEngineWait()
return err
}
diff --git a/go/vt/vttablet/endtoend/framework/server.go b/go/vt/vttablet/endtoend/framework/server.go
index 2cd3ccc354c..3374aadb450 100644
--- a/go/vt/vttablet/endtoend/framework/server.go
+++ b/go/vt/vttablet/endtoend/framework/server.go
@@ -65,11 +65,16 @@ func StartCustomServer(ctx context.Context, connParams, connAppDebugParams mysql
TabletType: topodatapb.TabletType_PRIMARY,
}
TopoServer = memorytopo.NewServer(ctx, "")
+ // Create the serving keyspace for throttler.
+ err := TopoServer.UpdateSrvKeyspace(ctx, "", "vttest", &topodatapb.SrvKeyspace{})
+ if err != nil {
+ return vterrors.Wrap(err, "could not create serving keyspace in topo")
+ }
srvTopoCounts := stats.NewCountersWithSingleLabel("", "Resilient srvtopo server operations", "type")
Server = tabletserver.NewTabletServer(ctx, vtenv.NewTestEnv(), "", cfg, TopoServer, &topodatapb.TabletAlias{}, srvTopoCounts)
Server.Register()
- err := Server.StartService(Target, dbcfgs, nil /* mysqld */)
+ err = Server.StartService(Target, dbcfgs, nil /* mysqld */)
if err != nil {
return vterrors.Wrap(err, "could not start service")
}
@@ -103,8 +108,7 @@ func StartCustomServer(ctx context.Context, connParams, connAppDebugParams mysql
func StartServer(ctx context.Context, connParams, connAppDebugParams mysql.ConnParams, dbName string) error {
config := tabletenv.NewDefaultConfig()
config.StrictTableACL = true
- config.TwoPCEnable = true
- config.TwoPCAbandonAge = 1
+ config.TwoPCAbandonAge = 1 * time.Second
config.HotRowProtection.Mode = tabletenv.Enable
config.TrackSchemaVersions = true
config.GracePeriods.Shutdown = 2 * time.Second
diff --git a/go/vt/vttablet/endtoend/main_test.go b/go/vt/vttablet/endtoend/main_test.go
index 1284f790b93..eedd893f3eb 100644
--- a/go/vt/vttablet/endtoend/main_test.go
+++ b/go/vt/vttablet/endtoend/main_test.go
@@ -343,7 +343,7 @@ var tableACLConfig = `{
},
{
"name": "vitess_twopc",
- "table_names_or_prefixes": ["dt_state"],
+ "table_names_or_prefixes": ["dt_state", "redo_state"],
"readers": ["dev"],
"writers": ["dev"],
"admins": ["dev"]
diff --git a/go/vt/vttablet/endtoend/misc_test.go b/go/vt/vttablet/endtoend/misc_test.go
index 29bbba56873..68f6f4b1af6 100644
--- a/go/vt/vttablet/endtoend/misc_test.go
+++ b/go/vt/vttablet/endtoend/misc_test.go
@@ -35,10 +35,15 @@ import (
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/callerid"
+ "vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/endtoend/framework"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
func TestSimpleRead(t *testing.T) {
@@ -890,6 +895,11 @@ func TestShowTablesWithSizes(t *testing.T) {
require.NoError(t, err)
defer conn.Close()
+ if query := conn.BaseShowTablesWithSizes(); query == "" {
+ // Happens in MySQL 8.0 where we use BaseShowInnodbTableSizes, instead.
+ t.Skip("BaseShowTablesWithSizes is empty in this version of MySQL")
+ }
+
setupQueries := []string{
`drop view if exists show_tables_with_sizes_v1`,
`drop table if exists show_tables_with_sizes_t1`,
@@ -897,12 +907,14 @@ func TestShowTablesWithSizes(t *testing.T) {
`create table show_tables_with_sizes_t1 (id int primary key)`,
`create view show_tables_with_sizes_v1 as select * from show_tables_with_sizes_t1`,
`CREATE TABLE show_tables_with_sizes_employees (id INT NOT NULL, store_id INT) PARTITION BY HASH(store_id) PARTITIONS 4`,
+ `create table show_tables_with_sizes_fts (id int primary key, name text, fulltext key name_fts (name))`,
}
defer func() {
_, _ = conn.ExecuteFetch(`drop view if exists show_tables_with_sizes_v1`, 1, false)
_, _ = conn.ExecuteFetch(`drop table if exists show_tables_with_sizes_t1`, 1, false)
_, _ = conn.ExecuteFetch(`drop table if exists show_tables_with_sizes_employees`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop table if exists show_tables_with_sizes_fts`, 1, false)
}()
for _, query := range setupQueries {
_, err := conn.ExecuteFetch(query, 1, false)
@@ -913,6 +925,7 @@ func TestShowTablesWithSizes(t *testing.T) {
"show_tables_with_sizes_t1",
"show_tables_with_sizes_v1",
"show_tables_with_sizes_employees",
+ "show_tables_with_sizes_fts",
}
actualTables := []string{}
@@ -933,7 +946,7 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.True(t, row[2].IsIntegral())
createTime, err := row[2].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, createTime, int64(0))
+ assert.Positive(t, createTime)
// TABLE_COMMENT
assert.Equal(t, "", row[3].ToString())
@@ -941,12 +954,12 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.True(t, row[4].IsDecimal())
fileSize, err := row[4].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, fileSize, int64(0))
+ assert.Positive(t, fileSize)
assert.True(t, row[4].IsDecimal())
allocatedSize, err := row[5].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, allocatedSize, int64(0))
+ assert.Positive(t, allocatedSize)
actualTables = append(actualTables, tableName)
} else if tableName == "show_tables_with_sizes_v1" {
@@ -956,7 +969,7 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.True(t, row[2].IsIntegral())
createTime, err := row[2].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, createTime, int64(0))
+ assert.Positive(t, createTime)
// TABLE_COMMENT
assert.Equal(t, "VIEW", row[3].ToString())
@@ -972,7 +985,30 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.True(t, row[2].IsIntegral())
createTime, err := row[2].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, createTime, int64(0))
+ assert.Positive(t, createTime)
+
+ // TABLE_COMMENT
+ assert.Equal(t, "", row[3].ToString())
+
+ assert.True(t, row[4].IsDecimal())
+ fileSize, err := row[4].ToCastInt64()
+ assert.NoError(t, err)
+ assert.Positive(t, fileSize)
+
+ assert.True(t, row[5].IsDecimal())
+ allocatedSize, err := row[5].ToCastInt64()
+ assert.NoError(t, err)
+ assert.Positive(t, allocatedSize)
+
+ actualTables = append(actualTables, tableName)
+ } else if tableName == "show_tables_with_sizes_fts" {
+ // TABLE_TYPE
+ assert.Equal(t, "BASE TABLE", row[1].ToString())
+
+ assert.True(t, row[2].IsIntegral())
+ createTime, err := row[2].ToCastInt64()
+ assert.NoError(t, err)
+ assert.Positive(t, createTime)
// TABLE_COMMENT
assert.Equal(t, "", row[3].ToString())
@@ -980,12 +1016,12 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.True(t, row[4].IsDecimal())
fileSize, err := row[4].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, fileSize, int64(0))
+ assert.Positive(t, fileSize)
assert.True(t, row[5].IsDecimal())
allocatedSize, err := row[5].ToCastInt64()
assert.NoError(t, err)
- assert.Greater(t, allocatedSize, int64(0))
+ assert.Positive(t, allocatedSize)
actualTables = append(actualTables, tableName)
}
@@ -995,6 +1031,137 @@ func TestShowTablesWithSizes(t *testing.T) {
assert.ElementsMatch(t, expectedTables, actualTables)
}
+func newTestSchemaEngine(connParams *mysql.ConnParams) *schema.Engine {
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = dbconfigs.NewTestDBConfigs(*connParams, *connParams, connParams.DbName)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "EngineTest")
+ se := schema.NewEngine(env)
+ se.InitDBConfig(dbconfigs.New(connParams))
+ return se
+}
+
+func TestEngineReload(t *testing.T) {
+ ctx := context.Background()
+ conn, err := mysql.Connect(ctx, &connParams)
+ require.NoError(t, err)
+ defer conn.Close()
+ t.Run("validate innodb size query", func(t *testing.T) {
+ q := conn.BaseShowInnodbTableSizes()
+ require.NotEmpty(t, q)
+ })
+ t.Run("validate conn schema", func(t *testing.T) {
+ rs, err := conn.ExecuteFetch(`select database() as d`, 1, true)
+ require.NoError(t, err)
+ row := rs.Named().Row()
+ require.NotNil(t, row)
+ database := row.AsString("d", "")
+ require.Equal(t, connParams.DbName, database)
+ })
+
+ defer func() {
+ _, _ = conn.ExecuteFetch(`drop view if exists view_simple`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop view if exists view_simple2`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop view if exists view_simple3`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop table if exists tbl_simple`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop table if exists tbl_part`, 1, false)
+ _, _ = conn.ExecuteFetch(`drop table if exists tbl_fts`, 1, false)
+ }()
+
+ engine := newTestSchemaEngine(&connParams)
+ require.NotNil(t, engine)
+ err = engine.Open()
+ require.NoError(t, err)
+ defer engine.Close()
+
+ t.Run("schema", func(t *testing.T) {
+ setupQueries := []string{
+ `drop view if exists view_simple`,
+ `drop view if exists view_simple2`,
+ `drop table if exists tbl_simple`,
+ `drop table if exists tbl_part`,
+ `drop table if exists tbl_fts`,
+ `create table tbl_simple (id int primary key)`,
+ `create view view_simple as select * from tbl_simple`,
+ `create view view_simple2 as select * from tbl_simple`,
+ `create table tbl_part (id INT NOT NULL, store_id INT) PARTITION BY HASH(store_id) PARTITIONS 4`,
+ `create table tbl_fts (id int primary key, name text, fulltext key name_fts (name))`,
+ }
+
+ for _, query := range setupQueries {
+ _, err := conn.ExecuteFetch(query, 1, false)
+ require.NoError(t, err)
+ }
+
+ expectedTables := []string{
+ "tbl_simple",
+ "tbl_part",
+ "tbl_fts",
+ "view_simple",
+ "view_simple2",
+ }
+ err := engine.Reload(ctx)
+ require.NoError(t, err)
+
+ schema := engine.GetSchema()
+ require.NotEmpty(t, schema)
+ for _, expectTable := range expectedTables {
+ t.Run(expectTable, func(t *testing.T) {
+ tbl := engine.GetTable(sqlparser.NewIdentifierCS(expectTable))
+ require.NotNil(t, tbl)
+
+ switch expectTable {
+ case "view_simple", "view_simple2":
+ assert.Zero(t, tbl.FileSize)
+ assert.Zero(t, tbl.AllocatedSize)
+ default:
+ assert.NotZero(t, tbl.FileSize)
+ assert.NotZero(t, tbl.AllocatedSize)
+ }
+ })
+ }
+ })
+ t.Run("schema changes", func(t *testing.T) {
+ setupQueries := []string{
+ `alter view view_simple as select *, 2 from tbl_simple`,
+ `drop view view_simple2`,
+ `create view view_simple3 as select * from tbl_simple`,
+ }
+
+ for _, query := range setupQueries {
+ _, err := conn.ExecuteFetch(query, 1, false)
+ require.NoError(t, err)
+ }
+
+ expectedTables := []string{
+ "tbl_simple",
+ "tbl_part",
+ "tbl_fts",
+ "view_simple",
+ "view_simple3",
+ }
+ err := engine.Reload(ctx)
+ require.NoError(t, err)
+
+ schema := engine.GetSchema()
+ require.NotEmpty(t, schema)
+ for _, expectTable := range expectedTables {
+ t.Run(expectTable, func(t *testing.T) {
+ tbl := engine.GetTable(sqlparser.NewIdentifierCS(expectTable))
+ require.NotNil(t, tbl)
+
+ switch expectTable {
+ case "view_simple", "view_simple2", "view_simple3":
+ assert.Zero(t, tbl.FileSize)
+ assert.Zero(t, tbl.AllocatedSize)
+ default:
+ assert.NotZero(t, tbl.FileSize)
+ assert.NotZero(t, tbl.AllocatedSize)
+ }
+ })
+ }
+ })
+}
+
// TestTuple tests that bind variables having tuple values work with vttablet.
func TestTuple(t *testing.T) {
client := framework.NewClient()
diff --git a/go/vt/vttablet/endtoend/queries_test.go b/go/vt/vttablet/endtoend/queries_test.go
index 5a57f681a10..f67d3219010 100644
--- a/go/vt/vttablet/endtoend/queries_test.go
+++ b/go/vt/vttablet/endtoend/queries_test.go
@@ -121,6 +121,20 @@ var TestQueryCases = []framework.Testable{
},
RowsReturned: 1,
},
+ &framework.TestCase{
+ Name: "explain with bindvars",
+ Query: "explain select :__vtudvp as `@p` from dual",
+ BindVars: map[string]*querypb.BindVariable{
+ "__vtudvp": sqltypes.Int64BindVariable(1),
+ },
+ Result: [][]string{
+ {"1", "SIMPLE", "", "", "", "", "", "", "", "", "", "No tables used"},
+ },
+ Rewritten: []string{
+ "explain select 1 as `@p` from dual",
+ },
+ RowsReturned: 1,
+ },
&framework.TestCase{
Name: "limit",
Query: "select /* limit */ eid, id from vitess_a limit :a",
diff --git a/go/vt/vttablet/endtoend/savepoint_test.go b/go/vt/vttablet/endtoend/savepoint_test.go
index 74572f2376f..90cf8fbd547 100644
--- a/go/vt/vttablet/endtoend/savepoint_test.go
+++ b/go/vt/vttablet/endtoend/savepoint_test.go
@@ -103,7 +103,7 @@ func TestSavepointInTransactionWithRelease(t *testing.T) {
diff int
}{{
tag: "Queries/Histograms/Savepoint/Count",
- diff: 1,
+ diff: 2, // savepoint a (post-begin) and savepoint b
}, {
tag: "Queries/Histograms/Release/Count",
diff: 1,
diff --git a/go/vt/vttablet/endtoend/settings_test.go b/go/vt/vttablet/endtoend/settings_test.go
index a459ad15844..0ccaf958737 100644
--- a/go/vt/vttablet/endtoend/settings_test.go
+++ b/go/vt/vttablet/endtoend/settings_test.go
@@ -138,16 +138,9 @@ func TestDDLNoConnectionReservationOnSettings(t *testing.T) {
query := "create table temp(c_date datetime default '0000-00-00')"
setting := "set sql_mode='TRADITIONAL'"
- for _, withTx := range []bool{false, true} {
- if withTx {
- err := client.Begin(false)
- require.NoError(t, err)
- }
- _, err := client.ReserveExecute(query, []string{setting}, nil)
- require.Error(t, err, "create table should have failed with TRADITIONAL mode")
- require.Contains(t, err.Error(), "Invalid default value")
- assert.Zero(t, client.ReservedID())
- }
+ _, err := client.ReserveExecute(query, []string{setting}, nil)
+ assert.ErrorContains(t, err, "Invalid default value for 'c_date'", "create table should have failed with TRADITIONAL mode")
+ assert.Zero(t, client.ReservedID())
}
func TestDMLNoConnectionReservationOnSettings(t *testing.T) {
@@ -156,7 +149,10 @@ func TestDMLNoConnectionReservationOnSettings(t *testing.T) {
_, err := client.Execute("create table temp(c_date datetime)", nil)
require.NoError(t, err)
- defer client.Execute("drop table temp", nil)
+ defer func() {
+ client.Rollback()
+ client.Execute("drop table temp", nil)
+ }()
_, err = client.Execute("insert into temp values ('2022-08-25')", nil)
require.NoError(t, err)
@@ -211,9 +207,8 @@ func TestDDLNoConnectionReservationOnSettingsWithTx(t *testing.T) {
query := "create table temp(c_date datetime default '0000-00-00')"
setting := "set sql_mode='TRADITIONAL'"
- _, err := client.ReserveBeginExecute(query, []string{setting}, nil, nil)
- require.Error(t, err, "create table should have failed with TRADITIONAL mode")
- require.Contains(t, err.Error(), "Invalid default value")
+ _, err := client.ReserveExecute(query, []string{setting}, nil)
+ require.ErrorContains(t, err, "Invalid default value for 'c_date'", "create table should have failed with TRADITIONAL mode")
assert.Zero(t, client.ReservedID())
}
@@ -297,12 +292,6 @@ func TestTempTableOnReserveExecute(t *testing.T) {
require.NoError(t,
client.Release())
- _, err = client.ReserveBeginExecute(tempTblQuery, nil, nil, nil)
- require.NoError(t, err)
- assert.NotZero(t, client.ReservedID())
- require.NoError(t,
- client.Release())
-
// drop the table
_, err = client.Execute("drop table if exists temp", nil)
require.NoError(t, err)
@@ -318,13 +307,6 @@ func TestTempTableOnReserveExecute(t *testing.T) {
assert.NotZero(t, client.ReservedID(), "as this goes through fallback path of reserving a connection due to temporary tables")
require.NoError(t,
client.Release())
-
- _, err = client.ReserveBeginExecute(tempTblQuery, []string{setting}, nil, nil)
- require.Error(t, err, "create table should have failed with TRADITIONAL mode")
- require.Contains(t, err.Error(), "Invalid default value")
- assert.NotZero(t, client.ReservedID(), "as this goes through fallback path of reserving a connection due to temporary tables")
- require.NoError(t,
- client.Release())
}
func TestInfiniteSessions(t *testing.T) {
diff --git a/go/vt/vttablet/endtoend/transaction_test.go b/go/vt/vttablet/endtoend/transaction_test.go
index ad6ff558c40..5e5889ad671 100644
--- a/go/vt/vttablet/endtoend/transaction_test.go
+++ b/go/vt/vttablet/endtoend/transaction_test.go
@@ -22,12 +22,9 @@ import (
"testing"
"time"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv/tabletenvtest"
-
- "google.golang.org/protobuf/proto"
-
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "google.golang.org/protobuf/proto"
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/test/utils"
@@ -35,7 +32,6 @@ import (
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/vttablet/endtoend/framework"
"vitess.io/vitess/go/vt/vttablet/tabletserver"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
func TestCommit(t *testing.T) {
@@ -202,30 +198,6 @@ func TestAutoCommit(t *testing.T) {
}
}
-func TestTxPoolSize(t *testing.T) {
- tabletenvtest.LoadTabletEnvFlags()
-
- vstart := framework.DebugVars()
-
- client1 := framework.NewClient()
- err := client1.Begin(false)
- require.NoError(t, err)
- defer client1.Rollback()
- verifyIntValue(t, framework.DebugVars(), "TransactionPoolAvailable", tabletenv.NewCurrentConfig().TxPool.Size-1)
-
- revert := changeVar(t, "TxPoolSize", "1")
- defer revert()
- vend := framework.DebugVars()
- verifyIntValue(t, vend, "TransactionPoolAvailable", 0)
- verifyIntValue(t, vend, "TransactionPoolCapacity", 1)
-
- client2 := framework.NewClient()
- err = client2.Begin(false)
- require.Error(t, err)
- require.Contains(t, err.Error(), "connection limit exceeded")
- compareIntDiff(t, framework.DebugVars(), "Errors/RESOURCE_EXHAUSTED", vstart, 1)
-}
-
func TestForUpdate(t *testing.T) {
for _, mode := range []string{"for update", "lock in share mode"} {
client := framework.NewClient()
@@ -256,7 +228,7 @@ func TestPrepareRollback(t *testing.T) {
err = client.Prepare("aa")
if err != nil {
client.RollbackPrepared("aa", 0)
- t.Fatalf(err.Error())
+ t.Fatal(err.Error())
}
err = client.RollbackPrepared("aa", 0)
require.NoError(t, err)
@@ -529,8 +501,9 @@ func TestMMCommitFlow(t *testing.T) {
require.Error(t, err)
require.Contains(t, err.Error(), "Duplicate entry")
- err = client.StartCommit("aa")
+ state, err := client.StartCommit("aa")
require.NoError(t, err)
+ assert.Equal(t, querypb.StartCommitState_Success, state)
err = client.SetRollback("aa", 0)
require.EqualError(t, err, "could not transition to ROLLBACK: aa (CallerID: dev)")
@@ -794,7 +767,14 @@ func TestUnresolvedTransactions(t *testing.T) {
State: querypb.TransactionState_PREPARE,
Participants: participants,
}}
- utils.MustMatch(t, want, transactions)
+
+ require.Len(t, want, len(transactions))
+ for i, transaction := range transactions {
+ // Skipping check for TimeCreated
+ assert.Equal(t, want[i].Dtid, transaction.Dtid)
+ assert.Equal(t, want[i].State, transaction.State)
+ assert.Equal(t, want[i].Participants, transaction.Participants)
+ }
}
// TestUnresolvedTransactions tests the UnresolvedTransactions API.
@@ -857,5 +837,12 @@ func TestUnresolvedTransactionsOrdering(t *testing.T) {
State: querypb.TransactionState_PREPARE,
Participants: participants1,
}}
- utils.MustMatch(t, want, transactions)
+
+ require.Len(t, want, len(transactions))
+ for i, transaction := range transactions {
+ // Skipping check for TimeCreated
+ assert.Equal(t, want[i].Dtid, transaction.Dtid)
+ assert.Equal(t, want[i].State, transaction.State)
+ assert.Equal(t, want[i].Participants, transaction.Participants)
+ }
}
diff --git a/go/vt/vttablet/endtoend/twopc/main_test.go b/go/vt/vttablet/endtoend/twopc/main_test.go
new file mode 100644
index 00000000000..3b68ce273e1
--- /dev/null
+++ b/go/vt/vttablet/endtoend/twopc/main_test.go
@@ -0,0 +1,100 @@
+/*
+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 endtoend
+
+import (
+ "context"
+ "flag"
+ "fmt"
+ "os"
+ "testing"
+ "time"
+
+ "vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/vt/vttablet/endtoend/framework"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
+ "vitess.io/vitess/go/vt/vttest"
+
+ vttestpb "vitess.io/vitess/go/vt/proto/vttest"
+)
+
+var (
+ connParams mysql.ConnParams
+ connAppDebugParams mysql.ConnParams
+ cluster vttest.LocalCluster
+)
+
+func TestMain(m *testing.M) {
+ flag.Parse() // Do not remove this comment, import into google3 depends on it
+ tabletenv.Init()
+
+ exitCode := func() int {
+ // Launch MySQL.
+ // We need a Keyspace in the topology, so the DbName is set.
+ // We need a Shard too, so the database 'vttest' is created.
+ cfg := vttest.Config{
+ Topology: &vttestpb.VTTestTopology{
+ Keyspaces: []*vttestpb.Keyspace{
+ {
+ Name: "vttest",
+ Shards: []*vttestpb.Shard{
+ {
+ Name: "0",
+ DbNameOverride: "vttest",
+ },
+ },
+ },
+ },
+ },
+ OnlyMySQL: true,
+ Charset: "utf8mb4_general_ci",
+ }
+ if err := cfg.InitSchemas("vttest", testSchema, nil); err != nil {
+ fmt.Fprintf(os.Stderr, "InitSchemas failed: %v\n", err)
+ return 1
+ }
+ defer os.RemoveAll(cfg.SchemaDir)
+ cluster = vttest.LocalCluster{
+ Config: cfg,
+ }
+ if err := cluster.Setup(); err != nil {
+ fmt.Fprintf(os.Stderr, "could not launch mysql: %v\n", err)
+ return 1
+ }
+
+ defer cluster.TearDown()
+
+ connParams = cluster.MySQLConnParams()
+ connAppDebugParams = cluster.MySQLAppDebugConnParams()
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ config := tabletenv.NewDefaultConfig()
+ config.TwoPCAbandonAge = 1 * time.Second
+ err := framework.StartCustomServer(ctx, connParams, connAppDebugParams, cluster.DbName(), config)
+ if err != nil {
+ fmt.Fprintf(os.Stderr, "%v", err)
+ return 1
+ }
+ defer framework.StopServer()
+
+ return m.Run()
+ }()
+ os.Exit(exitCode)
+}
+
+var testSchema = `create table vitess_test(intval int default 0 primary key);`
diff --git a/go/vt/vttablet/endtoend/twopc/prepare_test.go b/go/vt/vttablet/endtoend/twopc/prepare_test.go
new file mode 100644
index 00000000000..047b02bce82
--- /dev/null
+++ b/go/vt/vttablet/endtoend/twopc/prepare_test.go
@@ -0,0 +1,109 @@
+/*
+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 endtoend
+
+import (
+ "context"
+ "fmt"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/vt/vttablet/endtoend/framework"
+)
+
+// TestCommitPreparedFailNonRetryable tests the case where the commit_prepared fails trying to acquire update lock.
+// The transaction updates to failed state.
+func TestCommitPreparedFailNonRetryable(t *testing.T) {
+ dbaConnector := framework.Server.Config().DB.DbaWithDB()
+ conn, err := dbaConnector.Connect(context.Background())
+ require.NoError(t, err)
+ defer conn.Close()
+
+ _, err = conn.ExecuteFetch("set global innodb_lock_wait_timeout = 1", 1, false)
+ require.NoError(t, err)
+ defer conn.ExecuteFetch("set global innodb_lock_wait_timeout = default", 1, false)
+
+ client := framework.NewClient()
+ defer client.RollbackPrepared("bb", client.TransactionID())
+
+ _, err = client.BeginExecute(`insert into vitess_test (intval) values(50)`, nil, nil)
+ require.NoError(t, err)
+ err = client.Prepare("bb")
+ require.NoError(t, err)
+
+ client2 := framework.NewClient()
+ _, err = client2.BeginExecute(`select * from _vt.redo_state where dtid = 'bb' for update`, nil, nil)
+ require.NoError(t, err)
+
+ ch := make(chan any)
+ go func() {
+ err := client.CommitPrepared("bb")
+ ch <- nil
+ require.ErrorContains(t, err, "commit_prepared")
+ }()
+ time.Sleep(1500 * time.Millisecond)
+
+ client2.Release()
+ <-ch
+
+ qr, err := client2.Execute("select dtid, state, message from _vt.redo_state where dtid = 'bb'", nil)
+ require.NoError(t, err)
+ require.Equal(t, `[[VARBINARY("bb") INT64(0) TEXT("Lock wait timeout exceeded; try restarting transaction (errno 1205) (sqlstate HY000) during query: delete from _vt.redo_state where dtid = _binary'bb'")]]`, fmt.Sprintf("%v", qr.Rows))
+}
+
+// TestCommitPreparedFailRetryable tests the case where the commit_prepared fails when the query is killed.
+// The transaction remains in the prepare state.
+func TestCommitPreparedFailRetryable(t *testing.T) {
+ client := framework.NewClient()
+ defer client.RollbackPrepared("aa", client.TransactionID())
+
+ _, err := client.BeginExecute(`insert into vitess_test (intval) values(40)`, nil, nil)
+ require.NoError(t, err)
+ connRes, err := client.Execute(`select connection_id()`, nil)
+ require.NoError(t, err)
+ err = client.Prepare("aa")
+ require.NoError(t, err)
+
+ client2 := framework.NewClient()
+ _, err = client2.BeginExecute(`select * from _vt.redo_state where dtid = _binary'aa' for update`, nil, nil)
+ require.NoError(t, err)
+
+ ch := make(chan any)
+ go func() {
+ err := client.CommitPrepared("aa")
+ ch <- nil
+ require.ErrorContains(t, err, "commit_prepared")
+ }()
+ time.Sleep(100 * time.Millisecond)
+
+ dbaConnector := framework.Server.Config().DB.DbaWithDB()
+ conn, err := dbaConnector.Connect(context.Background())
+ require.NoError(t, err)
+ defer conn.Close()
+
+ _, err = conn.ExecuteFetch(fmt.Sprintf("kill query %s", connRes.Rows[0][0].ToString()), 1, false)
+ require.NoError(t, err)
+
+ client2.Release()
+ <-ch
+
+ qr, err := client2.Execute("select dtid, state, message from _vt.redo_state where dtid = _binary'aa'", nil)
+ require.NoError(t, err)
+ require.Equal(t, `[[VARBINARY("aa") INT64(1) TEXT("Query execution was interrupted (errno 1317) (sqlstate 70100) during query: delete from _vt.redo_state where dtid = _binary'aa'")]]`, fmt.Sprintf("%v", qr.Rows))
+}
diff --git a/go/vt/vttablet/faketmclient/fake_client.go b/go/vt/vttablet/faketmclient/fake_client.go
index 3aeeff71e85..78c87d142a9 100644
--- a/go/vt/vttablet/faketmclient/fake_client.go
+++ b/go/vt/vttablet/faketmclient/fake_client.go
@@ -66,6 +66,10 @@ func (client *FakeTabletManagerClient) CreateVReplicationWorkflow(ctx context.Co
return nil, nil
}
+func (client *FakeTabletManagerClient) DeleteTableData(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error) {
+ return nil, nil
+}
+
func (client *FakeTabletManagerClient) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) {
return nil, nil
}
@@ -94,6 +98,10 @@ func (client *FakeTabletManagerClient) UpdateVReplicationWorkflows(ctx context.C
return nil, nil
}
+func (client *FakeTabletManagerClient) ValidateVReplicationPermissions(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error) {
+ return nil, nil
+}
+
func (client *FakeTabletManagerClient) VDiff(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) {
return nil, nil
}
@@ -157,6 +165,11 @@ func (client *FakeTabletManagerClient) SetReadWrite(ctx context.Context, tablet
return nil
}
+// ChangeTags is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) ChangeTags(ctx context.Context, tablet *topodatapb.Tablet, tabletTags map[string]string, replace bool) (*tabletmanagerdatapb.ChangeTagsResponse, error) {
+ return &tabletmanagerdatapb.ChangeTagsResponse{}, nil
+}
+
// ChangeType is part of the tmclient.TabletManagerClient interface.
func (client *FakeTabletManagerClient) ChangeType(ctx context.Context, tablet *topodatapb.Tablet, dbType topodatapb.TabletType, semiSync bool) error {
return nil
@@ -212,6 +225,30 @@ func (client *FakeTabletManagerClient) ExecuteFetchAsApp(ctx context.Context, ta
return &querypb.QueryResult{}, nil
}
+// GetUnresolvedTransactions is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) GetUnresolvedTransactions(ctx context.Context, tablet *topodatapb.Tablet, abandonAge int64) ([]*querypb.TransactionMetadata, error) {
+ return nil, nil
+}
+
+// ConcludeTransaction is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) ConcludeTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string, mm bool) error {
+ return nil
+}
+
+// ConcludeTransaction is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) MysqlHostMetrics(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error) {
+ return nil, nil
+}
+
+// ReadTransaction is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) ReadTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*querypb.TransactionMetadata, error) {
+ return nil, nil
+}
+
+func (client *FakeTabletManagerClient) GetTransactionInfo(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ return nil, nil
+}
+
//
// Replication related methods
//
@@ -320,6 +357,11 @@ func (client *FakeTabletManagerClient) PopulateReparentJournal(ctx context.Conte
return nil
}
+// ReadReparentJournalInfo is part of the tmclient.TabletManagerClient interface.
+func (client *FakeTabletManagerClient) ReadReparentJournalInfo(ctx context.Context, tablet *topodatapb.Tablet) (int, error) {
+ return 10, nil
+}
+
// DemotePrimary is part of the tmclient.TabletManagerClient interface.
func (client *FakeTabletManagerClient) DemotePrimary(ctx context.Context, tablet *topodatapb.Tablet) (*replicationdatapb.PrimaryStatus, error) {
return nil, nil
diff --git a/go/vt/vttablet/flags.go b/go/vt/vttablet/flags.go
deleted file mode 100644
index 994080b95a5..00000000000
--- a/go/vt/vttablet/flags.go
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
-Copyright 2023 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 vttablet
-
-import (
- "time"
-
- "github.com/spf13/pflag"
-
- "vitess.io/vitess/go/vt/servenv"
-)
-
-const (
- // VReplicationExperimentalFlags is a bitmask of experimental features in vreplication.
- VReplicationExperimentalFlagOptimizeInserts = int64(1)
- VReplicationExperimentalFlagAllowNoBlobBinlogRowImage = int64(2)
- VReplicationExperimentalFlagVPlayerBatching = int64(4)
-)
-
-var (
- // Default flags.
- VReplicationExperimentalFlags = VReplicationExperimentalFlagOptimizeInserts | VReplicationExperimentalFlagAllowNoBlobBinlogRowImage
- VReplicationNetReadTimeout = 300
- VReplicationNetWriteTimeout = 600
- CopyPhaseDuration = 1 * time.Hour
-)
-
-func init() {
- servenv.OnParseFor("vttablet", registerFlags)
- servenv.OnParseFor("vtcombo", registerFlags)
-
-}
-
-func registerFlags(fs *pflag.FlagSet) {
- fs.Int64Var(&VReplicationExperimentalFlags, "vreplication_experimental_flags", VReplicationExperimentalFlags,
- "(Bitmask) of experimental features in vreplication to enable")
- fs.IntVar(&VReplicationNetReadTimeout, "vreplication_net_read_timeout", VReplicationNetReadTimeout, "Session value of net_read_timeout for vreplication, in seconds")
- fs.IntVar(&VReplicationNetWriteTimeout, "vreplication_net_write_timeout", VReplicationNetWriteTimeout, "Session value of net_write_timeout for vreplication, in seconds")
- fs.DurationVar(&CopyPhaseDuration, "vreplication_copy_phase_duration", CopyPhaseDuration, "Duration for each copy phase loop (before running the next catchup: default 1h)")
-}
diff --git a/go/vt/vttablet/grpcqueryservice/server.go b/go/vt/vttablet/grpcqueryservice/server.go
index c26d291aa50..e3c179ce856 100644
--- a/go/vt/vttablet/grpcqueryservice/server.go
+++ b/go/vt/vttablet/grpcqueryservice/server.go
@@ -183,11 +183,8 @@ func (q *query) StartCommit(ctx context.Context, request *querypb.StartCommitReq
request.EffectiveCallerId,
request.ImmediateCallerId,
)
- if err := q.server.StartCommit(ctx, request.Target, request.TransactionId, request.Dtid); err != nil {
- return nil, vterrors.ToGRPC(err)
- }
-
- return &querypb.StartCommitResponse{}, nil
+ state, err := q.server.StartCommit(ctx, request.Target, request.TransactionId, request.Dtid)
+ return &querypb.StartCommitResponse{State: state}, vterrors.ToGRPC(err)
}
// SetRollback is part of the queryservice.QueryServer interface
@@ -240,7 +237,7 @@ func (q *query) UnresolvedTransactions(ctx context.Context, request *querypb.Unr
request.EffectiveCallerId,
request.ImmediateCallerId,
)
- transactions, err := q.server.UnresolvedTransactions(ctx, request.Target)
+ transactions, err := q.server.UnresolvedTransactions(ctx, request.Target, request.AbandonAge)
if err != nil {
return nil, vterrors.ToGRPC(err)
}
diff --git a/go/vt/vttablet/grpctabletconn/conn.go b/go/vt/vttablet/grpctabletconn/conn.go
index a76505383b7..d2d5604d808 100644
--- a/go/vt/vttablet/grpctabletconn/conn.go
+++ b/go/vt/vttablet/grpctabletconn/conn.go
@@ -351,11 +351,12 @@ func (conn *gRPCQueryClient) CreateTransaction(ctx context.Context, target *quer
// StartCommit atomically commits the transaction along with the
// decision to commit the associated 2pc transaction.
-func (conn *gRPCQueryClient) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) error {
+func (conn *gRPCQueryClient) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (querypb.StartCommitState, error) {
conn.mu.RLock()
defer conn.mu.RUnlock()
if conn.cc == nil {
- return tabletconn.ConnClosed
+ // This can be marked as fail as not other process will try to commit this transaction.
+ return querypb.StartCommitState_Fail, tabletconn.ConnClosed
}
req := &querypb.StartCommitRequest{
@@ -365,11 +366,12 @@ func (conn *gRPCQueryClient) StartCommit(ctx context.Context, target *querypb.Ta
TransactionId: transactionID,
Dtid: dtid,
}
- _, err := conn.c.StartCommit(ctx, req)
- if err != nil {
- return tabletconn.ErrorFromGRPC(err)
+ resp, err := conn.c.StartCommit(ctx, req)
+ err = tabletconn.ErrorFromGRPC(err)
+ if resp != nil {
+ return resp.State, err
}
- return nil
+ return querypb.StartCommitState_Unknown, err
}
// SetRollback transitions the 2pc transaction to the Rollback state.
@@ -439,7 +441,7 @@ func (conn *gRPCQueryClient) ReadTransaction(ctx context.Context, target *queryp
}
// UnresolvedTransactions returns all unresolved distributed transactions.
-func (conn *gRPCQueryClient) UnresolvedTransactions(ctx context.Context, target *querypb.Target) ([]*querypb.TransactionMetadata, error) {
+func (conn *gRPCQueryClient) UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error) {
conn.mu.RLock()
defer conn.mu.RUnlock()
if conn.cc == nil {
@@ -450,6 +452,7 @@ func (conn *gRPCQueryClient) UnresolvedTransactions(ctx context.Context, target
Target: target,
EffectiveCallerId: callerid.EffectiveCallerIDFromContext(ctx),
ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx),
+ AbandonAge: abandonAgeSeconds,
}
response, err := conn.c.UnresolvedTransactions(ctx, req)
if err != nil {
@@ -691,6 +694,7 @@ func (conn *gRPCQueryClient) VStream(ctx context.Context, request *binlogdatapb.
Position: request.Position,
Filter: request.Filter,
TableLastPKs: request.TableLastPKs,
+ Options: request.Options,
}
stream, err := conn.c.VStream(ctx, req)
if err != nil {
diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go
index c3ff1b02767..20aa1e5f39f 100644
--- a/go/vt/vttablet/grpctmclient/client.go
+++ b/go/vt/vttablet/grpctmclient/client.go
@@ -384,6 +384,19 @@ func (client *Client) SetReadWrite(ctx context.Context, tablet *topodatapb.Table
return err
}
+// ChangeTags is part of the tmclient.TabletManagerClient interface.
+func (client *Client) ChangeTags(ctx context.Context, tablet *topodatapb.Tablet, tabletTags map[string]string, replace bool) (*tabletmanagerdatapb.ChangeTagsResponse, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+ return c.ChangeTags(ctx, &tabletmanagerdatapb.ChangeTagsRequest{
+ Tags: tabletTags,
+ Replace: replace,
+ })
+}
+
// ChangeType is part of the tmclient.TabletManagerClient interface.
func (client *Client) ChangeType(ctx context.Context, tablet *topodatapb.Tablet, dbType topodatapb.TabletType, semiSync bool) error {
c, closer, err := client.dialer.dial(ctx, tablet)
@@ -658,6 +671,86 @@ func (client *Client) ExecuteFetchAsApp(ctx context.Context, tablet *topodatapb.
return response.Result, nil
}
+// GetUnresolvedTransactions is part of the tmclient.TabletManagerClient interface.
+func (client *Client) GetUnresolvedTransactions(ctx context.Context, tablet *topodatapb.Tablet, abandonAge int64) ([]*querypb.TransactionMetadata, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+
+ response, err := c.GetUnresolvedTransactions(ctx, &tabletmanagerdatapb.GetUnresolvedTransactionsRequest{
+ AbandonAge: abandonAge,
+ })
+ if err != nil {
+ return nil, err
+ }
+ return response.Transactions, nil
+}
+
+// ConcludeTransaction is part of the tmclient.TabletManagerClient interface.
+func (client *Client) ConcludeTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string, mm bool) error {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return err
+ }
+ defer closer.Close()
+
+ _, err = c.ConcludeTransaction(ctx, &tabletmanagerdatapb.ConcludeTransactionRequest{
+ Dtid: dtid,
+ Mm: mm,
+ })
+ return err
+}
+
+func (client *Client) MysqlHostMetrics(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+
+ resp, err := c.MysqlHostMetrics(ctx, req)
+ if err != nil {
+ return nil, err
+ }
+ return resp, nil
+}
+
+// ReadTransaction is part of the tmclient.TabletManagerClient interface.
+func (client *Client) ReadTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*querypb.TransactionMetadata, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+
+ resp, err := c.ReadTransaction(ctx, &tabletmanagerdatapb.ReadTransactionRequest{
+ Dtid: dtid,
+ })
+ if err != nil {
+ return nil, err
+ }
+ return resp.Transaction, nil
+}
+
+// GetTransactionInfo is part of the tmclient.TabletManagerClient interface.
+func (client *Client) GetTransactionInfo(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+
+ resp, err := c.GetTransactionInfo(ctx, &tabletmanagerdatapb.GetTransactionInfoRequest{
+ Dtid: dtid,
+ })
+ if err != nil {
+ return nil, err
+ }
+ return resp, nil
+}
+
//
// Replication related methods
//
@@ -836,6 +929,19 @@ func (client *Client) CreateVReplicationWorkflow(ctx context.Context, tablet *to
return response, nil
}
+func (client *Client) DeleteTableData(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+ response, err := c.DeleteTableData(ctx, request)
+ if err != nil {
+ return nil, err
+ }
+ return response, nil
+}
+
func (client *Client) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) {
c, closer, err := client.dialer.dial(ctx, tablet)
if err != nil {
@@ -888,6 +994,19 @@ func (client *Client) ReadVReplicationWorkflow(ctx context.Context, tablet *topo
return response, nil
}
+func (client *Client) ValidateVReplicationPermissions(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return nil, err
+ }
+ defer closer.Close()
+ response, err := c.ValidateVReplicationPermissions(ctx, request)
+ if err != nil {
+ return nil, err
+ }
+ return response, nil
+}
+
// VReplicationExec is part of the tmclient.TabletManagerClient interface.
func (client *Client) VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) {
c, closer, err := client.dialer.dial(ctx, tablet)
@@ -1004,6 +1123,20 @@ func (client *Client) PopulateReparentJournal(ctx context.Context, tablet *topod
return err
}
+// ReadReparentJournalInfo is part of the tmclient.TabletManagerClient interface.
+func (client *Client) ReadReparentJournalInfo(ctx context.Context, tablet *topodatapb.Tablet) (int, error) {
+ c, closer, err := client.dialer.dial(ctx, tablet)
+ if err != nil {
+ return 0, err
+ }
+ defer closer.Close()
+ resp, err := c.ReadReparentJournalInfo(ctx, &tabletmanagerdatapb.ReadReparentJournalInfoRequest{})
+ if err != nil {
+ return 0, err
+ }
+ return int(resp.Length), nil
+}
+
// InitReplica is part of the tmclient.TabletManagerClient interface.
func (client *Client) InitReplica(ctx context.Context, tablet *topodatapb.Tablet, parent *topodatapb.TabletAlias, replicationPosition string, timeCreatedNS int64, semiSync bool) error {
c, closer, err := client.dialer.dial(ctx, tablet)
@@ -1114,7 +1247,7 @@ func (client *Client) StopReplicationAndGetStatus(ctx context.Context, tablet *t
if err != nil {
return nil, err
}
- return &replicationdatapb.StopReplicationStatus{ //nolint
+ return &replicationdatapb.StopReplicationStatus{ // nolint
Before: response.Status.Before,
After: response.Status.After,
}, nil
diff --git a/go/vt/vttablet/grpctmserver/server.go b/go/vt/vttablet/grpctmserver/server.go
index 42cfd441eeb..6f0fd2aa4dc 100644
--- a/go/vt/vttablet/grpctmserver/server.go
+++ b/go/vt/vttablet/grpctmserver/server.go
@@ -127,6 +127,15 @@ func (s *server) SetReadWrite(ctx context.Context, request *tabletmanagerdatapb.
return response, s.tm.SetReadOnly(ctx, false)
}
+func (s *server) ChangeTags(ctx context.Context, request *tabletmanagerdatapb.ChangeTagsRequest) (response *tabletmanagerdatapb.ChangeTagsResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "ChangeTags", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+ afterTags, err := s.tm.ChangeTags(ctx, request.Tags, request.Replace)
+ return &tabletmanagerdatapb.ChangeTagsResponse{
+ Tags: afterTags,
+ }, err
+}
+
func (s *server) ChangeType(ctx context.Context, request *tabletmanagerdatapb.ChangeTypeRequest) (response *tabletmanagerdatapb.ChangeTypeResponse, err error) {
defer s.tm.HandleRPCPanic(ctx, "ChangeType", request, response, true /*verbose*/, &err)
ctx = callinfo.GRPCCallInfo(ctx)
@@ -275,6 +284,69 @@ func (s *server) ExecuteFetchAsApp(ctx context.Context, request *tabletmanagerda
return response, nil
}
+//
+// Distributed Transaction related methods
+//
+
+func (s *server) GetUnresolvedTransactions(ctx context.Context, request *tabletmanagerdatapb.GetUnresolvedTransactionsRequest) (response *tabletmanagerdatapb.GetUnresolvedTransactionsResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "GetUnresolvedTransactions", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+
+ transactions, err := s.tm.GetUnresolvedTransactions(ctx, request.AbandonAge)
+ if err != nil {
+ return nil, vterrors.ToGRPC(err)
+ }
+
+ return &tabletmanagerdatapb.GetUnresolvedTransactionsResponse{Transactions: transactions}, nil
+}
+
+func (s *server) ReadTransaction(ctx context.Context, request *tabletmanagerdatapb.ReadTransactionRequest) (response *tabletmanagerdatapb.ReadTransactionResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "ReadTransaction", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+
+ transaction, err := s.tm.ReadTransaction(ctx, request)
+ if err != nil {
+ return nil, vterrors.ToGRPC(err)
+ }
+
+ return &tabletmanagerdatapb.ReadTransactionResponse{Transaction: transaction}, nil
+}
+
+func (s *server) GetTransactionInfo(ctx context.Context, request *tabletmanagerdatapb.GetTransactionInfoRequest) (response *tabletmanagerdatapb.GetTransactionInfoResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "GetTransactionInfo", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+
+ response, err = s.tm.GetTransactionInfo(ctx, request)
+ if err != nil {
+ return nil, vterrors.ToGRPC(err)
+ }
+
+ return response, nil
+}
+
+func (s *server) ConcludeTransaction(ctx context.Context, request *tabletmanagerdatapb.ConcludeTransactionRequest) (response *tabletmanagerdatapb.ConcludeTransactionResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "ConcludeTransaction", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+
+ err = s.tm.ConcludeTransaction(ctx, request)
+ if err != nil {
+ return nil, vterrors.ToGRPC(err)
+ }
+
+ return &tabletmanagerdatapb.ConcludeTransactionResponse{}, nil
+}
+
+func (s *server) MysqlHostMetrics(ctx context.Context, request *tabletmanagerdatapb.MysqlHostMetricsRequest) (response *tabletmanagerdatapb.MysqlHostMetricsResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "MysqlHostMetrics", request, response, false /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+
+ resp, err := s.tm.MysqlHostMetrics(ctx, request)
+ if err != nil {
+ return nil, vterrors.ToGRPC(err)
+ }
+ return resp, nil
+}
+
//
// Replication related methods
//
@@ -384,6 +456,13 @@ func (s *server) CreateVReplicationWorkflow(ctx context.Context, request *tablet
return s.tm.CreateVReplicationWorkflow(ctx, request)
}
+func (s *server) DeleteTableData(ctx context.Context, request *tabletmanagerdatapb.DeleteTableDataRequest) (response *tabletmanagerdatapb.DeleteTableDataResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "DeleteTableData", request, response, true /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+ response = &tabletmanagerdatapb.DeleteTableDataResponse{}
+ return s.tm.DeleteTableData(ctx, request)
+}
+
func (s *server) DeleteVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (response *tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, err error) {
defer s.tm.HandleRPCPanic(ctx, "DeleteVReplicationWorkflow", request, response, true /*verbose*/, &err)
ctx = callinfo.GRPCCallInfo(ctx)
@@ -412,6 +491,14 @@ func (s *server) ReadVReplicationWorkflow(ctx context.Context, request *tabletma
return s.tm.ReadVReplicationWorkflow(ctx, request)
}
+func (s *server) ValidateVReplicationPermissions(ctx context.Context, request *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (response *tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "ValidateVReplicationPermissions", request, response, true /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+ response = &tabletmanagerdatapb.ValidateVReplicationPermissionsResponse{}
+ response, err = s.tm.ValidateVReplicationPermissions(ctx, request)
+ return response, err
+}
+
func (s *server) VReplicationExec(ctx context.Context, request *tabletmanagerdatapb.VReplicationExecRequest) (response *tabletmanagerdatapb.VReplicationExecResponse, err error) {
defer s.tm.HandleRPCPanic(ctx, "VReplicationExec", request, response, true /*verbose*/, &err)
ctx = callinfo.GRPCCallInfo(ctx)
@@ -477,6 +564,17 @@ func (s *server) PopulateReparentJournal(ctx context.Context, request *tabletman
return response, s.tm.PopulateReparentJournal(ctx, request.TimeCreatedNs, request.ActionName, request.PrimaryAlias, request.ReplicationPosition)
}
+func (s *server) ReadReparentJournalInfo(ctx context.Context, request *tabletmanagerdatapb.ReadReparentJournalInfoRequest) (response *tabletmanagerdatapb.ReadReparentJournalInfoResponse, err error) {
+ defer s.tm.HandleRPCPanic(ctx, "ReadReparentJournalInfo", request, response, true /*verbose*/, &err)
+ ctx = callinfo.GRPCCallInfo(ctx)
+ response = &tabletmanagerdatapb.ReadReparentJournalInfoResponse{}
+ length, err := s.tm.ReadReparentJournalInfo(ctx)
+ if err == nil {
+ response.Length = int32(length)
+ }
+ return response, err
+}
+
func (s *server) InitReplica(ctx context.Context, request *tabletmanagerdatapb.InitReplicaRequest) (response *tabletmanagerdatapb.InitReplicaResponse, err error) {
defer s.tm.HandleRPCPanic(ctx, "InitReplica", request, response, true /*verbose*/, &err)
ctx = callinfo.GRPCCallInfo(ctx)
@@ -539,6 +637,7 @@ func (s *server) StopReplicationAndGetStatus(ctx context.Context, request *table
if err == nil {
response.Status = statusResponse.Status
}
+
return response, err
}
diff --git a/go/vt/vttablet/onlineddl/executor.go b/go/vt/vttablet/onlineddl/executor.go
index 0d43d52d7f4..f8b5cfd9b8d 100644
--- a/go/vt/vttablet/onlineddl/executor.go
+++ b/go/vt/vttablet/onlineddl/executor.go
@@ -60,6 +60,7 @@ import (
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle"
@@ -93,14 +94,18 @@ var (
ptOSCBinaryPath = "/usr/bin/pt-online-schema-change"
migrationCheckInterval = 1 * time.Minute
retainOnlineDDLTables = 24 * time.Hour
- defaultCutOverThreshold = 10 * time.Second
maxConcurrentOnlineDDLs = 256
migrationNextCheckIntervals = []time.Duration{1 * time.Second, 5 * time.Second, 10 * time.Second, 20 * time.Second}
- maxConstraintNameLength = 64
cutoverIntervals = []time.Duration{0, 1 * time.Minute, 5 * time.Minute, 10 * time.Minute, 30 * time.Minute}
)
+const (
+ defaultCutOverThreshold = 10 * time.Second
+ minCutOverThreshold = 5 * time.Second
+ maxCutOverThreshold = 30 * time.Second
+)
+
func init() {
servenv.OnParseFor("vtcombo", registerOnlineDDLFlags)
servenv.OnParseFor("vttablet", registerOnlineDDLFlags)
@@ -137,31 +142,6 @@ var (
onlineDDLGrant = fmt.Sprintf("'%s'@'%s'", onlineDDLUser, "%")
)
-type ConstraintType int
-
-const (
- UnknownConstraintType ConstraintType = iota
- CheckConstraintType
- ForeignKeyConstraintType
-)
-
-var (
- constraintIndicatorMap = map[int]string{
- int(CheckConstraintType): "chk",
- int(ForeignKeyConstraintType): "fk",
- }
-)
-
-func GetConstraintType(constraintInfo sqlparser.ConstraintInfo) ConstraintType {
- if _, ok := constraintInfo.(*sqlparser.CheckConstraintDefinition); ok {
- return CheckConstraintType
- }
- if _, ok := constraintInfo.(*sqlparser.ForeignKeyDefinition); ok {
- return ForeignKeyConstraintType
- }
- return UnknownConstraintType
-}
-
type mysqlVariables struct {
host string
port int
@@ -178,6 +158,7 @@ type Executor struct {
ts *topo.Server
lagThrottler *throttle.Throttler
toggleBufferTableFunc func(cancelCtx context.Context, tableName string, timeout time.Duration, bufferQueries bool)
+ isPreparedPoolEmpty func(tableName string) bool
requestGCChecksFunc func()
tabletAlias *topodatapb.TabletAlias
@@ -223,13 +204,19 @@ func newGCTableRetainTime() time.Time {
return time.Now().UTC().Add(retainOnlineDDLTables)
}
-// getMigrationCutOverThreshold returns the cut-over threshold for the given migration. The migration's
-// DDL Strategy may explicitly set the threshold; otherwise, we return the default cut-over threshold.
-func getMigrationCutOverThreshold(onlineDDL *schema.OnlineDDL) time.Duration {
- if threshold, _ := onlineDDL.StrategySetting().CutOverThreshold(); threshold != 0 {
- return threshold
+// safeMigrationCutOverThreshold receives a desired threshold, and returns a cut-over threshold that
+// is reasonable to use
+func safeMigrationCutOverThreshold(threshold time.Duration) (time.Duration, error) {
+ switch {
+ case threshold == 0:
+ return defaultCutOverThreshold, nil
+ case threshold < minCutOverThreshold:
+ return defaultCutOverThreshold, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cut-over min value is %v", minCutOverThreshold)
+ case threshold > maxCutOverThreshold:
+ return defaultCutOverThreshold, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cut-over max value is %v", maxCutOverThreshold)
+ default:
+ return threshold, nil
}
- return defaultCutOverThreshold
}
// NewExecutor creates a new gh-ost executor.
@@ -238,6 +225,7 @@ func NewExecutor(env tabletenv.Env, tabletAlias *topodatapb.TabletAlias, ts *top
tabletTypeFunc func() topodatapb.TabletType,
toggleBufferTableFunc func(cancelCtx context.Context, tableName string, timeout time.Duration, bufferQueries bool),
requestGCChecksFunc func(),
+ isPreparedPoolEmpty func(tableName string) bool,
) *Executor {
// sanitize flags
if maxConcurrentOnlineDDLs < 1 {
@@ -255,6 +243,7 @@ func NewExecutor(env tabletenv.Env, tabletAlias *topodatapb.TabletAlias, ts *top
ts: ts,
lagThrottler: lagThrottler,
toggleBufferTableFunc: toggleBufferTableFunc,
+ isPreparedPoolEmpty: isPreparedPoolEmpty,
requestGCChecksFunc: requestGCChecksFunc,
ticks: timer.NewTimer(migrationCheckInterval),
// Gracefully return an error if any caller tries to execute
@@ -807,7 +796,7 @@ func (e *Executor) killTableLockHoldersAndAccessors(ctx context.Context, tableNa
}
rs, err := conn.Conn.ExecuteFetch(query, -1, true)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "finding queries potentially operating on table")
}
log.Infof("killTableLockHoldersAndAccessors: found %v potential queries", len(rs.Rows))
@@ -863,14 +852,17 @@ func (e *Executor) killTableLockHoldersAndAccessors(ctx context.Context, tableNa
}
rs, err := conn.Conn.ExecuteFetch(query, -1, true)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "finding transactions locking table")
}
log.Infof("killTableLockHoldersAndAccessors: found %v locking transactions", len(rs.Rows))
for _, row := range rs.Named().Rows {
threadId := row.AsInt64("trx_mysql_thread_id", 0)
log.Infof("killTableLockHoldersAndAccessors: killing connection %v with transaction on table", threadId)
killConnection := fmt.Sprintf("KILL %d", threadId)
- _, _ = conn.Conn.ExecuteFetch(killConnection, 1, false)
+ _, err = conn.Conn.ExecuteFetch(killConnection, 1, false)
+ if err != nil {
+ log.Errorf("Unable to kill the connection %d: %v", threadId, err)
+ }
}
}
}
@@ -880,7 +872,7 @@ func (e *Executor) killTableLockHoldersAndAccessors(ctx context.Context, tableNa
// cutOverVReplMigration stops vreplication, then removes the _vt.vreplication entry for the given migration
func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, shouldForceCutOver bool) error {
if err := e.incrementCutoverAttempts(ctx, s.workflow); err != nil {
- return err
+ return vterrors.Wrapf(err, "cutover: failed incrementing cutover attempts")
}
tmClient := e.tabletManagerClient()
@@ -889,33 +881,35 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// sanity checks:
vreplTable, err := getVreplTable(s)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "cutover: failed getting vreplication table")
}
// get topology client & entities:
tablet, err := e.ts.GetTablet(ctx, e.tabletAlias)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "cutover: failed reading vreplication table")
}
// information about source tablet
- onlineDDL, _, err := e.readMigration(ctx, s.workflow)
+ onlineDDL, row, err := e.readMigration(ctx, s.workflow)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "cutover: failed reading migration")
}
+ needsShadowTableAnalysis := row["shadow_analyzed_timestamp"].IsNull()
isVreplicationTestSuite := onlineDDL.StrategySetting().IsVreplicationTestSuite()
e.updateMigrationStage(ctx, onlineDDL.UUID, "starting cut-over")
var sentryTableName string
- migrationCutOverThreshold := getMigrationCutOverThreshold(onlineDDL)
-
- waitForPos := func(s *VReplStream, pos replication.Position) error {
- ctx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ waitForPos := func(s *VReplStream, pos replication.Position, timeout time.Duration) error {
+ ctx, cancel := context.WithTimeout(ctx, timeout)
defer cancel()
// Wait for target to reach the up-to-date pos
if err := tmClient.VReplicationWaitForPos(ctx, tablet.Tablet, s.id, replication.EncodePosition(pos)); err != nil {
- return err
+ if s, _ := e.readVReplStream(ctx, s.workflow, true); s != nil {
+ err = vterrors.Wrapf(err, "read vrepl position %v", s.pos)
+ }
+ return vterrors.Wrapf(err, "failed waiting for position %v", replication.EncodePosition(pos))
}
// Target is now in sync with source!
return nil
@@ -929,14 +923,14 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// in that place as possible.
sentryTableName, err = schema.GenerateGCTableName(schema.HoldTableGCState, newGCTableRetainTime())
if err != nil {
- return nil
+ return vterrors.Wrapf(err, "failed creating sentry table name")
}
// We create the sentry table before toggling writes, because this involves a WaitForPos, which takes some time. We
// don't want to overload the buffering time with this excessive wait.
if err := e.updateArtifacts(ctx, onlineDDL.UUID, sentryTableName); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed updating artifacts with sentry table name")
}
dropSentryTableQuery := sqlparser.BuildParsedQuery(sqlDropTableIfExists, sentryTableName)
@@ -958,35 +952,83 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// still have a record of the sentry table, and gcArtifacts() will still be able to take
// care of it in the future.
}()
- parsed := sqlparser.BuildParsedQuery(sqlCreateSentryTable, sentryTableName)
- if _, err := e.execQuery(ctx, parsed.Query); err != nil {
- return err
- }
- e.updateMigrationStage(ctx, onlineDDL.UUID, "sentry table created: %s", sentryTableName)
+ preparation := func() error {
+ preparationsConn, err := e.pool.Get(ctx, nil)
+ if err != nil {
+ return vterrors.Wrap(err, "failed getting preparation connection")
+ }
+ defer preparationsConn.Recycle()
+ // Set large enough `@@lock_wait_timeout` so that it does not interfere with the cut-over operation.
+ // The code will ensure everything that needs to be terminated by `onlineDDL.CutOverThreshold` will be terminated.
+ preparationConnRestoreLockWaitTimeout, err := e.initConnectionLockWaitTimeout(ctx, preparationsConn.Conn, 3*onlineDDL.CutOverThreshold)
+ if err != nil {
+ return vterrors.Wrap(err, "failed setting lock_wait_timeout on locking connection")
+ }
+ defer preparationConnRestoreLockWaitTimeout()
+
+ if needsShadowTableAnalysis {
+ // Run `ANALYZE TABLE` on the vreplication table so that it has up-to-date statistics at cut-over
+ parsed := sqlparser.BuildParsedQuery(sqlAnalyzeTable, vreplTable)
+ if _, err := preparationsConn.Conn.Exec(ctx, parsed.Query, -1, false); err != nil {
+ // Best effort only. Do not fail the mgiration if this fails.
+ _ = e.updateMigrationMessage(ctx, "failed ANALYZE shadow table", s.workflow)
+ } else {
+ _ = e.updateMigrationTimestamp(ctx, "shadow_analyzed_timestamp", s.workflow)
+ }
+ // This command will have blocked the table for writes, presumably only for a brief time. But this can cause
+ // vreplication to now lag. Thankfully we're gonna create the sentry table and waitForPos.
+ }
+ parsed := sqlparser.BuildParsedQuery(sqlCreateSentryTable, sentryTableName)
+ if _, err := preparationsConn.Conn.Exec(ctx, parsed.Query, 1, false); err != nil {
+ return vterrors.Wrapf(err, "failed creating sentry table")
+ }
+ e.updateMigrationStage(ctx, onlineDDL.UUID, "sentry table created: %s", sentryTableName)
+ return nil
+ }
+ if err := preparation(); err != nil {
+ return vterrors.Wrapf(err, "failed preparation")
+ }
postSentryPos, err := e.primaryPosition(ctx)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed getting primary pos after sentry creation")
}
e.updateMigrationStage(ctx, onlineDDL.UUID, "waiting for post-sentry pos: %v", replication.EncodePosition(postSentryPos))
- if err := waitForPos(s, postSentryPos); err != nil {
- return err
+ // We have not yet locked anything, stopped anything, or done anything that otherwise
+ // impacts query serving so we wait for a multiple of the cutover threshold here, with
+ // that variable primarily serving to limit the max time we later spend waiting for
+ // a position again AFTER we've taken the locks and table access is blocked.
+ if err := waitForPos(s, postSentryPos, onlineDDL.CutOverThreshold*3); err != nil {
+ return vterrors.Wrapf(err, "failed waiting for pos after sentry creation")
}
e.updateMigrationStage(ctx, onlineDDL.UUID, "post-sentry pos reached")
}
lockConn, err := e.pool.Get(ctx, nil)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed getting locking connection")
}
defer lockConn.Recycle()
+ // Set large enough `@@lock_wait_timeout` so that it does not interfere with the cut-over operation.
+ // The code will ensure everything that needs to be terminated by `onlineDDL.CutOverThreshold` will be terminated.
+ lockConnRestoreLockWaitTimeout, err := e.initConnectionLockWaitTimeout(ctx, lockConn.Conn, 5*onlineDDL.CutOverThreshold)
+ if err != nil {
+ return vterrors.Wrapf(err, "failed setting lock_wait_timeout on locking connection")
+ }
+ defer lockConnRestoreLockWaitTimeout()
defer lockConn.Conn.Exec(ctx, sqlUnlockTables, 1, false)
renameCompleteChan := make(chan error)
renameWasSuccessful := false
renameConn, err := e.pool.Get(ctx, nil)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed getting rename connection")
+ }
+ // Set large enough `@@lock_wait_timeout` so that it does not interfere with the cut-over operation.
+ // The code will ensure everything that needs to be terminated by `onlineDDL.CutOverThreshold` will be terminated.
+ renameConnRestoreLockWaitTimeout, err := e.initConnectionLockWaitTimeout(ctx, renameConn.Conn, 5*onlineDDL.CutOverThreshold*4)
+ if err != nil {
+ return vterrors.Wrapf(err, "failed setting lock_wait_timeout on rename connection")
}
defer renameConn.Recycle()
defer func() {
@@ -997,10 +1039,12 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
}
}
}()
+ defer renameConnRestoreLockWaitTimeout()
+
// See if backend MySQL server supports 'rename_table_preserve_foreign_key' variable
preserveFKSupported, err := e.isPreserveForeignKeySupported(ctx)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed checking for 'rename_table_preserve_foreign_key' support")
}
if preserveFKSupported {
// This code is only applicable when MySQL supports the 'rename_table_preserve_foreign_key' variable. This variable
@@ -1017,13 +1061,13 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// This function waits until it finds the RENAME TABLE... query running in MySQL's PROCESSLIST, or until timeout
// The function assumes that one of the renamed tables is locked, thus causing the RENAME to block. If nothing
// is locked, then the RENAME will be near-instantaneous and it's unlikely that the function will find it.
- renameWaitCtx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ renameWaitCtx, cancel := context.WithTimeout(ctx, onlineDDL.CutOverThreshold)
defer cancel()
for {
renameProcessFound, err := e.doesConnectionInfoMatch(renameWaitCtx, renameConn.Conn.ID(), "rename")
if err != nil {
- return err
+ return vterrors.Wrapf(err, "searching for rename process")
}
if renameProcessFound {
return nil
@@ -1046,7 +1090,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// Preparation is complete. We proceed to cut-over.
toggleBuffering := func(bufferQueries bool) error {
log.Infof("toggling buffering: %t in migration %v", bufferQueries, onlineDDL.UUID)
- timeout := migrationCutOverThreshold + qrBufferExtraTimeout
+ timeout := onlineDDL.CutOverThreshold + qrBufferExtraTimeout
e.toggleBufferTableFunc(bufferingCtx, onlineDDL.Table, timeout, bufferQueries)
if !bufferQueries {
@@ -1057,7 +1101,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
bufferingContextCancel()
// force re-read of tables
if err := tmClient.RefreshState(grpcCtx, tablet.Tablet); err != nil {
- return err
+ return vterrors.Wrapf(err, "refreshing table state")
}
}
log.Infof("toggled buffering: %t in migration %v", bufferQueries, onlineDDL.UUID)
@@ -1077,7 +1121,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
err = toggleBuffering(true)
defer reenableWritesOnce()
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed enabling buffering")
}
// Give a fraction of a second for a scenario where a query is in
// query executor, it passed the ACLs and is _about to_ execute. This will be nicer to those queries:
@@ -1087,8 +1131,13 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
time.Sleep(100 * time.Millisecond)
if shouldForceCutOver {
+ // We should only proceed with forceful cut over if there is no pending atomic transaction for the table.
+ // This will help in keeping the atomicity guarantee of a prepared transaction.
+ if err := e.checkOnPreparedPool(ctx, onlineDDL.Table, 100*time.Millisecond); err != nil {
+ return vterrors.Wrapf(err, "checking prepared pool for table")
+ }
if err := e.killTableLockHoldersAndAccessors(ctx, onlineDDL.Table); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed killing table lock holders and accessors")
}
}
@@ -1107,11 +1156,11 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// real production
e.updateMigrationStage(ctx, onlineDDL.UUID, "locking tables")
- lockCtx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ lockCtx, cancel := context.WithTimeout(ctx, onlineDDL.CutOverThreshold)
defer cancel()
lockTableQuery := sqlparser.BuildParsedQuery(sqlLockTwoTablesWrite, sentryTableName, onlineDDL.Table)
if _, err := lockConn.Conn.Exec(lockCtx, lockTableQuery.Query, 1, false); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed locking tables")
}
e.updateMigrationStage(ctx, onlineDDL.UUID, "renaming tables")
@@ -1123,7 +1172,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// the rename should block, because of the LOCK. Wait for it to show up.
e.updateMigrationStage(ctx, onlineDDL.UUID, "waiting for RENAME to block")
if err := waitForRenameProcess(); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed waiting for rename process")
}
e.updateMigrationStage(ctx, onlineDDL.UUID, "RENAME found")
}
@@ -1131,7 +1180,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
e.updateMigrationStage(ctx, onlineDDL.UUID, "reading post-lock pos")
postWritesPos, err := e.primaryPosition(ctx)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed reading pos after locking")
}
// Right now: new queries are buffered, any existing query will have executed, and worst case scenario is
@@ -1143,19 +1192,19 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
// Writes are now disabled on table. Read up-to-date vreplication info, specifically to get latest (and fixed) pos:
s, err = e.readVReplStream(ctx, s.workflow, false)
if err != nil {
- return err
+ return vterrors.Wrapf(err, "failed reading vreplication table after locking")
}
e.updateMigrationStage(ctx, onlineDDL.UUID, "waiting for post-lock pos: %v", replication.EncodePosition(postWritesPos))
- if err := waitForPos(s, postWritesPos); err != nil {
+ if err := waitForPos(s, postWritesPos, onlineDDL.CutOverThreshold); err != nil {
e.updateMigrationStage(ctx, onlineDDL.UUID, "timeout while waiting for post-lock pos: %v", err)
- return err
+ return vterrors.Wrapf(err, "failed waiting for pos after locking")
}
go log.Infof("cutOverVReplMigration %v: done waiting for position %v", s.workflow, replication.EncodePosition(postWritesPos))
// Stop vreplication
e.updateMigrationStage(ctx, onlineDDL.UUID, "stopping vreplication")
if _, err := e.vreplicationExec(ctx, tablet.Tablet, binlogplayer.StopVReplication(s.id, "stopped for online DDL cutover")); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed stopping vreplication")
}
go log.Infof("cutOverVReplMigration %v: stopped vreplication", s.workflow)
@@ -1172,7 +1221,7 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
} else {
e.updateMigrationStage(ctx, onlineDDL.UUID, "validating rename is still in place")
if err := waitForRenameProcess(); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed waiting for rename process before dropping sentry table")
}
// Normal (non-testing) alter table
@@ -1180,26 +1229,26 @@ func (e *Executor) cutOverVReplMigration(ctx context.Context, s *VReplStream, sh
{
dropTableQuery := sqlparser.BuildParsedQuery(sqlDropTable, sentryTableName)
- lockCtx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ lockCtx, cancel := context.WithTimeout(ctx, onlineDDL.CutOverThreshold)
defer cancel()
if _, err := lockConn.Conn.Exec(lockCtx, dropTableQuery.Query, 1, false); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed dropping sentry table")
}
}
{
- lockCtx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ lockCtx, cancel := context.WithTimeout(ctx, onlineDDL.CutOverThreshold)
defer cancel()
e.updateMigrationStage(ctx, onlineDDL.UUID, "unlocking tables")
if _, err := lockConn.Conn.Exec(lockCtx, sqlUnlockTables, 1, false); err != nil {
- return err
+ return vterrors.Wrapf(err, "failed unlocking tables")
}
}
{
- lockCtx, cancel := context.WithTimeout(ctx, migrationCutOverThreshold)
+ lockCtx, cancel := context.WithTimeout(ctx, onlineDDL.CutOverThreshold)
defer cancel()
e.updateMigrationStage(lockCtx, onlineDDL.UUID, "waiting for RENAME to complete")
if err := <-renameCompleteChan; err != nil {
- return err
+ return vterrors.Wrapf(err, "failed waiting for rename to complete")
}
renameWasSuccessful = true
}
@@ -1260,183 +1309,22 @@ func (e *Executor) initMigrationSQLMode(ctx context.Context, onlineDDL *schema.O
return deferFunc, nil
}
-// newConstraintName generates a new, unique name for a constraint. Our problem is that a MySQL
-// constraint's name is unique in the schema (!). And so as we duplicate the original table, we must
-// create completely new names for all constraints.
-// Moreover, we really want this name to be consistent across all shards. We therefore use a deterministic
-// UUIDv5 (SHA) function over the migration UUID, table name, and constraint's _contents_.
-// We _also_ include the original constraint name as prefix, as room allows
-// for example, if the original constraint name is "check_1",
-// we might generate "check_1_cps1okb4uafunfqusi2lp22u3".
-// If we then again migrate a table whose constraint name is "check_1_cps1okb4uafunfqusi2lp22u3 " we
-// get for example "check_1_19l09s37kbhj4axnzmi10e18k" (hash changes, and we still try to preserve original name)
-//
-// Furthermore, per bug report https://bugs.mysql.com/bug.php?id=107772, if the user doesn't provide a name for
-// their CHECK constraint, then MySQL picks a name in this format _chk_.
-// Example: sometable_chk_1
-// Next, when MySQL is asked to RENAME TABLE and sees a constraint with this format, it attempts to rename
-// the constraint with the new table's name. This is problematic for Vitess, because we often rename tables to
-// very long names, such as _vt_HOLD_394f9e6dfc3d11eca0390a43f95f28a3_20220706091048.
-// As we rename the constraint to e.g. `sometable_chk_1_cps1okb4uafunfqusi2lp22u3`, this makes MySQL want to
-// call the new constraint something like _vt_HOLD_394f9e6dfc3d11eca0390a43f95f28a3_20220706091048_chk_1_cps1okb4uafunfqusi2lp22u3,
-// which exceeds the 64 character limit for table names. Long story short, we also trim down if the constraint seems
-// to be auto-generated.
-func (e *Executor) newConstraintName(onlineDDL *schema.OnlineDDL, constraintType ConstraintType, hashExists map[string]bool, seed string, oldName string) string {
- constraintIndicator := constraintIndicatorMap[int(constraintType)]
- oldName = schemadiff.ExtractConstraintOriginalName(onlineDDL.Table, oldName)
- hash := textutil.UUIDv5Base36(onlineDDL.UUID, onlineDDL.Table, seed)
- for i := 1; hashExists[hash]; i++ {
- hash = textutil.UUIDv5Base36(onlineDDL.UUID, onlineDDL.Table, seed, fmt.Sprintf("%d", i))
- }
- hashExists[hash] = true
- suffix := "_" + hash
- maxAllowedNameLength := maxConstraintNameLength - len(suffix)
- newName := oldName
- if newName == "" {
- newName = constraintIndicator // start with something that looks consistent with MySQL's naming
- }
- if len(newName) > maxAllowedNameLength {
- newName = newName[0:maxAllowedNameLength]
- }
- newName = newName + suffix
- return newName
-}
-
-// validateAndEditCreateTableStatement inspects the CreateTable AST and does the following:
-// - extra validation (no FKs for now...)
-// - generate new and unique names for all constraints (CHECK and FK; yes, why not handle FK names; even as we don't support FKs today, we may in the future)
-func (e *Executor) validateAndEditCreateTableStatement(onlineDDL *schema.OnlineDDL, createTable *sqlparser.CreateTable) (constraintMap map[string]string, err error) {
- constraintMap = map[string]string{}
- hashExists := map[string]bool{}
-
- validateWalk := func(node sqlparser.SQLNode) (kontinue bool, err error) {
- switch node := node.(type) {
- case *sqlparser.ForeignKeyDefinition:
- if !onlineDDL.StrategySetting().IsAllowForeignKeysFlag() {
- return false, schema.ErrForeignKeyFound
- }
- case *sqlparser.ConstraintDefinition:
- oldName := node.Name.String()
- newName := e.newConstraintName(onlineDDL, GetConstraintType(node.Details), hashExists, sqlparser.CanonicalString(node.Details), oldName)
- node.Name = sqlparser.NewIdentifierCI(newName)
- constraintMap[oldName] = newName
- }
- return true, nil
- }
- if err := sqlparser.Walk(validateWalk, createTable); err != nil {
- return constraintMap, err
- }
- return constraintMap, nil
-}
+// initConnectionLockWaitTimeout sets the given lock_wait_timeout for the given connection, with a deferred value restoration function
+func (e *Executor) initConnectionLockWaitTimeout(ctx context.Context, conn *connpool.Conn, lockWaitTimeout time.Duration) (deferFunc func(), err error) {
+ deferFunc = func() {}
-// validateAndEditAlterTableStatement inspects the AlterTable statement and:
-// - modifies any CONSTRAINT name according to given name mapping
-// - explode ADD FULLTEXT KEY into multiple statements
-func (e *Executor) validateAndEditAlterTableStatement(capableOf capabilities.CapableOf, onlineDDL *schema.OnlineDDL, alterTable *sqlparser.AlterTable, constraintMap map[string]string) (alters []*sqlparser.AlterTable, err error) {
- capableOfInstantDDLXtrabackup, err := capableOf(capabilities.InstantDDLXtrabackupCapability)
- if err != nil {
- return nil, err
+ if _, err := conn.Exec(ctx, `set @lock_wait_timeout=@@session.lock_wait_timeout`, 0, false); err != nil {
+ return deferFunc, vterrors.Errorf(vtrpcpb.Code_UNKNOWN, "could not read lock_wait_timeout: %v", err)
}
-
- hashExists := map[string]bool{}
- validateWalk := func(node sqlparser.SQLNode) (kontinue bool, err error) {
- switch node := node.(type) {
- case *sqlparser.DropKey:
- if node.Type == sqlparser.CheckKeyType || node.Type == sqlparser.ForeignKeyType {
- // drop a check or a foreign key constraint
- mappedName, ok := constraintMap[node.Name.String()]
- if !ok {
- return false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "Found DROP CONSTRAINT: %v, but could not find constraint name in map", sqlparser.CanonicalString(node))
- }
- node.Name = sqlparser.NewIdentifierCI(mappedName)
- }
- case *sqlparser.AddConstraintDefinition:
- oldName := node.ConstraintDefinition.Name.String()
- newName := e.newConstraintName(onlineDDL, GetConstraintType(node.ConstraintDefinition.Details), hashExists, sqlparser.CanonicalString(node.ConstraintDefinition.Details), oldName)
- node.ConstraintDefinition.Name = sqlparser.NewIdentifierCI(newName)
- constraintMap[oldName] = newName
- }
- return true, nil
- }
- if err := sqlparser.Walk(validateWalk, alterTable); err != nil {
- return alters, err
- }
- alters = append(alters, alterTable)
- // Handle ADD FULLTEXT KEY statements
- countAddFullTextStatements := 0
- redactedOptions := make([]sqlparser.AlterOption, 0, len(alterTable.AlterOptions))
- for i := range alterTable.AlterOptions {
- opt := alterTable.AlterOptions[i]
- switch opt := opt.(type) {
- case sqlparser.AlgorithmValue:
- if !capableOfInstantDDLXtrabackup {
- // we do not pass ALGORITHM. We choose our own ALGORITHM.
- continue
- }
- case *sqlparser.AddIndexDefinition:
- if opt.IndexDefinition.Info.Type == sqlparser.IndexTypeFullText {
- countAddFullTextStatements++
- if countAddFullTextStatements > 1 {
- // We've already got one ADD FULLTEXT KEY. We can't have another
- // in the same statement
- extraAlterTable := &sqlparser.AlterTable{
- Table: alterTable.Table,
- AlterOptions: []sqlparser.AlterOption{opt},
- }
- if !capableOfInstantDDLXtrabackup {
- extraAlterTable.AlterOptions = append(extraAlterTable.AlterOptions, copyAlgorithm)
- }
- alters = append(alters, extraAlterTable)
- continue
- }
- }
- }
- redactedOptions = append(redactedOptions, opt)
- }
- alterTable.AlterOptions = redactedOptions
- if !capableOfInstantDDLXtrabackup {
- alterTable.AlterOptions = append(alterTable.AlterOptions, copyAlgorithm)
- }
- return alters, nil
-}
-
-// duplicateCreateTable parses the given `CREATE TABLE` statement, and returns:
-// - The format CreateTable AST
-// - A new CreateTable AST, with the table renamed as `newTableName`, and with constraints renamed deterministically
-// - Map of renamed constraints
-func (e *Executor) duplicateCreateTable(ctx context.Context, onlineDDL *schema.OnlineDDL, originalCreateTable *sqlparser.CreateTable, newTableName string) (
- newCreateTable *sqlparser.CreateTable,
- constraintMap map[string]string,
- err error,
-) {
- newCreateTable = sqlparser.Clone(originalCreateTable)
- newCreateTable.SetTable(newCreateTable.GetTable().Qualifier.CompliantName(), newTableName)
-
- // If this table has a self-referencing foreign key constraint, ensure the referenced table gets renamed:
- renameSelfFK := func(node sqlparser.SQLNode) (kontinue bool, err error) {
- switch node := node.(type) {
- case *sqlparser.ConstraintDefinition:
- fk, ok := node.Details.(*sqlparser.ForeignKeyDefinition)
- if !ok {
- return true, nil
- }
- if referencedTableName := fk.ReferenceDefinition.ReferencedTable.Name.String(); referencedTableName == originalCreateTable.Table.Name.String() {
- // This is a self-referencing foreign key
- // We need to rename the referenced table as well
- fk.ReferenceDefinition.ReferencedTable.Name = sqlparser.NewIdentifierCS(newTableName)
- }
- }
- return true, nil
+ timeoutSeconds := int64(lockWaitTimeout.Seconds())
+ setQuery := fmt.Sprintf("set @@session.lock_wait_timeout=%d", timeoutSeconds)
+ if _, err := conn.Exec(ctx, setQuery, 0, false); err != nil {
+ return deferFunc, err
}
- _ = sqlparser.Walk(renameSelfFK, newCreateTable)
-
- // manipulate CreateTable statement: take care of constraints names which have to be
- // unique across the schema
- constraintMap, err = e.validateAndEditCreateTableStatement(onlineDDL, newCreateTable)
- if err != nil {
- return nil, nil, err
+ deferFunc = func() {
+ conn.Exec(ctx, "set @@session.lock_wait_timeout=@lock_wait_timeout", 0, false)
}
- return newCreateTable, constraintMap, nil
+ return deferFunc, nil
}
// createDuplicateTableLike creates the table named by `newTableName` in the likeness of onlineDDL.Table
@@ -1451,7 +1339,7 @@ func (e *Executor) createDuplicateTableLike(ctx context.Context, newTableName st
if err != nil {
return nil, nil, err
}
- vreplCreateTable, constraintMap, err := e.duplicateCreateTable(ctx, onlineDDL, originalCreateTable, newTableName)
+ vreplCreateTable, constraintMap, err := schemadiff.DuplicateCreateTable(originalCreateTable, onlineDDL.UUID, newTableName, onlineDDL.StrategySetting().IsAllowForeignKeysFlag())
if err != nil {
return nil, nil, err
}
@@ -1502,7 +1390,7 @@ func (e *Executor) initVreplicationOriginalMigration(ctx context.Context, online
// Also, change any constraint names:
capableOf := mysql.ServerVersionCapableOf(conn.ServerVersion)
- alters, err := e.validateAndEditAlterTableStatement(capableOf, onlineDDL, alterTable, constraintMap)
+ alters, err := schemadiff.ValidateAndEditAlterTableStatement(onlineDDL.Table, onlineDDL.UUID, capableOf, alterTable, constraintMap)
if err != nil {
return v, err
}
@@ -2155,7 +2043,9 @@ func (e *Executor) readMigration(ctx context.Context, uuid string) (onlineDDL *s
WasReadyToComplete: row.AsInt64("was_ready_to_complete", 0),
TabletAlias: row["tablet"].ToString(),
MigrationContext: row["migration_context"].ToString(),
+ CutOverThreshold: time.Second * time.Duration(row.AsInt64("cutover_threshold_seconds", 0)),
}
+ onlineDDL.CutOverThreshold, _ = safeMigrationCutOverThreshold(onlineDDL.CutOverThreshold)
return onlineDDL, row, nil
}
@@ -2970,7 +2860,7 @@ func (e *Executor) analyzeDropDDLActionMigration(ctx context.Context, onlineDDL
// Analyze foreign keys:
for _, constraint := range createTable.TableSpec.Constraints {
- if GetConstraintType(constraint.Details) == ForeignKeyConstraintType {
+ if schemadiff.GetConstraintType(constraint.Details) == schemadiff.ForeignKeyConstraintType {
removedForeignKeyNames = append(removedForeignKeyNames, constraint.Name.String())
}
}
@@ -3072,7 +2962,7 @@ func (e *Executor) executeCreateDDLActionMigration(ctx context.Context, onlineDD
newCreateTable := sqlparser.Clone(originalCreateTable)
// Rewrite this CREATE TABLE statement such that CONSTRAINT names are edited,
// specifically removing any prefix.
- if _, err := e.validateAndEditCreateTableStatement(onlineDDL, newCreateTable); err != nil {
+ if _, err := schemadiff.ValidateAndEditCreateTableStatement(onlineDDL.Table, onlineDDL.UUID, newCreateTable, onlineDDL.StrategySetting().IsAllowForeignKeysFlag()); err != nil {
return failMigration(err)
}
ddlStmt = newCreateTable
@@ -3199,20 +3089,6 @@ func (e *Executor) executeAlterViewOnline(ctx context.Context, onlineDDL *schema
return nil
}
-// addInstantAlgorithm adds or modifies the AlterTable's ALGORITHM to INSTANT
-func (e *Executor) addInstantAlgorithm(alterTable *sqlparser.AlterTable) {
- instantOpt := sqlparser.AlgorithmValue("INSTANT")
- for i, opt := range alterTable.AlterOptions {
- if _, ok := opt.(sqlparser.AlgorithmValue); ok {
- // replace an existing algorithm
- alterTable.AlterOptions[i] = instantOpt
- return
- }
- }
- // append an algorithm
- alterTable.AlterOptions = append(alterTable.AlterOptions, instantOpt)
-}
-
// executeSpecialAlterDDLActionMigrationIfApplicable sees if the given migration can be executed via special execution path, that isn't a full blown online schema change process.
func (e *Executor) executeSpecialAlterDDLActionMigrationIfApplicable(ctx context.Context, onlineDDL *schema.OnlineDDL) (specialMigrationExecuted bool, err error) {
// Before we jump on to strategies... Some ALTERs can be optimized without having to run through
@@ -3234,7 +3110,7 @@ func (e *Executor) executeSpecialAlterDDLActionMigrationIfApplicable(ctx context
switch specialPlan.operation {
case instantDDLSpecialOperation:
- e.addInstantAlgorithm(specialPlan.alterTable)
+ schemadiff.AddInstantAlgorithm(specialPlan.alterTable)
onlineDDL.SQL = sqlparser.CanonicalString(specialPlan.alterTable)
if _, err := e.executeDirectly(ctx, onlineDDL); err != nil {
return false, err
@@ -3660,6 +3536,28 @@ func (e *Executor) readVReplStream(ctx context.Context, uuid string, okIfMissing
if err := prototext.Unmarshal([]byte(s.source), s.bls); err != nil {
return nil, err
}
+ {
+ // It's possible that an earlier error was overshadowed by a new non-error `message` values.
+ // Let's read _vt.vreplication_log to see whether there's any terminal errors in vreplication's history.
+ query, err := sqlparser.ParseAndBind(sqlReadVReplLogErrors,
+ sqltypes.Int32BindVariable(s.id),
+ sqltypes.StringBindVariable(vreplication.TerminalErrorIndicator),
+ )
+ if err != nil {
+ return nil, err
+ }
+ r, err := e.execQuery(ctx, query)
+ if err != nil {
+ return nil, err
+ }
+ // The query has LIMIT 1, ie returns at most one row
+ if row := r.Named().Row(); row != nil {
+ s.state = binlogdatapb.VReplicationWorkflowState_Error
+ if message := row.AsString("message", ""); message != "" {
+ s.message = "vreplication: " + message
+ }
+ }
+ }
return s, nil
}
@@ -3685,55 +3583,36 @@ func (e *Executor) isPreserveForeignKeySupported(ctx context.Context) (isSupport
// and is up to date with the binlogs.
func (e *Executor) isVReplMigrationReadyToCutOver(ctx context.Context, onlineDDL *schema.OnlineDDL, s *VReplStream) (isReady bool, err error) {
// Check all the cases where migration is still running:
- {
- // when ready to cut-over, pos must have some value
- if s.pos == "" {
- return false, nil
- }
+ // when ready to cut-over, pos must have some value
+ if s.pos == "" {
+ return false, nil
}
- {
- // Both time_updated and transaction_timestamp must be in close proximity to each
- // other and to the time now, otherwise that means we're lagging and it's not a good time
- // to cut-over
- durationDiff := func(t1, t2 time.Time) time.Duration {
- return t1.Sub(t2).Abs()
- }
- migrationCutOverThreshold := getMigrationCutOverThreshold(onlineDDL)
-
- timeNow := time.Now()
- timeUpdated := time.Unix(s.timeUpdated, 0)
- if durationDiff(timeNow, timeUpdated) > migrationCutOverThreshold {
- return false, nil
- }
- // Let's look at transaction timestamp. This gets written by any ongoing
- // writes on the server (whether on this table or any other table)
- transactionTimestamp := time.Unix(s.transactionTimestamp, 0)
- if durationDiff(timeNow, transactionTimestamp) > migrationCutOverThreshold {
- return false, nil
- }
+ // Both time_updated and transaction_timestamp must be in close proximity to each
+ // other and to the time now, otherwise that means we're lagging and it's not a good time
+ // to cut-over
+ if s.Lag() > onlineDDL.CutOverThreshold {
+ return false, nil
}
- {
- // copy_state must have no entries for this vreplication id: if entries are
- // present that means copy is still in progress
- query, err := sqlparser.ParseAndBind(sqlReadCountCopyState,
- sqltypes.Int32BindVariable(s.id),
- )
- if err != nil {
- return false, err
- }
- r, err := e.execQuery(ctx, query)
- if err != nil {
- return false, err
- }
- csRow := r.Named().Row()
- if csRow == nil {
- return false, err
- }
- count := csRow.AsInt64("cnt", 0)
- if count > 0 {
- // Still copying
- return false, nil
- }
+ // copy_state must have no entries for this vreplication id: if entries are
+ // present that means copy is still in progress
+ query, err := sqlparser.ParseAndBind(sqlReadCountCopyState,
+ sqltypes.Int32BindVariable(s.id),
+ )
+ if err != nil {
+ return false, err
+ }
+ r, err := e.execQuery(ctx, query)
+ if err != nil {
+ return false, err
+ }
+ csRow := r.Named().Row()
+ if csRow == nil {
+ return false, err
+ }
+ count := csRow.AsInt64("cnt", 0)
+ if count > 0 {
+ // Still copying
+ return false, nil
}
return true, nil
@@ -3812,7 +3691,7 @@ func (e *Executor) reviewRunningMigrations(ctx context.Context) (countRunnning i
uuid := row["migration_uuid"].ToString()
cutoverAttempts := row.AsInt64("cutover_attempts", 0)
sinceLastCutoverAttempt := time.Second * time.Duration(row.AsInt64("seconds_since_last_cutover_attempt", 0))
- sinceReadyToComplete := time.Second * time.Duration(row.AsInt64("seconds_since_ready_to_complete", 0))
+ sinceReadyToComplete := time.Microsecond * time.Duration(row.AsInt64("microseconds_since_ready_to_complete", 0))
onlineDDL, migrationRow, err := e.readMigration(ctx, uuid)
if err != nil {
return countRunnning, cancellable, err
@@ -3880,6 +3759,7 @@ func (e *Executor) reviewRunningMigrations(ctx context.Context) (countRunnning i
}
_ = e.updateRowsCopied(ctx, uuid, s.rowsCopied)
_ = e.updateMigrationProgressByRowsCopied(ctx, uuid, s.rowsCopied)
+ _ = e.updateMigrationVreplicationLagSeconds(ctx, uuid, int64(s.Lag().Seconds()))
_ = e.updateMigrationETASecondsByProgress(ctx, uuid)
if s.timeThrottled != 0 {
// Avoid creating a 0000-00-00 00:00:00 timestamp
@@ -3936,8 +3816,10 @@ func (e *Executor) reviewRunningMigrations(ctx context.Context) (countRunnning i
if err := e.cutOverVReplMigration(ctx, s, shouldForceCutOver); err != nil {
_ = e.updateMigrationMessage(ctx, uuid, err.Error())
log.Errorf("cutOverVReplMigration failed: err=%v", err)
- if merr, ok := err.(*sqlerror.SQLError); ok {
- switch merr.Num {
+
+ if sqlErr, isSQLErr := sqlerror.NewSQLErrorFromError(err).(*sqlerror.SQLError); isSQLErr && sqlErr != nil {
+ // let's see if this error is actually acceptable
+ switch sqlErr.Num {
case sqlerror.ERTooLongIdent:
go e.CancelMigration(ctx, uuid, err.Error(), false)
}
@@ -4636,6 +4518,18 @@ func (e *Executor) updateRowsCopied(ctx context.Context, uuid string, rowsCopied
return err
}
+func (e *Executor) updateMigrationVreplicationLagSeconds(ctx context.Context, uuid string, vreplicationLagSeconds int64) error {
+ query, err := sqlparser.ParseAndBind(sqlUpdateMigrationVreplicationLagSeconds,
+ sqltypes.Int64BindVariable(vreplicationLagSeconds),
+ sqltypes.StringBindVariable(uuid),
+ )
+ if err != nil {
+ return err
+ }
+ _, err = e.execQuery(ctx, query)
+ return err
+}
+
func (e *Executor) updateVitessLivenessIndicator(ctx context.Context, uuid string, livenessIndicator int64) error {
query, err := sqlparser.ParseAndBind(sqlUpdateMigrationVitessLivenessIndicator,
sqltypes.Int64BindVariable(livenessIndicator),
@@ -4858,6 +4752,42 @@ func (e *Executor) ForceCutOverPendingMigrations(ctx context.Context) (result *s
return result, nil
}
+func (e *Executor) SetMigrationCutOverThreshold(ctx context.Context, uuid string, thresholdString string) (result *sqltypes.Result, err error) {
+ if atomic.LoadInt64(&e.isOpen) == 0 {
+ return nil, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, schema.ErrOnlineDDLDisabled.Error())
+ }
+ if !schema.IsOnlineDDLUUID(uuid) {
+ return nil, vterrors.Errorf(vtrpcpb.Code_UNKNOWN, "Not a valid migration ID in FORCE_CUTOVER: %s", uuid)
+ }
+ threshold, err := time.ParseDuration(thresholdString)
+ if err != nil {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid cut-over threshold value: %s. Try '5s' to '30s'", thresholdString)
+ }
+
+ log.Infof("SetMigrationCutOverThreshold: request to set cut-over threshold to %v on migration %s", threshold, uuid)
+ e.migrationMutex.Lock()
+ defer e.migrationMutex.Unlock()
+
+ threshold, err = safeMigrationCutOverThreshold(threshold)
+ if err != nil {
+ return nil, err
+ }
+ query, err := sqlparser.ParseAndBind(sqlUpdateCutOverThresholdSeconds,
+ sqltypes.Int64BindVariable(int64(threshold.Seconds())),
+ sqltypes.StringBindVariable(uuid),
+ )
+ if err != nil {
+ return nil, err
+ }
+ rs, err := e.execQuery(ctx, query)
+ if err != nil {
+ return nil, err
+ }
+ e.triggerNextCheckInterval()
+ log.Infof("SetMigrationCutOverThreshold: migration %s cut-over threshold was set to", uuid, threshold)
+ return rs, nil
+}
+
// CompleteMigration clears the postpone_completion flag for a given migration, assuming it was set in the first place
func (e *Executor) CompleteMigration(ctx context.Context, uuid string) (result *sqltypes.Result, err error) {
if atomic.LoadInt64(&e.isOpen) == 0 {
@@ -5012,7 +4942,7 @@ func (e *Executor) submitCallbackIfNonConflicting(
) (
result *sqltypes.Result, err error,
) {
- if !onlineDDL.StrategySetting().IsSingleton() && !onlineDDL.StrategySetting().IsSingletonContext() {
+ if !onlineDDL.StrategySetting().IsSingleton() && !onlineDDL.StrategySetting().IsSingletonContext() && !onlineDDL.StrategySetting().IsSingletonTable() {
// not a singleton. No conflict
return callback()
}
@@ -5058,6 +4988,15 @@ func (e *Executor) submitCallbackIfNonConflicting(
}
// no conflict? continue looking for other pending migrations
}
+ case onlineDDL.StrategySetting().IsSingletonTable():
+ // We will reject this migration if there's any pending migration for the same table
+ for _, row := range rows {
+ pendingTableName := row["mysql_table"].ToString()
+ if onlineDDL.Table == pendingTableName {
+ pendingUUID := row["migration_uuid"].ToString()
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "singleton-table migration rejected: found pending migration: %s for the same table: %s", pendingUUID, onlineDDL.Table)
+ }
+ }
}
return nil
}()
@@ -5134,7 +5073,14 @@ func (e *Executor) SubmitMigration(
// Explicit retention indicated by `--retain-artifact` DDL strategy flag for this migration. Override!
retainArtifactsSeconds = int64((retainArtifacts).Seconds())
}
-
+ cutoverThreshold, err := onlineDDL.StrategySetting().CutOverThreshold()
+ if err != nil {
+ return nil, vterrors.Wrapf(err, "parsing cut-over threshold in migration %v", onlineDDL.UUID)
+ }
+ cutoverThreshold, err = safeMigrationCutOverThreshold(cutoverThreshold)
+ if err != nil {
+ return nil, vterrors.Wrapf(err, "validating cut-over threshold in migration %v", onlineDDL.UUID)
+ }
_, allowConcurrentMigration := e.allowConcurrentMigration(onlineDDL)
submitQuery, err := sqlparser.ParseAndBind(sqlInsertMigration,
sqltypes.StringBindVariable(onlineDDL.UUID),
@@ -5150,6 +5096,7 @@ func (e *Executor) SubmitMigration(
sqltypes.StringBindVariable(string(schema.OnlineDDLStatusQueued)),
sqltypes.StringBindVariable(e.TabletAliasString()),
sqltypes.Int64BindVariable(retainArtifactsSeconds),
+ sqltypes.Int64BindVariable(int64(cutoverThreshold.Seconds())),
sqltypes.BoolBindVariable(onlineDDL.StrategySetting().IsPostponeLaunch()),
sqltypes.BoolBindVariable(onlineDDL.StrategySetting().IsPostponeCompletion()),
sqltypes.BoolBindVariable(allowConcurrentMigration),
@@ -5311,3 +5258,21 @@ func (e *Executor) OnSchemaMigrationStatus(ctx context.Context,
return e.onSchemaMigrationStatus(ctx, uuidParam, status, dryRun, progressPct, etaSeconds, rowsCopied, hint)
}
+
+// checkOnPreparedPool checks if there are any cross-shard prepared transactions on the given table
+func (e *Executor) checkOnPreparedPool(ctx context.Context, table string, waitTime time.Duration) error {
+ if e.isPreparedPoolEmpty(table) {
+ return nil
+ }
+
+ select {
+ case <-ctx.Done():
+ // Return context error if context is done
+ return ctx.Err()
+ case <-time.After(waitTime):
+ if e.isPreparedPoolEmpty(table) {
+ return nil
+ }
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "cannot force cut-over on non-empty prepared pool for table: %s", table)
+ }
+}
diff --git a/go/vt/vttablet/onlineddl/executor_test.go b/go/vt/vttablet/onlineddl/executor_test.go
index 1dc5447bbb9..105da7fc1e3 100644
--- a/go/vt/vttablet/onlineddl/executor_test.go
+++ b/go/vt/vttablet/onlineddl/executor_test.go
@@ -21,393 +21,13 @@ Functionality of this Executor is tested in go/test/endtoend/onlineddl/...
package onlineddl
import (
- "context"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
-
- "vitess.io/vitess/go/mysql"
- "vitess.io/vitess/go/vt/vtenv"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
-
- "vitess.io/vitess/go/vt/schema"
- "vitess.io/vitess/go/vt/sqlparser"
)
-var (
- testMySQLVersion = "8.0.34"
-)
-
-func TestGetConstraintType(t *testing.T) {
- {
- typ := GetConstraintType(&sqlparser.CheckConstraintDefinition{})
- assert.Equal(t, CheckConstraintType, typ)
- }
- {
- typ := GetConstraintType(&sqlparser.ForeignKeyDefinition{})
- assert.Equal(t, ForeignKeyConstraintType, typ)
- }
-}
-
-func TestValidateAndEditCreateTableStatement(t *testing.T) {
- e := Executor{
- env: tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "ValidateAndEditCreateTableStatementTest"),
- }
- tt := []struct {
- name string
- query string
- strategyOptions string
- expectError string
- countConstraints int
- expectConstraintMap map[string]string
- }{
- {
- name: "table with FK, not allowed",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- parent_id int not null,
- primary key(id),
- constraint test_ibfk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action
- )
- `,
- expectError: schema.ErrForeignKeyFound.Error(),
- },
- {
- name: "table with FK, allowed",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- parent_id int not null,
- primary key(id),
- constraint test_ibfk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action
- )
- `,
- strategyOptions: "--unsafe-allow-foreign-keys",
- countConstraints: 1,
- expectConstraintMap: map[string]string{"test_ibfk": "test_ibfk_2wtivm6zk4lthpz14g9uoyaqk"},
- },
- {
- name: "table with default FK name, strip table name",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- parent_id int not null,
- primary key(id),
- constraint onlineddl_test_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action
- )
- `,
- strategyOptions: "--unsafe-allow-foreign-keys",
- countConstraints: 1,
- // we want 'onlineddl_test_' to be stripped out:
- expectConstraintMap: map[string]string{"onlineddl_test_ibfk_1": "ibfk_1_2wtivm6zk4lthpz14g9uoyaqk"},
- },
- {
- name: "table with anonymous FK, allowed",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- parent_id int not null,
- primary key(id),
- foreign key (parent_id) references onlineddl_test_parent (id) on delete no action
- )
- `,
- strategyOptions: "--unsafe-allow-foreign-keys",
- countConstraints: 1,
- expectConstraintMap: map[string]string{"": "fk_2wtivm6zk4lthpz14g9uoyaqk"},
- },
- {
- name: "table with CHECK constraints",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- primary key(id),
- constraint check_1 CHECK ((i >= 0)),
- constraint check_2 CHECK ((i <> 5)),
- constraint check_3 CHECK ((i >= 0)),
- constraint chk_1111033c1d2d5908bf1f956ba900b192_check_4 CHECK ((i >= 0))
- )
- `,
- countConstraints: 4,
- expectConstraintMap: map[string]string{
- "check_1": "check_1_7dbssrkwdaxhdunwi5zj53q83",
- "check_2": "check_2_ehg3rtk6ejvbxpucimeess30o",
- "check_3": "check_3_0se0t8x98mf8v7lqmj2la8j9u",
- "chk_1111033c1d2d5908bf1f956ba900b192_check_4": "chk_1111033c1d2d5908bf1f956ba900b192_c_0c2c3bxi9jp4evqrct44wg3xh",
- },
- },
- {
- name: "table with both FOREIGN and CHECK constraints",
- query: `
- create table onlineddl_test (
- id int auto_increment,
- i int not null,
- primary key(id),
- constraint check_1 CHECK ((i >= 0)),
- constraint test_ibfk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action,
- constraint chk_1111033c1d2d5908bf1f956ba900b192_check_4 CHECK ((i >= 0))
- )
- `,
- strategyOptions: "--unsafe-allow-foreign-keys",
- countConstraints: 3,
- expectConstraintMap: map[string]string{
- "check_1": "check_1_7dbssrkwdaxhdunwi5zj53q83",
- "chk_1111033c1d2d5908bf1f956ba900b192_check_4": "chk_1111033c1d2d5908bf1f956ba900b192_c_0se0t8x98mf8v7lqmj2la8j9u",
- "test_ibfk": "test_ibfk_2wtivm6zk4lthpz14g9uoyaqk",
- },
- },
- }
- for _, tc := range tt {
- t.Run(tc.name, func(t *testing.T) {
- stmt, err := e.env.Environment().Parser().ParseStrictDDL(tc.query)
- require.NoError(t, err)
- createTable, ok := stmt.(*sqlparser.CreateTable)
- require.True(t, ok)
-
- onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "onlineddl_test", Options: tc.strategyOptions}
- constraintMap, err := e.validateAndEditCreateTableStatement(onlineDDL, createTable)
- if tc.expectError != "" {
- assert.ErrorContains(t, err, tc.expectError)
- return
- }
- assert.NoError(t, err)
- assert.Equal(t, tc.expectConstraintMap, constraintMap)
-
- uniqueConstraintNames := map[string]bool{}
- err = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) {
- switch node := node.(type) {
- case *sqlparser.ConstraintDefinition:
- uniqueConstraintNames[node.Name.String()] = true
- }
- return true, nil
- }, createTable)
- assert.NoError(t, err)
- assert.Equal(t, tc.countConstraints, len(uniqueConstraintNames))
- assert.Equalf(t, tc.countConstraints, len(constraintMap), "got contraints: %v", constraintMap)
- })
- }
-}
-
-func TestValidateAndEditAlterTableStatement(t *testing.T) {
- e := Executor{
- env: tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestValidateAndEditAlterTableStatementTest"),
- }
- tt := []struct {
- alter string
- mySQLVersion string
- m map[string]string
- expect []string
- }{
- {
- alter: "alter table t add column i int",
- mySQLVersion: "8.0.29",
- expect: []string{"alter table t add column i int, algorithm = copy"},
- },
- {
- alter: "alter table t add column i int",
- mySQLVersion: "8.0.32",
- expect: []string{"alter table t add column i int"},
- },
- {
- alter: "alter table t add column i int, add fulltext key name1_ft (name1)",
- expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1)"},
- },
- {
- alter: "alter table t add column i int, add fulltext key name1_ft (name1), add fulltext key name2_ft (name2)",
- expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1)", "alter table t add fulltext key name2_ft (name2)"},
- },
- {
- alter: "alter table t add fulltext key name0_ft (name0), add column i int, add fulltext key name1_ft (name1), add fulltext key name2_ft (name2)",
- expect: []string{"alter table t add fulltext key name0_ft (name0), add column i int", "alter table t add fulltext key name1_ft (name1)", "alter table t add fulltext key name2_ft (name2)"},
- },
- {
- alter: "alter table t add constraint check (id != 1)",
- expect: []string{"alter table t add constraint chk_aulpn7bjeortljhguy86phdn9 check (id != 1)"},
- },
- {
- alter: "alter table t add constraint t_chk_1 check (id != 1)",
- expect: []string{"alter table t add constraint chk_1_aulpn7bjeortljhguy86phdn9 check (id != 1)"},
- },
- {
- alter: "alter table t add constraint some_check check (id != 1)",
- expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1)"},
- },
- {
- alter: "alter table t add constraint some_check check (id != 1), add constraint another_check check (id != 2)",
- expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1), add constraint another_check_4fa197273p3w96267pzm3gfi3 check (id != 2)"},
- },
- {
- alter: "alter table t add foreign key (parent_id) references onlineddl_test_parent (id) on delete no action",
- expect: []string{"alter table t add constraint fk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"},
- },
- {
- alter: "alter table t add constraint myfk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action",
- expect: []string{"alter table t add constraint myfk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"},
- },
- {
- // strip out table name
- alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action",
- expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"},
- },
- {
- // stript out table name
- alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action",
- expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"},
- },
- {
- alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, add constraint some_check check (id != 1)",
- expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1)"},
- },
- {
- alter: "alter table t drop foreign key t_ibfk_1",
- m: map[string]string{
- "t_ibfk_1": "ibfk_1_aaaaaaaaaaaaaa",
- },
- expect: []string{"alter table t drop foreign key ibfk_1_aaaaaaaaaaaaaa"},
- },
- }
-
- for _, tc := range tt {
- t.Run(tc.alter, func(t *testing.T) {
- stmt, err := e.env.Environment().Parser().ParseStrictDDL(tc.alter)
- require.NoError(t, err)
- alterTable, ok := stmt.(*sqlparser.AlterTable)
- require.True(t, ok)
-
- m := map[string]string{}
- for k, v := range tc.m {
- m[k] = v
- }
- if tc.mySQLVersion == "" {
- tc.mySQLVersion = testMySQLVersion
- }
- capableOf := mysql.ServerVersionCapableOf(tc.mySQLVersion)
- onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "t", Options: "--unsafe-allow-foreign-keys"}
- alters, err := e.validateAndEditAlterTableStatement(capableOf, onlineDDL, alterTable, m)
- assert.NoError(t, err)
- var altersStrings []string
- for _, alter := range alters {
- altersStrings = append(altersStrings, sqlparser.String(alter))
- }
- assert.Equal(t, tc.expect, altersStrings)
- })
- }
-}
-
-func TestAddInstantAlgorithm(t *testing.T) {
- e := Executor{
- env: tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "AddInstantAlgorithmTest"),
- }
- tt := []struct {
- alter string
- expect string
- }{
- {
- alter: "alter table t add column i2 int not null",
- expect: "ALTER TABLE `t` ADD COLUMN `i2` int NOT NULL, ALGORITHM = INSTANT",
- },
- {
- alter: "alter table t add column i2 int not null, lock=none",
- expect: "ALTER TABLE `t` ADD COLUMN `i2` int NOT NULL, LOCK NONE, ALGORITHM = INSTANT",
- },
- {
- alter: "alter table t add column i2 int not null, algorithm=inplace",
- expect: "ALTER TABLE `t` ADD COLUMN `i2` int NOT NULL, ALGORITHM = INSTANT",
- },
- {
- alter: "alter table t add column i2 int not null, algorithm=inplace, lock=none",
- expect: "ALTER TABLE `t` ADD COLUMN `i2` int NOT NULL, ALGORITHM = INSTANT, LOCK NONE",
- },
- }
- for _, tc := range tt {
- t.Run(tc.alter, func(t *testing.T) {
- stmt, err := e.env.Environment().Parser().ParseStrictDDL(tc.alter)
- require.NoError(t, err)
- alterTable, ok := stmt.(*sqlparser.AlterTable)
- require.True(t, ok)
-
- e.addInstantAlgorithm(alterTable)
- alterInstant := sqlparser.CanonicalString(alterTable)
-
- assert.Equal(t, tc.expect, alterInstant)
-
- stmt, err = e.env.Environment().Parser().ParseStrictDDL(alterInstant)
- require.NoError(t, err)
- _, ok = stmt.(*sqlparser.AlterTable)
- require.True(t, ok)
- })
- }
-}
-
-func TestDuplicateCreateTable(t *testing.T) {
- e := Executor{
- env: tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "DuplicateCreateTableTest"),
- }
- ctx := context.Background()
- onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "something", Strategy: "vitess", Options: "--unsafe-allow-foreign-keys"}
-
- tcases := []struct {
- sql string
- newName string
- expectSQL string
- expectMapSize int
- }{
- {
- sql: "create table t (id int primary key)",
- newName: "mytable",
- expectSQL: "create table mytable (\n\tid int primary key\n)",
- },
- {
- sql: "create table t (id int primary key, i int, constraint f foreign key (i) references parent (id) on delete cascade)",
- newName: "mytable",
- expectSQL: "create table mytable (\n\tid int primary key,\n\ti int,\n\tconstraint f_bjj16562shq086ozik3zf6kjg foreign key (i) references parent (id) on delete cascade\n)",
- expectMapSize: 1,
- },
- {
- sql: "create table self (id int primary key, i int, constraint f foreign key (i) references self (id))",
- newName: "mytable",
- expectSQL: "create table mytable (\n\tid int primary key,\n\ti int,\n\tconstraint f_8aymb58nzb78l5jhq600veg6y foreign key (i) references mytable (id)\n)",
- expectMapSize: 1,
- },
- {
- sql: "create table self (id int primary key, i1 int, i2 int, constraint f1 foreign key (i1) references self (id), constraint f1 foreign key (i2) references parent (id))",
- newName: "mytable",
- expectSQL: `create table mytable (
- id int primary key,
- i1 int,
- i2 int,
- constraint f1_1rlsg9yls1t91i35zq5gyeoq7 foreign key (i1) references mytable (id),
- constraint f1_59t4lvb1ncti6fxy27drad4jp foreign key (i2) references parent (id)
-)`,
- expectMapSize: 1,
- },
- }
- for _, tcase := range tcases {
- t.Run(tcase.sql, func(t *testing.T) {
- stmt, err := e.env.Environment().Parser().ParseStrictDDL(tcase.sql)
- require.NoError(t, err)
- originalCreateTable, ok := stmt.(*sqlparser.CreateTable)
- require.True(t, ok)
- require.NotNil(t, originalCreateTable)
- newCreateTable, constraintMap, err := e.duplicateCreateTable(ctx, onlineDDL, originalCreateTable, tcase.newName)
- assert.NoError(t, err)
- assert.NotNil(t, newCreateTable)
- assert.NotNil(t, constraintMap)
-
- newSQL := sqlparser.String(newCreateTable)
- assert.Equal(t, tcase.expectSQL, newSQL)
- assert.Equal(t, tcase.expectMapSize, len(constraintMap))
- })
- }
-}
-
func TestShouldCutOverAccordingToBackoff(t *testing.T) {
tcases := []struct {
name string
@@ -481,6 +101,38 @@ func TestShouldCutOverAccordingToBackoff(t *testing.T) {
expectShouldCutOver: false,
expectShouldForceCutOver: false,
},
+ {
+ name: "zero since ready",
+ cutoverAttempts: 3,
+ forceCutOverAfter: time.Second,
+ sinceReadyToComplete: 0,
+ expectShouldCutOver: false,
+ expectShouldForceCutOver: false,
+ },
+ {
+ name: "zero since read, zero cut-over-after",
+ cutoverAttempts: 3,
+ forceCutOverAfter: 0,
+ sinceReadyToComplete: 0,
+ expectShouldCutOver: false,
+ expectShouldForceCutOver: false,
+ },
+ {
+ name: "microsecond",
+ cutoverAttempts: 3,
+ forceCutOverAfter: time.Microsecond,
+ sinceReadyToComplete: time.Millisecond,
+ expectShouldCutOver: true,
+ expectShouldForceCutOver: true,
+ },
+ {
+ name: "microsecond, not ready",
+ cutoverAttempts: 3,
+ forceCutOverAfter: time.Millisecond,
+ sinceReadyToComplete: time.Microsecond,
+ expectShouldCutOver: false,
+ expectShouldForceCutOver: false,
+ },
{
name: "cutover-after overrides backoff",
cutoverAttempts: 3,
@@ -513,3 +165,59 @@ func TestShouldCutOverAccordingToBackoff(t *testing.T) {
})
}
}
+
+func TestSafeMigrationCutOverThreshold(t *testing.T) {
+ require.NotZero(t, defaultCutOverThreshold)
+ require.GreaterOrEqual(t, defaultCutOverThreshold, minCutOverThreshold)
+ require.LessOrEqual(t, defaultCutOverThreshold, maxCutOverThreshold)
+
+ tcases := []struct {
+ threshold time.Duration
+ expect time.Duration
+ isErr bool
+ }{
+ {
+ threshold: 0,
+ expect: defaultCutOverThreshold,
+ },
+ {
+ threshold: 2 * time.Second,
+ expect: defaultCutOverThreshold,
+ isErr: true,
+ },
+ {
+ threshold: 75 * time.Second,
+ expect: defaultCutOverThreshold,
+ isErr: true,
+ },
+ {
+ threshold: defaultCutOverThreshold,
+ expect: defaultCutOverThreshold,
+ },
+ {
+ threshold: 5 * time.Second,
+ expect: 5 * time.Second,
+ },
+ {
+ threshold: 15 * time.Second,
+ expect: 15 * time.Second,
+ },
+ {
+ threshold: 25 * time.Second,
+ expect: 25 * time.Second,
+ },
+ }
+ for _, tcase := range tcases {
+ t.Run(tcase.threshold.String(), func(t *testing.T) {
+ threshold, err := safeMigrationCutOverThreshold(tcase.threshold)
+ if tcase.isErr {
+ assert.Error(t, err)
+ require.Equal(t, tcase.expect, defaultCutOverThreshold)
+ // And keep testing, because we then also expect the threshold to be the default
+ } else {
+ assert.NoError(t, err)
+ }
+ assert.Equal(t, tcase.expect, threshold)
+ })
+ }
+}
diff --git a/go/vt/vttablet/onlineddl/schema.go b/go/vt/vttablet/onlineddl/schema.go
index 28e32e7dab4..943a3b1df07 100644
--- a/go/vt/vttablet/onlineddl/schema.go
+++ b/go/vt/vttablet/onlineddl/schema.go
@@ -32,13 +32,14 @@ const (
migration_status,
tablet,
retain_artifacts_seconds,
+ cutover_threshold_seconds,
postpone_launch,
postpone_completion,
allow_concurrent,
reverted_uuid,
is_view
) VALUES (
- %a, %a, %a, %a, %a, %a, %a, %a, %a, NOW(6), %a, %a, %a, %a, %a, %a, %a, %a, %a
+ %a, %a, %a, %a, %a, %a, %a, %a, %a, NOW(6), %a, %a, %a, %a, %a, %a, %a, %a, %a, %a
)`
sqlSelectQueuedMigrations = `SELECT
@@ -86,6 +87,11 @@ const (
WHERE
migration_uuid=%a
`
+ sqlUpdateMigrationVreplicationLagSeconds = `UPDATE _vt.schema_migrations
+ SET vreplication_lag_seconds=%a
+ WHERE
+ migration_uuid=%a
+ `
sqlUpdateMigrationIsView = `UPDATE _vt.schema_migrations
SET is_view=%a
WHERE
@@ -181,6 +187,11 @@ const (
WHERE
migration_uuid=%a
`
+ sqlUpdateCutOverThresholdSeconds = `UPDATE _vt.schema_migrations
+ SET cutover_threshold_seconds=%a
+ WHERE
+ migration_uuid=%a
+ `
sqlUpdateLaunchMigration = `UPDATE _vt.schema_migrations
SET postpone_launch=0
WHERE
@@ -209,7 +220,9 @@ const (
migration_uuid=%a
`
sqlUpdateMessage = `UPDATE _vt.schema_migrations
- SET message=%a
+ SET
+ message=%a,
+ message_timestamp=NOW(6)
WHERE
migration_uuid=%a
`
@@ -268,6 +281,7 @@ const (
cancelled_timestamp=NULL,
completed_timestamp=NULL,
last_cutover_attempt_timestamp=NULL,
+ shadow_analyzed_timestamp=NULL,
cleanup_timestamp=NULL
WHERE
migration_status IN ('failed', 'cancelled')
@@ -289,6 +303,7 @@ const (
cancelled_timestamp=NULL,
completed_timestamp=NULL,
last_cutover_attempt_timestamp=NULL,
+ shadow_analyzed_timestamp=NULL,
cleanup_timestamp=NULL
WHERE
migration_status IN ('failed', 'cancelled')
@@ -304,7 +319,7 @@ const (
postpone_completion,
force_cutover,
cutover_attempts,
- ifnull(timestampdiff(second, ready_to_complete_timestamp, now()), 0) as seconds_since_ready_to_complete,
+ ifnull(timestampdiff(microsecond, ready_to_complete_timestamp, now(6)), 0) as microseconds_since_ready_to_complete,
ifnull(timestampdiff(second, last_cutover_attempt_timestamp, now()), 0) as seconds_since_last_cutover_attempt,
timestampdiff(second, started_timestamp, now()) as elapsed_seconds
FROM _vt.schema_migrations
@@ -425,6 +440,7 @@ const (
removed_unique_keys,
migration_context,
retain_artifacts_seconds,
+ cutover_threshold_seconds,
is_view,
ready_to_complete,
ready_to_complete_timestamp is not null as was_ready_to_complete,
@@ -439,6 +455,7 @@ const (
postpone_launch,
postpone_completion,
is_immediate_operation,
+ shadow_analyzed_timestamp,
reviewed_timestamp
FROM _vt.schema_migrations
WHERE
@@ -513,6 +530,20 @@ const (
WHERE
workflow=%a
`
+ sqlReadVReplLogErrors = `SELECT
+ state,
+ message
+ FROM _vt.vreplication_log
+ WHERE
+ vrepl_id=%a
+ AND (
+ state='Error'
+ OR locate (concat(%a, ':'), message) = 1
+ )
+ ORDER BY
+ id DESC
+ LIMIT 1
+ `
sqlReadCountCopyState = `SELECT
count(*) as cnt
FROM
diff --git a/go/vt/vttablet/onlineddl/vrepl.go b/go/vt/vttablet/onlineddl/vrepl.go
index 14c52d352bf..2761c27c801 100644
--- a/go/vt/vttablet/onlineddl/vrepl.go
+++ b/go/vt/vttablet/onlineddl/vrepl.go
@@ -30,6 +30,7 @@ import (
"net/url"
"strconv"
"strings"
+ "time"
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/mysql/collations/charset"
@@ -96,6 +97,19 @@ func (v *VReplStream) hasError() (isTerminal bool, vreplError error) {
return false, nil
}
+// Lag returns the vreplication lag, as determined by the higher of the transaction timestamp and the time updated.
+func (s *VReplStream) Lag() time.Duration {
+ durationDiff := func(t1, t2 time.Time) time.Duration {
+ return t1.Sub(t2).Abs()
+ }
+ timeNow := time.Now()
+ timeUpdated := time.Unix(s.timeUpdated, 0)
+ // Let's look at transaction timestamp. This gets written by any ongoing
+ // writes on the server (whether on this table or any other table)
+ transactionTimestamp := time.Unix(s.transactionTimestamp, 0)
+ return max(durationDiff(timeNow, timeUpdated), durationDiff(timeNow, transactionTimestamp))
+}
+
// VRepl is an online DDL helper for VReplication based migrations (ddl_strategy="online")
type VRepl struct {
workflow string
@@ -289,6 +303,9 @@ func (v *VRepl) generateFilterQuery() error {
sb.WriteString(fmt.Sprintf("CONCAT(%s)", escapeName(name)))
case sourceCol.Type() == "json":
sb.WriteString(fmt.Sprintf("convert(%s using utf8mb4)", escapeName(name)))
+ case targetCol.Type() == "json": // we already know the source col is not JSON, per the above `case` condition
+ // Convert any type to JSON: encode the type as utf8mb4 text
+ sb.WriteString(fmt.Sprintf("convert(%s using utf8mb4)", escapeName(name)))
case sourceCol.IsTextual():
// Check source and target charset/encoding. If needed, create
// a binlogdatapb.CharsetConversion entry (later written to vreplication)
@@ -301,19 +318,19 @@ func (v *VRepl) generateFilterQuery() error {
if targetCol.IsTextual() && toCollation == collations.Unknown {
return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "Character set %s not supported for column %s", targetCol.Charset(), targetCol.Name())
}
-
- if trivialCharset(fromCollation) && trivialCharset(toCollation) && targetCol.Type() != "json" {
+ if trivialCharset(fromCollation) && trivialCharset(toCollation) {
+ sb.WriteString(escapeName(name))
+ } else if fromCollation == toCollation {
+ // No need for charset conversions as both have the same collation.
sb.WriteString(escapeName(name))
} else {
+ // Charset conversion required:
v.convertCharset[targetName] = &binlogdatapb.CharsetConversion{
FromCharset: sourceCol.Charset(),
ToCharset: targetCol.Charset(),
}
- sb.WriteString(fmt.Sprintf("convert(%s using utf8mb4)", escapeName(name)))
+ sb.WriteString(escapeName(name))
}
- case targetCol.Type() == "json" && sourceCol.Type() != "json":
- // Convert any type to JSON: encode the type as utf8mb4 text
- sb.WriteString(fmt.Sprintf("convert(%s using utf8mb4)", escapeName(name)))
default:
sb.WriteString(escapeName(name))
}
@@ -386,7 +403,7 @@ func (v *VRepl) analyze(ctx context.Context, conn *dbconnpool.DBConnection) erro
func (v *VRepl) generateInsertStatement() (string, error) {
ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, v.dbName)
ig.AddRow(v.workflow, v.bls, v.pos, "", "in_order:REPLICA,PRIMARY",
- binlogdatapb.VReplicationWorkflowType_OnlineDDL, binlogdatapb.VReplicationWorkflowSubType_None, false)
+ binlogdatapb.VReplicationWorkflowType_OnlineDDL, binlogdatapb.VReplicationWorkflowSubType_None, false, "")
return ig.String(), nil
}
diff --git a/go/vt/vttablet/queryservice/queryservice.go b/go/vt/vttablet/queryservice/queryservice.go
index 10cf48fd8b1..d6972bfb6a3 100644
--- a/go/vt/vttablet/queryservice/queryservice.go
+++ b/go/vt/vttablet/queryservice/queryservice.go
@@ -63,7 +63,7 @@ type QueryService interface {
// StartCommit atomically commits the transaction along with the
// decision to commit the associated 2pc transaction.
- StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error)
+ StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error)
// SetRollback transitions the 2pc transaction to the Rollback state.
// If a transaction id is provided, that transaction is also rolled back.
@@ -77,7 +77,7 @@ type QueryService interface {
ReadTransaction(ctx context.Context, target *querypb.Target, dtid string) (metadata *querypb.TransactionMetadata, err error)
// UnresolvedTransactions returns the list of unresolved distributed transactions.
- UnresolvedTransactions(ctx context.Context, target *querypb.Target) ([]*querypb.TransactionMetadata, error)
+ UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error)
// Execute for query execution
Execute(ctx context.Context, target *querypb.Target, sql string, bindVariables map[string]*querypb.BindVariable, transactionID, reservedID int64, options *querypb.ExecuteOptions) (*sqltypes.Result, error)
diff --git a/go/vt/vttablet/queryservice/wrapped.go b/go/vt/vttablet/queryservice/wrapped.go
index d24bd051c4a..c72a472a5cb 100644
--- a/go/vt/vttablet/queryservice/wrapped.go
+++ b/go/vt/vttablet/queryservice/wrapped.go
@@ -146,11 +146,13 @@ func (ws *wrappedService) CreateTransaction(ctx context.Context, target *querypb
})
}
-func (ws *wrappedService) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
- return ws.wrapper(ctx, target, ws.impl, "StartCommit", true, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) {
- innerErr := conn.StartCommit(ctx, target, transactionID, dtid)
+func (ws *wrappedService) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error) {
+ err = ws.wrapper(ctx, target, ws.impl, "StartCommit", true, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) {
+ var innerErr error
+ state, innerErr = conn.StartCommit(ctx, target, transactionID, dtid)
return canRetry(ctx, innerErr), innerErr
})
+ return state, err
}
func (ws *wrappedService) SetRollback(ctx context.Context, target *querypb.Target, dtid string, transactionID int64) (err error) {
@@ -176,10 +178,10 @@ func (ws *wrappedService) ReadTransaction(ctx context.Context, target *querypb.T
return metadata, err
}
-func (ws *wrappedService) UnresolvedTransactions(ctx context.Context, target *querypb.Target) (transactions []*querypb.TransactionMetadata, err error) {
+func (ws *wrappedService) UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) (transactions []*querypb.TransactionMetadata, err error) {
err = ws.wrapper(ctx, target, ws.impl, "UnresolvedTransactions", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (bool, error) {
var innerErr error
- transactions, innerErr = conn.UnresolvedTransactions(ctx, target)
+ transactions, innerErr = conn.UnresolvedTransactions(ctx, target, abandonAgeSeconds)
return canRetry(ctx, innerErr), innerErr
})
return transactions, err
diff --git a/go/vt/vttablet/sandboxconn/sandboxconn.go b/go/vt/vttablet/sandboxconn/sandboxconn.go
index 3c7f4c8c445..a34baef1238 100644
--- a/go/vt/vttablet/sandboxconn/sandboxconn.go
+++ b/go/vt/vttablet/sandboxconn/sandboxconn.go
@@ -49,13 +49,14 @@ type SandboxConn struct {
// These errors are triggered only for specific functions.
// For now these are just for the 2PC functions.
- MustFailPrepare int
- MustFailCommitPrepared int
- MustFailRollbackPrepared int
- MustFailCreateTransaction int
- MustFailStartCommit int
- MustFailSetRollback int
- MustFailConcludeTransaction int
+ MustFailPrepare int
+ MustFailCommitPrepared int
+ MustFailRollbackPrepared int
+ MustFailCreateTransaction int
+ MustFailStartCommit int
+ MustFailStartCommitUncertain int
+ MustFailSetRollback int
+ MustFailConcludeTransaction int
// MustFailExecute is keyed by the statement type and stores the number
// of times to fail when it sees that statement type.
// Once, exhausted it will start returning non-error response.
@@ -157,6 +158,27 @@ func NewSandboxConn(t *topodatapb.Tablet) *SandboxConn {
}
}
+// ResetCounter resets the counters in the sandboxconn.
+func (sbc *SandboxConn) ResetCounter() {
+ sbc.ExecCount.Store(0)
+ sbc.BeginCount.Store(0)
+ sbc.CommitCount.Store(0)
+ sbc.RollbackCount.Store(0)
+ sbc.AsTransactionCount.Store(0)
+ sbc.PrepareCount.Store(0)
+ sbc.CommitPreparedCount.Store(0)
+ sbc.RollbackPreparedCount.Store(0)
+ sbc.CreateTransactionCount.Store(0)
+ sbc.StartCommitCount.Store(0)
+ sbc.SetRollbackCount.Store(0)
+ sbc.ConcludeTransactionCount.Store(0)
+ sbc.ReadTransactionCount.Store(0)
+ sbc.UnresolvedTransactionsCount.Store(0)
+ sbc.ReserveCount.Store(0)
+ sbc.ReleaseCount.Store(0)
+ sbc.GetSchemaCount.Store(0)
+}
+
// RequireQueriesLocking sets the sandboxconn to require locking the access of Queries field.
func (sbc *SandboxConn) RequireQueriesLocking() {
sbc.queriesRequireLocking = true
@@ -172,6 +194,28 @@ func (sbc *SandboxConn) GetQueries() []*querypb.BoundQuery {
return sbc.Queries
}
+// GetFinalQueries gets the final queries as strings from sandboxconn.
+func (sbc *SandboxConn) GetFinalQueries() ([]string, error) {
+ if sbc.queriesRequireLocking {
+ sbc.queriesMu.Lock()
+ defer sbc.queriesMu.Unlock()
+ }
+ var queries []string
+ for _, q := range sbc.Queries {
+ stmt, err := sbc.parser.Parse(q.Sql)
+ if err != nil {
+ return nil, err
+ }
+ pq := sqlparser.NewParsedQuery(stmt)
+ query, err := pq.GenerateQuery(q.BindVariables, nil)
+ if err != nil {
+ return nil, err
+ }
+ queries = append(queries, query)
+ }
+ return queries, nil
+}
+
// ClearQueries clears the Queries in sandboxconn.
func (sbc *SandboxConn) ClearQueries() {
if sbc.queriesRequireLocking {
@@ -382,14 +426,22 @@ func (sbc *SandboxConn) CreateTransaction(ctx context.Context, target *querypb.T
// StartCommit atomically commits the transaction along with the
// decision to commit the associated 2pc transaction.
-func (sbc *SandboxConn) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
+func (sbc *SandboxConn) StartCommit(context.Context, *querypb.Target, int64, string) (state querypb.StartCommitState, err error) {
sbc.panicIfNeeded()
sbc.StartCommitCount.Add(1)
if sbc.MustFailStartCommit > 0 {
sbc.MustFailStartCommit--
- return vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "error: err")
+ return querypb.StartCommitState_Fail, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "error: err")
}
- return sbc.getError()
+ if sbc.MustFailStartCommitUncertain > 0 {
+ sbc.MustFailStartCommitUncertain--
+ return querypb.StartCommitState_Unknown, vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "uncertain error")
+ }
+ err = sbc.getError()
+ if err != nil {
+ return querypb.StartCommitState_Unknown, err
+ }
+ return querypb.StartCommitState_Success, nil
}
// SetRollback transitions the 2pc transaction to the Rollback state.
@@ -430,7 +482,7 @@ func (sbc *SandboxConn) ReadTransaction(ctx context.Context, target *querypb.Tar
}
// UnresolvedTransactions is part of the QueryService interface.
-func (sbc *SandboxConn) UnresolvedTransactions(context.Context, *querypb.Target) ([]*querypb.TransactionMetadata, error) {
+func (sbc *SandboxConn) UnresolvedTransactions(context.Context, *querypb.Target, int64) ([]*querypb.TransactionMetadata, error) {
sbc.UnresolvedTransactionsCount.Add(1)
if err := sbc.getError(); err != nil {
return nil, err
diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go
index 54deee1aed7..2d62b017433 100644
--- a/go/vt/vttablet/tabletconntest/fakequeryservice.go
+++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go
@@ -286,9 +286,9 @@ func (f *FakeQueryService) CreateTransaction(ctx context.Context, target *queryp
}
// StartCommit is part of the queryservice.QueryService interface
-func (f *FakeQueryService) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
+func (f *FakeQueryService) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error) {
if f.HasError {
- return f.TabletError
+ return querypb.StartCommitState_Fail, f.TabletError
}
if f.Panics {
panic(fmt.Errorf("test-triggered panic"))
@@ -300,7 +300,7 @@ func (f *FakeQueryService) StartCommit(ctx context.Context, target *querypb.Targ
if dtid != Dtid {
f.t.Errorf("StartCommit: invalid dtid: got %s expected %s", dtid, Dtid)
}
- return nil
+ return querypb.StartCommitState_Success, nil
}
// SetRollback is part of the queryservice.QueryService interface
@@ -360,7 +360,7 @@ func (f *FakeQueryService) ReadTransaction(ctx context.Context, target *querypb.
}
// UnresolvedTransactions is part of the queryservice.QueryService interface
-func (f *FakeQueryService) UnresolvedTransactions(ctx context.Context, target *querypb.Target) ([]*querypb.TransactionMetadata, error) {
+func (f *FakeQueryService) UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error) {
if f.HasError {
return nil, f.TabletError
}
diff --git a/go/vt/vttablet/tabletconntest/tabletconntest.go b/go/vt/vttablet/tabletconntest/tabletconntest.go
index 0f443d8d58d..416d4de9d00 100644
--- a/go/vt/vttablet/tabletconntest/tabletconntest.go
+++ b/go/vt/vttablet/tabletconntest/tabletconntest.go
@@ -294,26 +294,31 @@ func testStartCommit(t *testing.T, conn queryservice.QueryService, f *FakeQueryS
t.Log("testStartCommit")
ctx := context.Background()
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
- err := conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
- if err != nil {
- t.Fatalf("StartCommit failed: %v", err)
- }
+ state, err := conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
+ assert.Equal(t, querypb.StartCommitState_Success, state, "Unexpected state from StartCommit")
+ assert.NoError(t, err, "StartCommit failed")
}
func testStartCommitError(t *testing.T, conn queryservice.QueryService, f *FakeQueryService) {
t.Log("testStartCommitError")
f.HasError = true
- testErrorHelper(t, f, "StartCommit", func(ctx context.Context) error {
- return conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
+ var state querypb.StartCommitState
+ testErrorHelper(t, f, "StartCommit", func(ctx context.Context) (err error) {
+ state, err = conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
+ return err
})
f.HasError = false
+ assert.Equal(t, querypb.StartCommitState_Unknown, state, "Unexpected state from StartCommit")
}
func testStartCommitPanics(t *testing.T, conn queryservice.QueryService, f *FakeQueryService) {
t.Log("testStartCommitPanics")
- testPanicHelper(t, f, "StartCommit", func(ctx context.Context) error {
- return conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
+ var state querypb.StartCommitState
+ testPanicHelper(t, f, "StartCommit", func(ctx context.Context) (err error) {
+ state, err = conn.StartCommit(ctx, TestTarget, commitTransactionID, Dtid)
+ return err
})
+ assert.Equal(t, querypb.StartCommitState_Unknown, state, "Unexpected state from StartCommit")
}
func testSetRollback(t *testing.T, conn queryservice.QueryService, f *FakeQueryService) {
@@ -403,7 +408,7 @@ func testUnresolvedTransactions(t *testing.T, conn queryservice.QueryService, f
t.Log("testUnresolvedTransactions")
ctx := context.Background()
ctx = callerid.NewContext(ctx, TestCallerID, TestVTGateCallerID)
- transactions, err := conn.UnresolvedTransactions(ctx, TestTarget)
+ transactions, err := conn.UnresolvedTransactions(ctx, TestTarget, 0 /* abandonAgeSeconds */)
require.NoError(t, err)
require.True(t, proto.Equal(transactions[0], Metadata))
}
@@ -412,7 +417,7 @@ func testUnresolvedTransactionsError(t *testing.T, conn queryservice.QueryServic
t.Log("testUnresolvedTransactionsError")
f.HasError = true
testErrorHelper(t, f, "UnresolvedTransactions", func(ctx context.Context) error {
- _, err := conn.UnresolvedTransactions(ctx, TestTarget)
+ _, err := conn.UnresolvedTransactions(ctx, TestTarget, 0 /* abandonAgeSeconds */)
return err
})
f.HasError = false
@@ -421,7 +426,7 @@ func testUnresolvedTransactionsError(t *testing.T, conn queryservice.QueryServic
func testUnresolvedTransactionsPanics(t *testing.T, conn queryservice.QueryService, f *FakeQueryService) {
t.Log("testUnresolvedTransactionsPanics")
testPanicHelper(t, f, "UnresolvedTransactions", func(ctx context.Context) error {
- _, err := conn.UnresolvedTransactions(ctx, TestTarget)
+ _, err := conn.UnresolvedTransactions(ctx, TestTarget, 0 /* abandonAgeSeconds */)
return err
})
}
diff --git a/go/vt/vttablet/tabletmanager/framework_test.go b/go/vt/vttablet/tabletmanager/framework_test.go
index 27a3a562cd3..35aa7a08b46 100644
--- a/go/vt/vttablet/tabletmanager/framework_test.go
+++ b/go/vt/vttablet/tabletmanager/framework_test.go
@@ -37,6 +37,8 @@ import (
"vitess.io/vitess/go/vt/mysqlctl/tmutils"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/memorytopo"
+ "vitess.io/vitess/go/vt/vtenv"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/queryservice"
"vitess.io/vitess/go/vt/vttablet/tabletconn"
"vitess.io/vitess/go/vt/vttablet/tabletconntest"
@@ -46,12 +48,14 @@ import (
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
+ replicationdatapb "vitess.io/vitess/go/vt/proto/replicationdata"
tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
)
const (
gtidFlavor = "MySQL56"
+ serverUUID = "16b1039f-22b6-11ed-b765-0a43f95f28a3"
gtidPosition = "16b1039f-22b6-11ed-b765-0a43f95f28a3:1-220"
)
@@ -77,6 +81,7 @@ type testEnv struct {
ctx context.Context
ts *topo.Server
cells []string
+ db *fakesqldb.DB
mysqld *mysqlctl.FakeMysqlDaemon
tmc *fakeTMClient
dbName string
@@ -84,6 +89,7 @@ type testEnv struct {
}
func newTestEnv(t *testing.T, ctx context.Context, sourceKeyspace string, sourceShards []string) *testEnv {
+ vttablet.InitVReplicationConfigDefaults()
tenv := &testEnv{
ctx: context.Background(),
tmc: newFakeTMClient(),
@@ -112,9 +118,13 @@ func newTestEnv(t *testing.T, ctx context.Context, sourceKeyspace string, source
})
tmclienttest.SetProtocol(fmt.Sprintf("go.vt.vttablet.tabletmanager.framework_test_%s", t.Name()), tenv.protoName)
- tenv.mysqld = mysqlctl.NewFakeMysqlDaemon(fakesqldb.New(t))
- var err error
- tenv.mysqld.CurrentPrimaryPosition, err = replication.ParsePosition(gtidFlavor, gtidPosition)
+ tenv.db = fakesqldb.New(t)
+ tenv.mysqld = mysqlctl.NewFakeMysqlDaemon(tenv.db)
+ curPosition, err := replication.ParsePosition(gtidFlavor, gtidPosition)
+ require.NoError(t, err)
+ tenv.mysqld.SetPrimaryPositionLocked(curPosition)
+
+ err = tenv.ts.RebuildSrvVSchema(ctx, nil)
require.NoError(t, err)
return tenv
@@ -127,7 +137,7 @@ func (tenv *testEnv) close() {
tenv.mysqld.Close()
}
-//--------------------------------------
+// --------------------------------------
// Tablets
func (tenv *testEnv) addTablet(t *testing.T, id int, keyspace, shard string) *fakeTabletConn {
@@ -179,8 +189,11 @@ func (tenv *testEnv) addTablet(t *testing.T, id int, keyspace, shard string) *fa
DBConfigs: &dbconfigs.DBConfigs{
DBName: tenv.dbName,
},
+ Env: vtenv.NewTestEnv(),
+ _waitForGrantsComplete: make(chan struct{}),
+ MysqlDaemon: tenv.mysqld,
}
-
+ close(tenv.tmc.tablets[id].tm._waitForGrantsComplete) // So that we don't wait for the grants
return tenv.tmc.tablets[id]
}
@@ -241,8 +254,8 @@ func (ftc *fakeTabletConn) CreateTransaction(ctx context.Context, target *queryp
}
// fakeTabletConn implements the QueryService interface.
-func (ftc *fakeTabletConn) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
- return nil
+func (ftc *fakeTabletConn) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error) {
+ return querypb.StartCommitState_Unknown, nil
}
// fakeTabletConn implements the QueryService interface.
@@ -369,7 +382,7 @@ func (ftc *fakeTabletConn) StreamHealth(ctx context.Context, callback func(*quer
})
}
-//----------------------------------------------
+// ----------------------------------------------
// fakeTMClient
type fakeTMClient struct {
@@ -417,8 +430,6 @@ func (tmc *fakeTMClient) ApplySchema(ctx context.Context, tablet *topodatapb.Tab
}
func (tmc *fakeTMClient) schemaRequested(uid int) {
- tmc.mu.Lock()
- defer tmc.mu.Unlock()
key := strconv.Itoa(int(uid))
n, ok := tmc.getSchemaCounts[key]
if !ok {
@@ -436,6 +447,8 @@ func (tmc *fakeTMClient) getSchemaRequestCount(uid int) int {
}
func (tmc *fakeTMClient) GetSchema(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
tmc.schemaRequested(int(tablet.Alias.Uid))
// Return the schema for the tablet if it exists.
if schema, ok := tmc.tabletSchemas[int(tablet.Alias.Uid)]; ok {
@@ -463,6 +476,8 @@ func (tmc *fakeTMClient) ExecuteFetchAsDba(ctx context.Context, tablet *topodata
// and their results. You can specify exact strings or strings prefixed with
// a '/', in which case they will be treated as a valid regexp.
func (tmc *fakeTMClient) setVReplicationExecResults(tablet *topodatapb.Tablet, query string, result *sqltypes.Result) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
queries, ok := tmc.vreQueries[int(tablet.Alias.Uid)]
if !ok {
queries = make(map[string]*querypb.QueryResult)
@@ -472,13 +487,15 @@ func (tmc *fakeTMClient) setVReplicationExecResults(tablet *topodatapb.Tablet, q
}
func (tmc *fakeTMClient) VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
if result, ok := tmc.vreQueries[int(tablet.Alias.Uid)][query]; ok {
return result, nil
}
for qry, res := range tmc.vreQueries[int(tablet.Alias.Uid)] {
if strings.HasPrefix(qry, "/") {
- re := regexp.MustCompile(qry)
- if re.MatchString(qry) {
+ re := regexp.MustCompile(qry[1:])
+ if re.MatchString(query) {
return res, nil
}
}
@@ -490,14 +507,23 @@ func (tmc *fakeTMClient) PrimaryPosition(ctx context.Context, tablet *topodatapb
return fmt.Sprintf("%s/%s", gtidFlavor, gtidPosition), nil
}
+func (tmc *fakeTMClient) PrimaryStatus(ctx context.Context, tablet *topodatapb.Tablet) (*replicationdatapb.PrimaryStatus, error) {
+ pos, _ := tmc.PrimaryPosition(ctx, tablet)
+ return &replicationdatapb.PrimaryStatus{
+ Position: pos,
+ FilePosition: pos,
+ ServerUuid: serverUUID,
+ }, nil
+}
+
func (tmc *fakeTMClient) VReplicationWaitForPos(ctx context.Context, tablet *topodatapb.Tablet, id int32, pos string) error {
return nil
}
func (tmc *fakeTMClient) ExecuteFetchAsAllPrivs(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ExecuteFetchAsAllPrivsRequest) (*querypb.QueryResult, error) {
- return &querypb.QueryResult{
- RowsAffected: 1,
- }, nil
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
+ return tmc.tablets[int(tablet.Alias.Uid)].tm.ExecuteFetchAsAllPrivs(ctx, req)
}
func (tmc *fakeTMClient) VDiff(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) {
@@ -511,9 +537,17 @@ func (tmc *fakeTMClient) VDiff(ctx context.Context, tablet *topodatapb.Tablet, r
}
func (tmc *fakeTMClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
return tmc.tablets[int(tablet.Alias.Uid)].tm.CreateVReplicationWorkflow(ctx, req)
}
+func (tmc *fakeTMClient) DeleteTableData(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
+ return tmc.tablets[int(tablet.Alias.Uid)].tm.DeleteTableData(ctx, req)
+}
+
func (tmc *fakeTMClient) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (response *tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, err error) {
tmc.mu.Lock()
defer tmc.mu.Unlock()
@@ -526,17 +560,25 @@ func (tmc *fakeTMClient) DeleteVReplicationWorkflow(ctx context.Context, tablet
}
func (tmc *fakeTMClient) HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
return tmc.tablets[int(tablet.Alias.Uid)].tm.HasVReplicationWorkflows(ctx, req)
}
func (tmc *fakeTMClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
return tmc.tablets[int(tablet.Alias.Uid)].tm.ReadVReplicationWorkflow(ctx, req)
}
func (tmc *fakeTMClient) ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
return tmc.tablets[int(tablet.Alias.Uid)].tm.ReadVReplicationWorkflows(ctx, req)
}
func (tmc *fakeTMClient) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) {
+ tmc.mu.Lock()
+ defer tmc.mu.Unlock()
return tmc.tablets[int(tablet.Alias.Uid)].tm.UpdateVReplicationWorkflow(ctx, req)
}
diff --git a/go/vt/vttablet/tabletmanager/restore.go b/go/vt/vttablet/tabletmanager/restore.go
index 236d048340b..35587124108 100644
--- a/go/vt/vttablet/tabletmanager/restore.go
+++ b/go/vt/vttablet/tabletmanager/restore.go
@@ -36,28 +36,26 @@ import (
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/mysqlctl"
"vitess.io/vitess/go/vt/mysqlctl/backupstats"
+ binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/proto/vttime"
"vitess.io/vitess/go/vt/servenv"
- "vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
- "vitess.io/vitess/go/vt/vttablet/tmclient"
-
- binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
- tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
// This file handles the initial backup restore upon startup.
// It is only enabled if restore_from_backup is set.
var (
- restoreFromBackup bool
- restoreFromBackupTsStr string
- restoreConcurrency = 4
- waitForBackupInterval time.Duration
+ restoreFromBackup bool
+ restoreFromBackupAllowedEngines []string
+ restoreFromBackupTsStr string
+ restoreConcurrency = 4
+ waitForBackupInterval time.Duration
statsRestoreBackupTime *stats.String
statsRestoreBackupPosition *stats.String
@@ -65,6 +63,7 @@ var (
func registerRestoreFlags(fs *pflag.FlagSet) {
fs.BoolVar(&restoreFromBackup, "restore_from_backup", restoreFromBackup, "(init restore parameter) will check BackupStorage for a recent backup at startup and start there")
+ fs.StringSliceVar(&restoreFromBackupAllowedEngines, "restore-from-backup-allowed-engines", restoreFromBackupAllowedEngines, "(init restore parameter) if set, only backups taken with the specified engines are eligible to be restored")
fs.StringVar(&restoreFromBackupTsStr, "restore_from_backup_ts", restoreFromBackupTsStr, "(init restore parameter) if set, restore the latest backup taken at or before this timestamp. Example: '2021-04-29.133050'")
fs.IntVar(&restoreConcurrency, "restore_concurrency", restoreConcurrency, "(init restore parameter) how many concurrent files to restore at once")
fs.DurationVar(&waitForBackupInterval, "wait_for_backup_interval", waitForBackupInterval, "(init restore parameter) if this is greater than 0, instead of starting up empty when no backups are found, keep checking at this interval for a backup to appear")
@@ -132,6 +131,7 @@ func (tm *TabletManager) RestoreData(
backupTime time.Time,
restoreToTimetamp time.Time,
restoreToPos string,
+ allowedBackupEngines []string,
mysqlShutdownTimeout time.Duration) error {
if err := tm.lock(ctx); err != nil {
return err
@@ -177,9 +177,10 @@ func (tm *TabletManager) RestoreData(
startTime = time.Now()
req := &tabletmanagerdatapb.RestoreFromBackupRequest{
- BackupTime: protoutil.TimeToProto(backupTime),
- RestoreToPos: restoreToPos,
- RestoreToTimestamp: protoutil.TimeToProto(restoreToTimetamp),
+ BackupTime: protoutil.TimeToProto(backupTime),
+ RestoreToPos: restoreToPos,
+ RestoreToTimestamp: protoutil.TimeToProto(restoreToTimetamp),
+ AllowedBackupEngines: allowedBackupEngines,
}
err = tm.restoreDataLocked(ctx, logger, waitForBackupInterval, deleteBeforeRestore, req, mysqlShutdownTimeout)
if err != nil {
@@ -231,6 +232,7 @@ func (tm *TabletManager) restoreDataLocked(ctx context.Context, logger logutil.L
DryRun: request.DryRun,
Stats: backupstats.RestoreStats(),
MysqlShutdownTimeout: mysqlShutdownTimeout,
+ AllowedBackupEngines: request.AllowedBackupEngines,
}
restoreToTimestamp := protoutil.TimeFromProto(request.RestoreToTimestamp).UTC()
if request.RestoreToPos != "" && !restoreToTimestamp.IsZero() {
@@ -321,7 +323,7 @@ func (tm *TabletManager) restoreDataLocked(ctx context.Context, logger logutil.L
} else if keyspaceInfo.KeyspaceType == topodatapb.KeyspaceType_NORMAL {
// Reconnect to primary only for "NORMAL" keyspaces
params.Logger.Infof("Restore: starting replication at position %v", pos)
- if err := tm.startReplication(context.Background(), pos, originalType); err != nil {
+ if err := tm.startReplication(ctx, pos, originalType); err != nil {
return err
}
}
@@ -572,47 +574,30 @@ func (tm *TabletManager) disableReplication(ctx context.Context) error {
}
func (tm *TabletManager) startReplication(ctx context.Context, pos replication.Position, tabletType topodatapb.TabletType) error {
- if err := tm.MysqlDaemon.StopReplication(ctx, nil); err != nil {
+ // The first three steps of stopping replication, and setting the replication position,
+ // we want to do even if the context expires, so we use a background context for these tasks.
+ if err := tm.MysqlDaemon.StopReplication(context.Background(), nil); err != nil {
return vterrors.Wrap(err, "failed to stop replication")
}
- if err := tm.MysqlDaemon.ResetReplicationParameters(ctx); err != nil {
+ if err := tm.MysqlDaemon.ResetReplicationParameters(context.Background()); err != nil {
return vterrors.Wrap(err, "failed to reset replication")
}
// Set the position at which to resume from the primary.
- if err := tm.MysqlDaemon.SetReplicationPosition(ctx, pos); err != nil {
+ if err := tm.MysqlDaemon.SetReplicationPosition(context.Background(), pos); err != nil {
return vterrors.Wrap(err, "failed to set replication position")
}
- primary, err := tm.initializeReplication(ctx, tabletType)
+ primaryPosStr, err := tm.initializeReplication(ctx, tabletType)
// If we ran into an error while initializing replication, then there is no point in waiting for catch-up.
// Also, if there is no primary tablet in the shard, we don't need to proceed further.
- if err != nil || primary == nil {
+ if err != nil || primaryPosStr == "" {
return err
}
- // wait for reliable replication_lag_seconds
- // we have pos where we want to resume from
- // if PrimaryPosition is the same, that means no writes
- // have happened to primary, so we are up-to-date
- // otherwise, wait for replica's Position to change from
- // the initial pos before proceeding
- tmc := tmclient.NewTabletManagerClient()
- defer tmc.Close()
- remoteCtx, remoteCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
- defer remoteCancel()
- posStr, err := tmc.PrimaryPosition(remoteCtx, primary.Tablet)
- if err != nil {
- // It is possible that though PrimaryAlias is set, the primary tablet is unreachable
- // Log a warning and let tablet restore in that case
- // If we had instead considered this fatal, all tablets would crash-loop
- // until a primary appears, which would make it impossible to elect a primary.
- log.Warningf("Can't get primary replication position after restore: %v", err)
- return nil
- }
- primaryPos, err := replication.DecodePosition(posStr)
+ primaryPos, err := replication.DecodePosition(primaryPosStr)
if err != nil {
- return vterrors.Wrapf(err, "can't decode primary replication position: %q", posStr)
+ return vterrors.Wrapf(err, "can't decode primary replication position: %q", primaryPos)
}
if !pos.Equal(primaryPos) {
diff --git a/go/vt/vttablet/tabletmanager/rpc_actions.go b/go/vt/vttablet/tabletmanager/rpc_actions.go
index 45dd51670ba..2ebef6540ed 100644
--- a/go/vt/vttablet/tabletmanager/rpc_actions.go
+++ b/go/vt/vttablet/tabletmanager/rpc_actions.go
@@ -78,10 +78,47 @@ func (tm *TabletManager) SetReadOnly(ctx context.Context, rdonly bool) error {
return err
}
defer tm.unlock()
+ superRo, err := tm.MysqlDaemon.IsSuperReadOnly(ctx)
+ if err != nil {
+ return err
+ }
+ if !rdonly && superRo {
+ // If super read only is set, then we need to prepare the transactions before setting read_only OFF.
+ // We need to redo the prepared transactions in read only mode using the dba user to ensure we don't lose them.
+ // setting read_only OFF will also set super_read_only OFF if it was set.
+ // If super read only is already off, then we probably called this function from PRS or some other place
+ // because it is idempotent. We only need to redo prepared transactions the first time we transition from super read only
+ // to read write.
+ return tm.redoPreparedTransactionsAndSetReadWrite(ctx)
+ }
return tm.MysqlDaemon.SetReadOnly(ctx, rdonly)
}
+// ChangeTags changes the tablet tags
+func (tm *TabletManager) ChangeTags(ctx context.Context, tabletTags map[string]string, replace bool) (map[string]string, error) {
+ if err := tm.lock(ctx); err != nil {
+ return nil, err
+ }
+ defer tm.unlock()
+
+ tags := tm.tmState.Tablet().Tags
+ if replace || len(tags) == 0 {
+ tags = tabletTags
+ } else {
+ for key, val := range tabletTags {
+ if val == "" {
+ delete(tags, key)
+ continue
+ }
+ tags[key] = val
+ }
+ }
+
+ tm.tmState.ChangeTabletTags(ctx, tags)
+ return tags, nil
+}
+
// ChangeType changes the tablet type
func (tm *TabletManager) ChangeType(ctx context.Context, tabletType topodatapb.TabletType, semiSync bool) error {
if err := tm.lock(ctx); err != nil {
@@ -97,7 +134,7 @@ func (tm *TabletManager) ChangeType(ctx context.Context, tabletType topodatapb.T
return tm.changeTypeLocked(ctx, tabletType, DBActionNone, semiSyncAction)
}
-// ChangeType changes the tablet type
+// changeTypeLocked changes the tablet type under a lock
func (tm *TabletManager) changeTypeLocked(ctx context.Context, tabletType topodatapb.TabletType, action DBAction, semiSync SemiSyncAction) error {
// We don't want to allow multiple callers to claim a tablet as drained.
if tabletType == topodatapb.TabletType_DRAINED && tm.Tablet().Type == topodatapb.TabletType_DRAINED {
diff --git a/go/vt/vttablet/tabletmanager/rpc_agent.go b/go/vt/vttablet/tabletmanager/rpc_agent.go
index 6dd21a21915..445d74cb930 100644
--- a/go/vt/vttablet/tabletmanager/rpc_agent.go
+++ b/go/vt/vttablet/tabletmanager/rpc_agent.go
@@ -51,6 +51,8 @@ type RPCTM interface {
SetReadOnly(ctx context.Context, rdonly bool) error
+ ChangeTags(ctx context.Context, tabletTags map[string]string, replace bool) (map[string]string, error)
+
ChangeType(ctx context.Context, tabletType topodatapb.TabletType, semiSync bool) error
Sleep(ctx context.Context, duration time.Duration)
@@ -83,6 +85,16 @@ type RPCTM interface {
ExecuteFetchAsApp(ctx context.Context, req *tabletmanagerdatapb.ExecuteFetchAsAppRequest) (*querypb.QueryResult, error)
+ GetUnresolvedTransactions(ctx context.Context, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error)
+
+ ReadTransaction(ctx context.Context, req *tabletmanagerdatapb.ReadTransactionRequest) (*querypb.TransactionMetadata, error)
+
+ GetTransactionInfo(ctx context.Context, req *tabletmanagerdatapb.GetTransactionInfoRequest) (*tabletmanagerdatapb.GetTransactionInfoResponse, error)
+
+ ConcludeTransaction(ctx context.Context, req *tabletmanagerdatapb.ConcludeTransactionRequest) error
+
+ MysqlHostMetrics(ctx context.Context, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error)
+
// Replication related methods
PrimaryStatus(ctx context.Context) (*replicationdatapb.PrimaryStatus, error)
@@ -106,10 +118,12 @@ type RPCTM interface {
// VReplication API
CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error)
+ DeleteTableData(ctx context.Context, req *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error)
DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error)
HasVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error)
ReadVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error)
ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error)
+ ValidateVReplicationPermissions(ctx context.Context, req *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error)
VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error)
VReplicationWaitForPos(ctx context.Context, id int32, pos string) error
UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error)
@@ -126,6 +140,8 @@ type RPCTM interface {
PopulateReparentJournal(ctx context.Context, timeCreatedNS int64, actionName string, tabletAlias *topodatapb.TabletAlias, pos string) error
+ ReadReparentJournalInfo(ctx context.Context) (int, error)
+
InitReplica(ctx context.Context, parent *topodatapb.TabletAlias, replicationPosition string, timeCreatedNS int64, semiSync bool) error
DemotePrimary(ctx context.Context) (*replicationdatapb.PrimaryStatus, error)
@@ -150,6 +166,8 @@ type RPCTM interface {
RestoreFromBackup(ctx context.Context, logger logutil.Logger, request *tabletmanagerdatapb.RestoreFromBackupRequest) error
+ IsBackupRunning() bool
+
// HandleRPCPanic is to be called in a defer statement in each
// RPC input point.
HandleRPCPanic(ctx context.Context, name string, args, reply any, verbose bool, err *error)
diff --git a/go/vt/vttablet/tabletmanager/rpc_backup.go b/go/vt/vttablet/tabletmanager/rpc_backup.go
index 906e34ca9d7..22fe72716dd 100644
--- a/go/vt/vttablet/tabletmanager/rpc_backup.go
+++ b/go/vt/vttablet/tabletmanager/rpc_backup.go
@@ -53,7 +53,13 @@ func (tm *TabletManager) Backup(ctx context.Context, logger logutil.Logger, req
if !req.AllowPrimary && currentTablet.Type == topodatapb.TabletType_PRIMARY {
return fmt.Errorf("type PRIMARY cannot take backup. if you really need to do this, rerun the backup command with --allow_primary")
}
- engine, err := mysqlctl.GetBackupEngine()
+
+ backupEngine := ""
+ if req.BackupEngine != nil {
+ backupEngine = *req.BackupEngine
+ }
+
+ engine, err := mysqlctl.GetBackupEngine(backupEngine)
if err != nil {
return vterrors.Wrap(err, "failed to find backup engine")
}
@@ -163,6 +169,7 @@ func (tm *TabletManager) Backup(ctx context.Context, logger logutil.Logger, req
Stats: backupstats.BackupStats(),
UpgradeSafe: req.UpgradeSafe,
MysqlShutdownTimeout: mysqlShutdownTimeout,
+ BackupEngine: backupEngine,
}
returnErr := mysqlctl.Backup(ctx, backupParams)
@@ -198,6 +205,10 @@ func (tm *TabletManager) RestoreFromBackup(ctx context.Context, logger logutil.L
return err
}
+func (tm *TabletManager) IsBackupRunning() bool {
+ return tm._isBackupRunning
+}
+
func (tm *TabletManager) beginBackup(backupMode string) error {
tm.mutex.Lock()
defer tm.mutex.Unlock()
diff --git a/go/vt/vttablet/tabletmanager/rpc_query.go b/go/vt/vttablet/tabletmanager/rpc_query.go
index 303bcd4614d..b0da7d9d71f 100644
--- a/go/vt/vttablet/tabletmanager/rpc_query.go
+++ b/go/vt/vttablet/tabletmanager/rpc_query.go
@@ -275,6 +275,18 @@ func (tm *TabletManager) ExecuteFetchAsApp(ctx context.Context, req *tabletmanag
return sqltypes.ResultToProto3(result), err
}
+// MysqlHostMetrics gets system metrics from mysqlctl[d]
+func (tm *TabletManager) MysqlHostMetrics(ctx context.Context, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error) {
+ mysqlResp, err := tm.MysqlDaemon.HostMetrics(ctx, tm.Cnf)
+ if err != nil {
+ return nil, err
+ }
+ resp := &tabletmanagerdatapb.MysqlHostMetricsResponse{
+ HostMetrics: mysqlResp,
+ }
+ return resp, nil
+}
+
// ExecuteQuery submits a new online DDL request
func (tm *TabletManager) ExecuteQuery(ctx context.Context, req *tabletmanagerdatapb.ExecuteQueryRequest) (*querypb.QueryResult, error) {
if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
diff --git a/go/vt/vttablet/tabletmanager/rpc_replication.go b/go/vt/vttablet/tabletmanager/rpc_replication.go
index 3e745222092..f13efa66124 100644
--- a/go/vt/vttablet/tabletmanager/rpc_replication.go
+++ b/go/vt/vttablet/tabletmanager/rpc_replication.go
@@ -22,16 +22,16 @@ import (
"strings"
"time"
+ "vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/protoutil"
-
- "vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/mysqlctl"
"vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver"
replicationdatapb "vitess.io/vitess/go/vt/proto/replicationdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
@@ -46,7 +46,11 @@ func (tm *TabletManager) ReplicationStatus(ctx context.Context) (*replicationdat
if err != nil {
return nil, err
}
- return replication.ReplicationStatusToProto(status), nil
+
+ protoStatus := replication.ReplicationStatusToProto(status)
+ protoStatus.BackupRunning = tm.IsBackupRunning()
+
+ return protoStatus, nil
}
// FullStatus returns the full status of MySQL including the replication information, semi-sync information, GTID information among others
@@ -348,6 +352,15 @@ func (tm *TabletManager) InitPrimary(ctx context.Context, semiSync bool) (string
}
defer tm.unlock()
+ semiSyncAction, err := tm.convertBoolToSemiSyncAction(ctx, semiSync)
+ if err != nil {
+ return "", err
+ }
+
+ // If semi-sync is enabled, we need to set two pc to be allowed.
+ // Otherwise, we block all Prepared calls because atomic transactions require semi-sync for correctness..
+ tm.QueryServiceControl.SetTwoPCAllowed(tabletserver.TwoPCAllowed_SemiSync, semiSyncAction == SemiSyncActionSet)
+
// Setting super_read_only `OFF` so that we can run the DDL commands
if _, err := tm.MysqlDaemon.SetSuperReadOnly(ctx, false); err != nil {
if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERUnknownSystemVariable {
@@ -369,11 +382,6 @@ func (tm *TabletManager) InitPrimary(ctx context.Context, semiSync bool) (string
return "", err
}
- semiSyncAction, err := tm.convertBoolToSemiSyncAction(ctx, semiSync)
- if err != nil {
- return "", err
- }
-
// Set the server read-write, from now on we can accept real
// client writes. Note that if semi-sync replication is enabled,
// we'll still need some replicas to be able to commit transactions.
@@ -407,6 +415,24 @@ func (tm *TabletManager) PopulateReparentJournal(ctx context.Context, timeCreate
return tm.MysqlDaemon.ExecuteSuperQueryList(ctx, cmds)
}
+// ReadReparentJournalInfo reads the information from reparent journal.
+func (tm *TabletManager) ReadReparentJournalInfo(ctx context.Context) (int, error) {
+ log.Infof("ReadReparentJournalInfo")
+ if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
+ return 0, err
+ }
+
+ query := mysqlctl.ReadReparentJournalInfoQuery()
+ res, err := tm.MysqlDaemon.FetchSuperQuery(ctx, query)
+ if err != nil {
+ return 0, err
+ }
+ if len(res.Rows) != 1 {
+ return 0, vterrors.Errorf(vtrpc.Code_INTERNAL, "unexpected rows when reading reparent journal, got %v", len(res.Rows))
+ }
+ return res.Rows[0][0].ToInt()
+}
+
// InitReplica sets replication primary and position, and waits for the
// reparent_journal table entry up to context timeout
func (tm *TabletManager) InitReplica(ctx context.Context, parent *topodatapb.TabletAlias, position string, timeCreatedNS int64, semiSync bool) error {
@@ -544,9 +570,10 @@ func (tm *TabletManager) demotePrimary(ctx context.Context, revertPartialFailure
defer func() {
if finalErr != nil && revertPartialFailure && !wasReadOnly {
+ // We need to redo the prepared transactions in read only mode using the dba user to ensure we don't lose them.
// setting read_only OFF will also set super_read_only OFF if it was set
- if err := tm.MysqlDaemon.SetReadOnly(ctx, false); err != nil {
- log.Warningf("SetReadOnly(false) failed during revert: %v", err)
+ if err = tm.redoPreparedTransactionsAndSetReadWrite(ctx); err != nil {
+ log.Warningf("RedoPreparedTransactionsAndSetReadWrite failed during revert: %v", err)
}
}
}()
@@ -594,13 +621,17 @@ func (tm *TabletManager) UndoDemotePrimary(ctx context.Context, semiSync bool) e
return err
}
+ // If semi-sync is enabled, we need to set two pc to be allowed.
+ // Otherwise, we block all Prepared calls because atomic transactions require semi-sync for correctness..
+ tm.QueryServiceControl.SetTwoPCAllowed(tabletserver.TwoPCAllowed_SemiSync, semiSyncAction == SemiSyncActionSet)
+
// If using semi-sync, we need to enable source-side.
if err := tm.fixSemiSync(ctx, topodatapb.TabletType_PRIMARY, semiSyncAction); err != nil {
return err
}
- // Now, set the server read-only false.
- if err := tm.MysqlDaemon.SetReadOnly(ctx, false); err != nil {
+ // We need to redo the prepared transactions in read only mode using the dba user to ensure we don't lose them.
+ if err = tm.redoPreparedTransactionsAndSetReadWrite(ctx); err != nil {
return err
}
@@ -699,6 +730,7 @@ func (tm *TabletManager) setReplicationSourceLocked(ctx context.Context, parentA
wasReplicating := false
shouldbeReplicating := false
status, err := tm.MysqlDaemon.ReplicationStatus(ctx)
+ replicaPosition := status.RelayLogPosition
if err == mysql.ErrNotReplica {
// This is a special error that means we actually succeeded in reading
// the status, but the status is empty because replication is not
@@ -708,6 +740,12 @@ func (tm *TabletManager) setReplicationSourceLocked(ctx context.Context, parentA
// Since we continue in the case of this error, make sure 'status' is
// in a known, empty state.
status = replication.ReplicationStatus{}
+ // The replica position we use for the errant GTID detection should be the executed
+ // GTID set since this tablet is not running replication at all.
+ replicaPosition, err = tm.MysqlDaemon.PrimaryPosition(ctx)
+ if err != nil {
+ return err
+ }
} else if err != nil {
// Abort on any other non-nil error.
return err
@@ -739,12 +777,39 @@ func (tm *TabletManager) setReplicationSourceLocked(ctx context.Context, parentA
if err != nil {
return err
}
+
host := parent.Tablet.MysqlHostname
port := parent.Tablet.MysqlPort
// If host is empty, then we shouldn't even attempt the reparent. That tablet has already shutdown.
if host == "" {
return vterrors.New(vtrpc.Code_FAILED_PRECONDITION, "Shard primary has empty mysql hostname")
}
+ // Errant GTID detection.
+ {
+ // Find the executed GTID set of the tablet that we are reparenting to.
+ // We will then compare our own position against it to verify that we don't
+ // have an errant GTID. If we find any GTID that we have, but the primary doesn't,
+ // we will not enter the replication graph and instead fail replication.
+ primaryStatus, err := tm.tmc.PrimaryStatus(ctx, parent.Tablet)
+ if err != nil {
+ return err
+ }
+ primaryPosition, err := replication.DecodePosition(primaryStatus.Position)
+ if err != nil {
+ return err
+ }
+ primarySid, err := replication.ParseSID(primaryStatus.ServerUuid)
+ if err != nil {
+ return err
+ }
+ errantGtid, err := replication.ErrantGTIDsOnReplica(replicaPosition, primaryPosition, primarySid)
+ if err != nil {
+ return err
+ }
+ if errantGtid != "" {
+ return vterrors.New(vtrpc.Code_FAILED_PRECONDITION, fmt.Sprintf("Errant GTID detected - %s; Primary GTID - %s, Replica GTID - %s", errantGtid, primaryPosition, replicaPosition.String()))
+ }
+ }
if status.SourceHost != host || status.SourcePort != port || heartbeatInterval != 0 {
// This handles both changing the address and starting replication.
if err := tm.MysqlDaemon.SetReplicationSource(ctx, host, port, heartbeatInterval, wasReplicating, shouldbeReplicating); err != nil {
@@ -832,6 +897,7 @@ func (tm *TabletManager) StopReplicationAndGetStatus(ctx context.Context, stopRe
return StopReplicationAndGetStatusResponse{}, vterrors.Wrap(err, "before status failed")
}
before := replication.ReplicationStatusToProto(rs)
+ before.BackupRunning = tm.IsBackupRunning()
if stopReplicationMode == replicationdatapb.StopReplicationMode_IOTHREADONLY {
if !rs.IOHealthy() {
@@ -878,6 +944,7 @@ func (tm *TabletManager) StopReplicationAndGetStatus(ctx context.Context, stopRe
}, vterrors.Wrap(err, "acquiring replication status failed")
}
after := replication.ReplicationStatusToProto(rsAfter)
+ after.BackupRunning = tm.IsBackupRunning()
rs.Position = rsAfter.Position
rs.RelayLogPosition = rsAfter.RelayLogPosition
@@ -910,12 +977,16 @@ func (tm *TabletManager) PromoteReplica(ctx context.Context, semiSync bool) (str
}
defer tm.unlock()
- pos, err := tm.MysqlDaemon.Promote(ctx, tm.hookExtraEnv())
+ semiSyncAction, err := tm.convertBoolToSemiSyncAction(ctx, semiSync)
if err != nil {
return "", err
}
- semiSyncAction, err := tm.convertBoolToSemiSyncAction(ctx, semiSync)
+ // If semi-sync is enabled, we need to set two pc to be allowed.
+ // Otherwise, we block all Prepared calls because atomic transactions require semi-sync for correctness..
+ tm.QueryServiceControl.SetTwoPCAllowed(tabletserver.TwoPCAllowed_SemiSync, semiSyncAction == SemiSyncActionSet)
+
+ pos, err := tm.MysqlDaemon.Promote(ctx, tm.hookExtraEnv())
if err != nil {
return "", err
}
diff --git a/go/vt/vttablet/tabletmanager/rpc_throttler.go b/go/vt/vttablet/tabletmanager/rpc_throttler.go
index ec75db6da43..1617a5b275b 100644
--- a/go/vt/vttablet/tabletmanager/rpc_throttler.go
+++ b/go/vt/vttablet/tabletmanager/rpc_throttler.go
@@ -21,14 +21,15 @@ import (
"vitess.io/vitess/go/protoutil"
"vitess.io/vitess/go/stats"
- tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/base"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
)
var (
diff --git a/go/vt/vttablet/tabletmanager/rpc_transaction.go b/go/vt/vttablet/tabletmanager/rpc_transaction.go
new file mode 100644
index 00000000000..97c2d22db0c
--- /dev/null
+++ b/go/vt/vttablet/tabletmanager/rpc_transaction.go
@@ -0,0 +1,71 @@
+/*
+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 tabletmanager
+
+import (
+ "context"
+
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+)
+
+// GetUnresolvedTransactions returns the unresolved distributed transactions list for the Metadata manager.
+func (tm *TabletManager) GetUnresolvedTransactions(ctx context.Context, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error) {
+ if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
+ return nil, err
+ }
+
+ tablet := tm.Tablet()
+ target := &querypb.Target{Keyspace: tablet.Keyspace, Shard: tablet.Shard, TabletType: tablet.Type}
+ return tm.QueryServiceControl.UnresolvedTransactions(ctx, target, abandonAgeSeconds)
+}
+
+// ReadTransaction returns the transaction metadata for the given distributed transaction ID.
+func (tm *TabletManager) ReadTransaction(ctx context.Context, req *tabletmanagerdatapb.ReadTransactionRequest) (*querypb.TransactionMetadata, error) {
+ if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
+ return nil, err
+ }
+
+ tablet := tm.Tablet()
+ target := &querypb.Target{Keyspace: tablet.Keyspace, Shard: tablet.Shard, TabletType: tablet.Type}
+ return tm.QueryServiceControl.ReadTransaction(ctx, target, req.Dtid)
+}
+
+// GetTransactionInfo returns the transaction data for the given distributed transaction ID.
+func (tm *TabletManager) GetTransactionInfo(ctx context.Context, req *tabletmanagerdatapb.GetTransactionInfoRequest) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
+ return nil, err
+ }
+
+ tablet := tm.Tablet()
+ target := &querypb.Target{Keyspace: tablet.Keyspace, Shard: tablet.Shard, TabletType: tablet.Type}
+ return tm.QueryServiceControl.GetTransactionInfo(ctx, target, req.Dtid)
+}
+
+// ConcludeTransaction concludes the given distributed transaction.
+func (tm *TabletManager) ConcludeTransaction(ctx context.Context, req *tabletmanagerdatapb.ConcludeTransactionRequest) error {
+ if err := tm.waitForGrantsToHaveApplied(ctx); err != nil {
+ return err
+ }
+
+ tablet := tm.Tablet()
+ target := &querypb.Target{Keyspace: tablet.Keyspace, Shard: tablet.Shard, TabletType: tablet.Type}
+ if req.Mm {
+ return tm.QueryServiceControl.ConcludeTransaction(ctx, target, req.Dtid)
+ }
+ return tm.QueryServiceControl.RollbackPrepared(ctx, target, req.Dtid, 0)
+}
diff --git a/go/vt/vttablet/tabletmanager/rpc_transaction_test.go b/go/vt/vttablet/tabletmanager/rpc_transaction_test.go
new file mode 100644
index 00000000000..567a6561ec5
--- /dev/null
+++ b/go/vt/vttablet/tabletmanager/rpc_transaction_test.go
@@ -0,0 +1,94 @@
+/*
+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 tabletmanager
+
+import (
+ "context"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletservermock"
+)
+
+func TestTabletManager_UnresolvedTransactions(t *testing.T) {
+ ctx := context.Background()
+
+ qsc := tabletservermock.NewController()
+ tm := &TabletManager{
+ QueryServiceControl: qsc,
+ Env: vtenv.NewTestEnv(),
+ _waitForGrantsComplete: make(chan struct{}),
+ BatchCtx: ctx,
+ }
+ close(tm._waitForGrantsComplete)
+ tm.tmState = newTMState(tm, newTestTablet(t, 100, "ks", "-80", nil))
+
+ _, err := tm.GetUnresolvedTransactions(ctx, 0)
+ require.NoError(t, err)
+ require.True(t, qsc.MethodCalled["UnresolvedTransactions"])
+}
+
+func TestTabletManager_ReadTransaction(t *testing.T) {
+ ctx := context.Background()
+
+ qsc := tabletservermock.NewController()
+ tm := &TabletManager{
+ QueryServiceControl: qsc,
+ Env: vtenv.NewTestEnv(),
+ _waitForGrantsComplete: make(chan struct{}),
+ BatchCtx: ctx,
+ }
+ close(tm._waitForGrantsComplete)
+ tm.tmState = newTMState(tm, newTestTablet(t, 100, "ks", "-80", nil))
+
+ _, err := tm.ReadTransaction(ctx, &tabletmanagerdatapb.ReadTransactionRequest{
+ Dtid: "dtid01",
+ })
+ require.NoError(t, err)
+ require.True(t, qsc.MethodCalled["ReadTransaction"])
+}
+
+func TestTabletManager_ConcludeTransaction(t *testing.T) {
+ ctx := context.Background()
+
+ qsc := tabletservermock.NewController()
+ tm := &TabletManager{
+ QueryServiceControl: qsc,
+ Env: vtenv.NewTestEnv(),
+ _waitForGrantsComplete: make(chan struct{}),
+ BatchCtx: ctx,
+ }
+ close(tm._waitForGrantsComplete)
+ tm.tmState = newTMState(tm, newTestTablet(t, 100, "ks", "-80", nil))
+
+ err := tm.ConcludeTransaction(ctx, &tabletmanagerdatapb.ConcludeTransactionRequest{
+ Dtid: "dtid01",
+ Mm: false,
+ })
+ require.NoError(t, err)
+ require.True(t, qsc.MethodCalled["RollbackPrepared"])
+
+ err = tm.ConcludeTransaction(ctx, &tabletmanagerdatapb.ConcludeTransactionRequest{
+ Dtid: "dtid01",
+ Mm: true,
+ })
+ require.NoError(t, err)
+ require.True(t, qsc.MethodCalled["ConcludeTransaction"])
+}
diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go
index c8c334d896e..8ddf1391ac4 100644
--- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go
+++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go
@@ -19,21 +19,27 @@ package tabletmanager
import (
"context"
"fmt"
+ "sort"
"strings"
+ "time"
"golang.org/x/exp/maps"
"google.golang.org/protobuf/encoding/prototext"
+ "vitess.io/vitess/go/cmd/vtctldclient/command/vreplication/movetables"
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/protoutil"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/vt/discovery"
+ "vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/proto/vttime"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vtctl/workflow"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -55,12 +61,32 @@ const (
// Retrieve the current configuration values for a workflow's vreplication stream(s).
sqlSelectVReplicationWorkflowConfig = "select id, source, cell, tablet_types, state, message from %s.vreplication where workflow = %a"
// Update the configuration values for a workflow's vreplication stream.
- sqlUpdateVReplicationWorkflowStreamConfig = "update %s.vreplication set state = %a, source = %a, cell = %a, tablet_types = %a where id = %a"
+ sqlUpdateVReplicationWorkflowStreamConfig = "update %s.vreplication set state = %a, source = %a, cell = %a, tablet_types = %a %s where id = %a"
// Update field values for multiple workflows. The final format specifier is
// used to optionally add any additional predicates to the query.
sqlUpdateVReplicationWorkflows = "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ %s.vreplication set%s where db_name = '%s'%s"
// Check if workflow is still copying.
sqlGetVReplicationCopyStatus = "select distinct vrepl_id from %s.copy_state where vrepl_id = %d"
+ // Validate the minimum set of permissions needed to manage vreplication metadata.
+ // This is a simple check for a matching user rather than any specific user@host
+ // combination.
+ sqlValidateVReplicationPermissions = `
+select count(*)>0 as good from mysql.user as u
+ left join mysql.db as d on (u.user = d.user)
+ left join mysql.tables_priv as t on (u.user = t.user)
+where u.user = %a
+ and (
+ (u.select_priv = 'y' and u.insert_priv = 'y' and u.update_priv = 'y' and u.delete_priv = 'y') /* user has global privs */
+ or (d.db = %a and d.select_priv = 'y' and d.insert_priv = 'y' and d.update_priv = 'y' and d.delete_priv = 'y') /* user has db privs */
+ or (t.db = %a and t.table_name = 'vreplication' /* user has table privs */
+ and find_in_set('select', t.table_priv)
+ and find_in_set('insert', t.table_priv)
+ and find_in_set('update', t.table_priv)
+ and find_in_set('delete', t.table_priv)
+ )
+ )
+limit 1
+`
)
var (
@@ -118,6 +144,99 @@ func (tm *TabletManager) CreateVReplicationWorkflow(ctx context.Context, req *ta
return &tabletmanagerdatapb.CreateVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil
}
+// DeleteTableData will delete data from the given tables (keys in the
+// req.Tabletfilters map) using the given filter or WHERE clauses (values
+// in the map). It will perform this work in batches of req.BatchSize
+// until all matching rows have been deleted in all tables, or the context
+// expires.
+func (tm *TabletManager) DeleteTableData(ctx context.Context, req *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error) {
+ if req == nil {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid nil request")
+ }
+
+ if len(req.TableFilters) == 0 { // Nothing to do
+ return &tabletmanagerdatapb.DeleteTableDataResponse{}, nil
+ }
+
+ // So that we do them in a predictable and uniform order.
+ tables := maps.Keys(req.TableFilters)
+ sort.Strings(tables)
+
+ batchSize := req.BatchSize
+ if batchSize < 1 {
+ batchSize = movetables.DefaultDeleteBatchSize
+ }
+ limit := &sqlparser.Limit{Rowcount: sqlparser.NewIntLiteral(fmt.Sprintf("%d", batchSize))}
+ // We will log some progress info every 100 delete batches.
+ progressRows := uint64(batchSize * 100)
+
+ throttledLogger := logutil.NewThrottledLogger("DeleteTableData", 1*time.Minute)
+ checkIfCanceled := func() error {
+ select {
+ case <-ctx.Done():
+ return vterrors.Wrap(ctx.Err(), "context expired while deleting data")
+ default:
+ return nil
+ }
+ }
+
+ for _, table := range tables {
+ stmt, err := tm.Env.Parser().Parse(fmt.Sprintf("delete from %s %s", table, req.TableFilters[table]))
+ if err != nil {
+ return nil, vterrors.Wrapf(err, "unable to build delete query for table %s", table)
+ }
+ del, ok := stmt.(*sqlparser.Delete)
+ if !ok {
+ return nil, vterrors.Wrapf(err, "unable to build delete query for table %s", table)
+ }
+ del.Limit = limit
+ query := sqlparser.String(del)
+ rowsDeleted := uint64(0)
+ // Delete all of the matching rows from the table, in batches, until we've
+ // deleted them all.
+ log.Infof("Starting deletion of data from table %s using query %q", table, query)
+ for {
+ // Back off if we're causing too much load on the database with these
+ // batch deletes.
+ if _, ok := tm.VREngine.ThrottlerClient().ThrottleCheckOKOrWaitAppName(ctx, throttlerapp.VReplicationName); !ok {
+ throttledLogger.Infof("throttling bulk data delete for table %s using query %q",
+ table, query)
+ if err := checkIfCanceled(); err != nil {
+ return nil, err
+ }
+ continue
+ }
+ res, err := tm.ExecuteFetchAsAllPrivs(ctx,
+ &tabletmanagerdatapb.ExecuteFetchAsAllPrivsRequest{
+ Query: []byte(query),
+ DbName: tm.DBConfigs.DBName,
+ })
+ if err != nil {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error deleting data using query %q: %v",
+ query, err)
+ }
+ rowsDeleted += res.RowsAffected
+ // Log some progress info periodically to give the operator some idea of
+ // how much work we've done, how much is left, and how long it may take
+ // (considering throttling, system performance, etc).
+ if rowsDeleted%progressRows == 0 {
+ log.Infof("Successfully deleted %d rows of data from table %s so far, using query %q",
+ rowsDeleted, table, query)
+ }
+ if res.RowsAffected == 0 { // We're done with this table
+ break
+ }
+ if err := checkIfCanceled(); err != nil {
+ return nil, err
+ }
+ }
+ log.Infof("Completed deletion of data (%d rows) from table %s using query %q",
+ rowsDeleted, table, query)
+ }
+
+ return &tabletmanagerdatapb.DeleteTableDataResponse{}, nil
+}
+
func (tm *TabletManager) DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) {
if req == nil || req.Workflow == "" {
return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid request, no workflow provided")
@@ -438,13 +557,11 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta
source := row.AsBytes("source", []byte{})
state := row.AsString("state", "")
message := row.AsString("message", "")
- if req.State == binlogdatapb.VReplicationWorkflowState_Running && strings.ToUpper(message) == workflow.Frozen {
+ if req.State != nil && *req.State == binlogdatapb.VReplicationWorkflowState_Running &&
+ strings.ToUpper(message) == workflow.Frozen {
return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{Result: nil},
vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "cannot start a workflow when it is frozen")
}
- // For the string based values, we use NULL to differentiate
- // from an empty string. The NULL value indicates that we
- // should keep the existing value.
if !textutil.ValueIsSimulatedNull(req.Cells) {
cells = req.Cells
}
@@ -452,24 +569,27 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta
tabletTypes = req.TabletTypes
}
tabletTypesStr := topoproto.MakeStringTypeCSV(tabletTypes)
- if (inorder && req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN) ||
- (req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER) {
+ if req.TabletSelectionPreference != nil &&
+ ((inorder && *req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN) ||
+ (*req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER)) {
tabletTypesStr = discovery.InOrderHint + tabletTypesStr
}
if err = prototext.Unmarshal(source, bls); err != nil {
return nil, err
}
- // If we don't want to update the existing value then pass
- // the simulated NULL value of -1.
- if !textutil.ValueIsSimulatedNull(req.OnDdl) {
- bls.OnDdl = req.OnDdl
+ // We also need to check for a SimulatedNull here to support older clients and
+ // smooth upgrades. All non-slice simulated NULL checks can be removed in v22+.
+ if req.OnDdl != nil && *req.OnDdl != binlogdatapb.OnDDLAction(textutil.SimulatedNullInt) {
+ bls.OnDdl = *req.OnDdl
}
source, err = prototext.Marshal(bls)
if err != nil {
return nil, err
}
- if !textutil.ValueIsSimulatedNull(req.State) {
- state = binlogdatapb.VReplicationWorkflowState_name[int32(req.State)]
+ // We also need to check for a SimulatedNull here to support older clients and
+ // smooth upgrades. All non-slice simulated NULL checks can be removed in v22+.
+ if req.State != nil && *req.State != binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt) {
+ state = binlogdatapb.VReplicationWorkflowState_name[int32(*req.State)]
}
if state == binlogdatapb.VReplicationWorkflowState_Running.String() {
// `Workflow Start` sets the new state to Running. However, if stream is still copying tables, we should set
@@ -482,6 +602,8 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta
state = binlogdatapb.VReplicationWorkflowState_Copying.String()
}
}
+ options := getOptionSetString(req.ConfigOverrides)
+
bindVars = map[string]*querypb.BindVariable{
"st": sqltypes.StringBindVariable(state),
"sc": sqltypes.StringBindVariable(string(source)),
@@ -489,7 +611,7 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta
"tt": sqltypes.StringBindVariable(tabletTypesStr),
"id": sqltypes.Int64BindVariable(id),
}
- parsed = sqlparser.BuildParsedQuery(sqlUpdateVReplicationWorkflowStreamConfig, sidecar.GetIdentifier(), ":st", ":sc", ":cl", ":tt", ":id")
+ parsed = sqlparser.BuildParsedQuery(sqlUpdateVReplicationWorkflowStreamConfig, sidecar.GetIdentifier(), ":st", ":sc", ":cl", ":tt", options, ":id")
stmt, err = parsed.GenerateQuery(bindVars, nil)
if err != nil {
return nil, err
@@ -508,6 +630,51 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta
}, nil
}
+// getOptionSetString takes the option keys passed in and creates a sql clause to update the existing options
+// field in the vreplication table. The clause is built using the json_set() for new and updated options
+// and json_remove() for deleted options, denoted by an empty value.
+func getOptionSetString(config map[string]string) string {
+ if len(config) == 0 {
+ return ""
+ }
+
+ var (
+ options string
+ deletedKeys []string
+ keys []string
+ )
+ for k, v := range config {
+ if strings.TrimSpace(v) == "" {
+ deletedKeys = append(deletedKeys, k)
+ } else {
+ keys = append(keys, k)
+ }
+ }
+ sort.Strings(keys)
+ sort.Strings(deletedKeys)
+ clause := "options"
+ if len(deletedKeys) > 0 {
+ // We need to quote the key in the json functions because flag names can contain hyphens.
+ clause = fmt.Sprintf("json_remove(options, '$.config.\"%s\"'", deletedKeys[0])
+ for _, k := range deletedKeys[1:] {
+ clause += fmt.Sprintf(", '$.config.\"%s\"'", k)
+ }
+ clause += ")"
+ }
+ if len(keys) > 0 {
+ clause = fmt.Sprintf("json_set(%s, '$.config', json_object(), ", clause)
+ for i, k := range keys {
+ if i > 0 {
+ clause += ", "
+ }
+ clause += fmt.Sprintf("'$.config.\"%s\"', '%s'", k, strings.TrimSpace(config[k]))
+ }
+ clause += ")"
+ }
+ options = fmt.Sprintf(", options = %s", clause)
+ return options
+}
+
// UpdateVReplicationWorkflows operates in much the same way that
// UpdateVReplicationWorkflow does, but it allows you to update the
// metadata/flow control fields -- state, message, and stop_pos -- for
@@ -531,6 +698,42 @@ func (tm *TabletManager) UpdateVReplicationWorkflows(ctx context.Context, req *t
}, nil
}
+// ValidateVReplicationPermissions validates that the --db_filtered_user has
+// the minimum permissions required on the sidecardb vreplication table
+// needed in order to manage vreplication metadata.
+func (tm *TabletManager) ValidateVReplicationPermissions(ctx context.Context, req *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error) {
+ query, err := sqlparser.ParseAndBind(sqlValidateVReplicationPermissions,
+ sqltypes.StringBindVariable(tm.DBConfigs.Filtered.User),
+ sqltypes.StringBindVariable(sidecar.GetName()),
+ sqltypes.StringBindVariable(sidecar.GetName()),
+ )
+ if err != nil {
+ return nil, err
+ }
+ conn, err := tm.MysqlDaemon.GetAllPrivsConnection(ctx)
+ if err != nil {
+ return nil, err
+ }
+ defer conn.Close()
+ qr, err := conn.ExecuteFetch(query, 1, false)
+ if err != nil {
+ return nil, err
+ }
+ if len(qr.Rows) != 1 { // Should never happen
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected response to query %s: expected 1 row with 1 column, got: %+v",
+ query, qr)
+ }
+ val, err := qr.Rows[0][0].ToBool()
+ if err != nil { // Should never happen
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected result for query %s: expected boolean-like value, got: %q",
+ query, qr.Rows[0][0].ToString())
+ }
+ return &tabletmanagerdatapb.ValidateVReplicationPermissionsResponse{
+ User: tm.DBConfigs.Filtered.User,
+ Ok: val,
+ }, nil
+}
+
// VReplicationExec executes a vreplication command.
func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) {
// Replace any provided sidecar database qualifiers with the correct one.
@@ -623,14 +826,16 @@ func (tm *TabletManager) buildUpdateVReplicationWorkflowsQuery(req *tabletmanage
if req.GetAllWorkflows() && (len(req.GetIncludeWorkflows()) > 0 || len(req.GetExcludeWorkflows()) > 0) {
return "", errAllWithIncludeExcludeWorkflows
}
- if textutil.ValueIsSimulatedNull(req.GetState()) && textutil.ValueIsSimulatedNull(req.GetMessage()) && textutil.ValueIsSimulatedNull(req.GetStopPosition()) {
+ if req.State == nil && req.Message == nil && req.StopPosition == nil {
return "", errNoFieldsToUpdate
}
sets := strings.Builder{}
predicates := strings.Builder{}
// First add the SET clauses.
- if !textutil.ValueIsSimulatedNull(req.GetState()) {
+ // We also need to check for a SimulatedNull here to support older clients and
+ // smooth upgrades. All non-slice simulated NULL checks can be removed in v22+.
+ if req.State != nil && *req.State != binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt) {
state, ok := binlogdatapb.VReplicationWorkflowState_name[int32(req.GetState())]
if !ok {
return "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid state value: %v", req.GetState())
@@ -638,14 +843,18 @@ func (tm *TabletManager) buildUpdateVReplicationWorkflowsQuery(req *tabletmanage
sets.WriteString(" state = ")
sets.WriteString(sqltypes.EncodeStringSQL(state))
}
- if !textutil.ValueIsSimulatedNull(req.GetMessage()) {
+ // We also need to check for a SimulatedNull here to support older clients and
+ // smooth upgrades. All non-slice simulated NULL checks can be removed in v22+.
+ if req.Message != nil && *req.Message != sqltypes.Null.String() {
if sets.Len() > 0 {
sets.WriteByte(',')
}
sets.WriteString(" message = ")
sets.WriteString(sqltypes.EncodeStringSQL(req.GetMessage()))
}
- if !textutil.ValueIsSimulatedNull(req.GetStopPosition()) {
+ // We also need to check for a SimulatedNull here to support older clients and
+ // smooth upgrades. All non-slice simulated NULL checks can be removed in v22+.
+ if req.StopPosition != nil && *req.StopPosition != sqltypes.Null.String() {
if sets.Len() > 0 {
sets.WriteByte(',')
}
diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go
index 7ab959c1e17..3f8bc85ac7f 100644
--- a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go
+++ b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go
@@ -30,11 +30,13 @@ import (
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/constants/sidecar"
+ "vitess.io/vitess/go/ptr"
"vitess.io/vitess/go/sqlescape"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/dbconfigs"
+ "vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
@@ -42,7 +44,7 @@ import (
"vitess.io/vitess/go/vt/vtctl/workflow"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vtgate/vindexes"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -79,6 +81,7 @@ const (
readWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where workflow = '%s' and db_name = '%s'"
readWorkflowConfig = "select id, source, cell, tablet_types, state, message from _vt.vreplication where workflow = '%s'"
updateWorkflow = "update _vt.vreplication set state = '%s', source = '%s', cell = '%s', tablet_types = '%s' where id in (%d)"
+ getNonEmptyTableQuery = "select 1 from `%s` limit 1"
)
var (
@@ -94,8 +97,13 @@ var (
},
}
position = fmt.Sprintf("%s/%s", gtidFlavor, gtidPosition)
- setNetReadTimeout = fmt.Sprintf("set @@session.net_read_timeout = %v", vttablet.VReplicationNetReadTimeout)
- setNetWriteTimeout = fmt.Sprintf("set @@session.net_write_timeout = %v", vttablet.VReplicationNetWriteTimeout)
+ setNetReadTimeout = fmt.Sprintf("set @@session.net_read_timeout = %v", vttablet.GetVReplicationNetReadTimeout())
+ setNetWriteTimeout = fmt.Sprintf("set @@session.net_write_timeout = %v", vttablet.GetVReplicationNetWriteTimeout())
+ inOrder = tabletmanagerdatapb.TabletSelectionPreference_INORDER
+ running = binlogdatapb.VReplicationWorkflowState_Running
+ stopped = binlogdatapb.VReplicationWorkflowState_Stopped
+ exec = binlogdatapb.OnDDLAction_EXEC
+ execIgnore = binlogdatapb.OnDDLAction_EXEC_IGNORE
)
// TestCreateVReplicationWorkflow tests the query generated
@@ -278,9 +286,11 @@ func TestCreateVReplicationWorkflow(t *testing.T) {
// This is our expected query, which will also short circuit
// the test with an error as at this point we've tested what
// we wanted to test.
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ for _, table := range tt.schema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
targetTablet.vrdbClient.ExpectRequest(tt.query, &sqltypes.Result{}, errShortCircuit)
_, err := ws.MoveTablesCreate(ctx, tt.req)
tenv.tmc.tablets[targetTabletUID].vrdbClient.Wait()
@@ -289,12 +299,12 @@ func TestCreateVReplicationWorkflow(t *testing.T) {
}
}
-// TestMoveTables tests the query sequence originating from a
+// TestMoveTablesUnsharded tests the query sequence originating from a
// VtctldServer MoveTablesCreate request to ensure that the
// VReplication stream(s) are created correctly and expected
// results returned. Followed by ensuring that SwitchTraffic
// and ReverseTraffic also work as expected.
-func TestMoveTables(t *testing.T) {
+func TestMoveTablesUnsharded(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
sourceKs := "sourceks"
@@ -318,10 +328,8 @@ func TestMoveTables(t *testing.T) {
sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard)
defer tenv.deleteTablet(sourceTablet.tablet)
- targetShards["-80"] = tenv.addTablet(t, 300, targetKs, "-80")
- defer tenv.deleteTablet(targetShards["-80"].tablet)
- targetShards["80-"] = tenv.addTablet(t, 310, targetKs, "80-")
- defer tenv.deleteTablet(targetShards["80-"].tablet)
+ targetShards["0"] = tenv.addTablet(t, 300, targetKs, "0")
+ defer tenv.deleteTablet(targetShards["0"].tablet)
globalTablet := tenv.addTablet(t, 500, globalKs, globalShard)
defer tenv.deleteTablet(globalTablet.tablet)
@@ -387,39 +395,42 @@ func TestMoveTables(t *testing.T) {
tenv.tmc.SetSchema(defaultSchema)
tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, checkForJournal, &sqltypes.Result{})
-
for _, ftc := range targetShards {
+ log.Infof("Testing target shard %s", ftc.tablet.Alias)
addInvariants(ftc.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
getCopyStateQuery := fmt.Sprintf(sqlGetVReplicationCopyStatus, sidecar.GetIdentifier(), vreplID)
ftc.vrdbClient.AddInvariant(getCopyStateQuery, &sqltypes.Result{})
tenv.tmc.setVReplicationExecResults(ftc.tablet, getCopyState, &sqltypes.Result{})
ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- insert := fmt.Sprintf(`%s values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0, '{}')`,
- insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, ftc.tablet.Shard, tenv.cells[0], tenv.dbName, vreplID)
+ for _, table := range defaultSchema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
+ insert := fmt.Sprintf(`%s values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0, '{}')`,
+ insertVReplicationPrefix, wf, sourceKs, sourceShard, tenv.cells[0], tenv.dbName, vreplID)
ftc.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: 1}, nil)
ftc.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source",
- "int64|varchar",
+ "id|source|options",
+ "int64|varchar|varchar",
),
- fmt.Sprintf("%d|%s", vreplID, bls),
+ fmt.Sprintf("%d|%s|{}", vreplID, bls),
), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
- ), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
- ), nil)
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowConfig, wf), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
"id|source|cell|tablet_types|state|message",
@@ -432,31 +443,280 @@ func TestMoveTables(t *testing.T) {
ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source",
- "int64|varchar",
+ "id|source|options",
+ "int64|varchar|varchar",
),
- fmt.Sprintf("%d|%s", vreplID, bls),
+ fmt.Sprintf("%d|%s|{}", vreplID, bls),
), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
+ fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", wf, vreplID, bls, position, targetKs),
), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
+ tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getLatestCopyState, vreplID, vreplID), &sqltypes.Result{})
+ }
+
+ // We use the tablet's UID in the mocked results for the max value used on each target shard.
+ nextSeqVal := int(float64(targetShards["0"].tablet.Alias.Uid)) + 1
+ tenv.tmc.setVReplicationExecResults(globalTablet.tablet,
+ sqlparser.BuildParsedQuery(initSequenceTable, sqlescape.EscapeID(fmt.Sprintf("vt_%s", globalKs)), sqlescape.EscapeID("t1_seq"), nextSeqVal, nextSeqVal, nextSeqVal).Query,
+ &sqltypes.Result{RowsAffected: 0},
+ )
+
+ _, err = ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{
+ SourceKeyspace: sourceKs,
+ TargetKeyspace: targetKs,
+ Workflow: wf,
+ TabletTypes: tabletTypes,
+ Cells: tenv.cells,
+ AllTables: true,
+ AutoStart: true,
+ })
+ require.NoError(t, err)
+
+ for _, ftc := range targetShards {
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", wf, vreplID, bls, position, targetKs),
- ), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", wf, vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ }
+
+ _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{
+ Keyspace: targetKs,
+ Workflow: wf,
+ Cells: tenv.cells,
+ MaxReplicationLagAllowed: &vttimepb.Duration{Seconds: 922337203},
+ EnableReverseReplication: true,
+ InitializeTargetSequences: true,
+ Direction: int32(workflow.DirectionForward),
+ })
+ require.NoError(t, err)
+ for _, ftc := range targetShards {
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
"id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
"int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
),
fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
+ ))
+ }
+ addInvariants(sourceTablet.vrdbClient, vreplID, sourceTabletUID, position, workflow.ReverseWorkflowName(wf), tenv.cells[0])
+ sourceTablet.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, workflow.ReverseWorkflowName(wf), tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, sourceKs),
+ ))
+ sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, workflow.ReverseWorkflowName(wf)), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
+ "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ ),
+ fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", workflow.ReverseWorkflowName(wf), vreplID, bls, position, sourceKs),
+ ), nil)
+ sourceTablet.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, sourceKs),
+ ))
+
+ _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{
+ Keyspace: targetKs,
+ Workflow: wf,
+ Cells: tenv.cells,
+ MaxReplicationLagAllowed: &vttimepb.Duration{Seconds: 922337203},
+ EnableReverseReplication: true,
+ Direction: int32(workflow.DirectionBackward),
+ })
+ require.NoError(t, err)
+}
+
+// TestMoveTablesSharded tests the query sequence originating from a
+// VtctldServer MoveTablesCreate request to ensure that the
+// VReplication stream(s) are created correctly and expected
+// results returned. Followed by ensuring that SwitchTraffic
+// and ReverseTraffic also work as expected.
+func TestMoveTablesSharded(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ sourceKs := "sourceks"
+ sourceTabletUID := 200
+ targetKs := "targetks"
+ targetShards := make(map[string]*fakeTabletConn)
+ sourceShard := "0"
+ globalKs := "global"
+ globalShard := "0"
+ wf := "testwf"
+ vreplID := 1
+ tabletTypes := []topodatapb.TabletType{
+ topodatapb.TabletType_PRIMARY,
+ topodatapb.TabletType_REPLICA,
+ topodatapb.TabletType_RDONLY,
+ }
+
+ tenv := newTestEnv(t, ctx, sourceKs, []string{sourceShard})
+ defer tenv.close()
+
+ sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard)
+ defer tenv.deleteTablet(sourceTablet.tablet)
+
+ targetShards["-80"] = tenv.addTablet(t, 300, targetKs, "-80")
+ defer tenv.deleteTablet(targetShards["-80"].tablet)
+ targetShards["80-"] = tenv.addTablet(t, 310, targetKs, "80-")
+ defer tenv.deleteTablet(targetShards["80-"].tablet)
+
+ globalTablet := tenv.addTablet(t, 500, globalKs, globalShard)
+ defer tenv.deleteTablet(globalTablet.tablet)
+
+ err := tenv.ts.SaveVSchema(ctx, globalKs, &vschemapb.Keyspace{
+ Sharded: false,
+ Tables: map[string]*vschemapb.Table{
+ "t1_seq": {
+ Type: vindexes.TypeSequence,
+ },
+ },
+ })
+ require.NoError(t, err)
+ err = tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{
+ Sharded: true,
+ Vindexes: map[string]*vschemapb.Vindex{
+ "hash": {
+ Type: "hash",
+ },
+ },
+ Tables: map[string]*vschemapb.Table{
+ "t1": {
+ ColumnVindexes: []*vschemapb.ColumnVindex{{
+ Column: "id",
+ Name: "hash",
+ }},
+ AutoIncrement: &vschemapb.AutoIncrement{
+ Column: "id",
+ Sequence: "t1_seq",
+ },
+ },
+ },
+ })
+ require.NoError(t, err)
+
+ ws := workflow.NewServer(vtenv.NewTestEnv(), tenv.ts, tenv.tmc)
+
+ idQuery, err := sqlparser.ParseAndBind("select id from _vt.vreplication where id = %a",
+ sqltypes.Int64BindVariable(int64(vreplID)))
+ require.NoError(t, err)
+ idRes := sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id",
+ "int64",
+ ),
+ fmt.Sprintf("%d", vreplID),
+ )
+
+ tenv.mysqld.Schema = defaultSchema
+ tenv.mysqld.Schema.DatabaseSchema = tenv.dbName
+ tenv.mysqld.FetchSuperQueryMap = make(map[string]*sqltypes.Result)
+ tenv.mysqld.FetchSuperQueryMap[`select character_set_name, collation_name, column_name, data_type, column_type, extra from information_schema.columns where .*`] = sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "character_set_name|collation_name|column_name|data_type|column_type|extra",
+ "varchar|varchar|varchar|varchar|varchar|varchar",
+ ),
+ "NULL|NULL|id|bigint|bigint|",
+ "NULL|NULL|c2|bigint|bigint|",
+ )
+
+ bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"}}", sourceKs, sourceShard)
+
+ tenv.tmc.SetSchema(defaultSchema)
+
+ tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, checkForJournal, &sqltypes.Result{})
+ for _, ftc := range targetShards {
+ log.Infof("Testing target shard %s", ftc.tablet.Alias)
+ addInvariants(ftc.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
+ getCopyStateQuery := fmt.Sprintf(sqlGetVReplicationCopyStatus, sidecar.GetIdentifier(), vreplID)
+ ftc.vrdbClient.AddInvariant(getCopyStateQuery, &sqltypes.Result{})
+ tenv.tmc.setVReplicationExecResults(ftc.tablet, getCopyState, &sqltypes.Result{})
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range defaultSchema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
+ insert := fmt.Sprintf(`%s values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0, '{}')`,
+ insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, ftc.tablet.Shard, tenv.cells[0], tenv.dbName, vreplID)
+ ftc.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: 1}, nil)
+ ftc.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|options",
+ "int64|varchar|varchar",
+ ),
+ fmt.Sprintf("%d|%s|{}", vreplID, bls),
+ ), nil)
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowConfig, wf), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|cell|tablet_types|state|message",
+ "int64|blob|varchar|varchar|varchar|varchar",
+ ),
+ fmt.Sprintf("%d|%s|||Stopped|", vreplID, bls),
+ ), nil)
+ ftc.vrdbClient.ExpectRequest(idQuery, idRes, nil)
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(updateWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String(), bls, "", "", vreplID), &sqltypes.Result{}, nil)
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|options",
+ "int64|varchar|varchar",
+ ),
+ fmt.Sprintf("%d|%s|{}", vreplID, bls),
+ ), nil)
+ ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", wf, vreplID, bls, position, targetKs),
), nil)
tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getLatestCopyState, vreplID, vreplID), &sqltypes.Result{})
}
@@ -478,22 +738,35 @@ func TestMoveTables(t *testing.T) {
AutoStart: true,
})
require.NoError(t, err)
-
for _, ftc := range targetShards {
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", wf, vreplID, bls, position, targetKs),
- ), nil)
- ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
- ), nil)
+ fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", wf, vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
+ ftc.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
}
_, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{
@@ -506,7 +779,6 @@ func TestMoveTables(t *testing.T) {
Direction: int32(workflow.DirectionForward),
})
require.NoError(t, err)
-
for _, ftc := range targetShards {
ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
@@ -524,13 +796,13 @@ func TestMoveTables(t *testing.T) {
),
fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, sourceKs),
), nil)
- sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, workflow.ReverseWorkflowName(wf)), sqltypes.MakeTestResult(
+ sourceTablet.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, workflow.ReverseWorkflowName(wf)), sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
"workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
"workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
),
fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", workflow.ReverseWorkflowName(wf), vreplID, bls, position, sourceKs),
- ), nil)
+ ))
sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), &sqltypes.Result{}, nil)
_, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{
@@ -544,6 +816,51 @@ func TestMoveTables(t *testing.T) {
require.NoError(t, err)
}
+func TestGetOptionSetString(t *testing.T) {
+ tests := []struct {
+ name string
+ config map[string]string
+ want string
+ }{
+ {
+ name: "nil config",
+ config: nil,
+ want: "",
+ },
+ {
+ name: "empty params",
+ config: map[string]string{},
+ want: "",
+ },
+ {
+ name: "valid params",
+ config: map[string]string{
+ "password": "secret",
+ "user": "admin",
+ },
+ want: ", options = json_set(options, '$.config', json_object(), '$.config.\"password\"', 'secret', '$.config.\"user\"', 'admin')",
+ },
+ {
+ name: "valid params, deleting two",
+ config: map[string]string{
+ "password": "secret",
+ "user": "admin",
+ "port": "",
+ "host": "",
+ },
+ want: ", options = json_set(json_remove(options, '$.config.\"host\"', '$.config.\"port\"'), '$.config', json_object(), '$.config.\"password\"', 'secret', '$.config.\"user\"', 'admin')",
+ },
+ // Additional tests for handling escaping errors or complex scenarios can be added here
+ }
+
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ got := getOptionSetString(tt.config)
+ require.Equal(t, tt.want, got)
+ })
+ }
+}
+
func TestUpdateVReplicationWorkflow(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@@ -605,7 +922,6 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update cells",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
Cells: []string{"zone2"},
// TabletTypes is an empty value, so the current value should be cleared
},
@@ -616,9 +932,8 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update cells, NULL tablet_types",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
Cells: []string{"zone3"},
- TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)}, // So keep the current value of replica
+ TabletTypes: textutil.SimulatedNullTabletTypeSlice, // So keep the current value of replica
},
query: fmt.Sprintf(`update _vt.vreplication set state = 'Running', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}}', cell = '%s', tablet_types = '%s' where id in (%d)`,
keyspace, shard, "zone3", tabletTypes[0], vreplID),
@@ -627,8 +942,7 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update tablet_types",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
- TabletSelectionPreference: tabletmanagerdatapb.TabletSelectionPreference_INORDER,
+ TabletSelectionPreference: &inOrder,
TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_REPLICA},
},
query: fmt.Sprintf(`update _vt.vreplication set state = 'Running', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}}', cell = '', tablet_types = '%s' where id in (%d)`,
@@ -638,7 +952,6 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update tablet_types, NULL cells",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
Cells: textutil.SimulatedNullStringSlice, // So keep the current value of zone1
TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY},
},
@@ -649,8 +962,7 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update on_ddl",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
- OnDdl: binlogdatapb.OnDDLAction_EXEC,
+ OnDdl: &exec,
},
query: fmt.Sprintf(`update _vt.vreplication set state = 'Running', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}} on_ddl:%s', cell = '', tablet_types = '' where id in (%d)`,
keyspace, shard, binlogdatapb.OnDDLAction_EXEC.String(), vreplID),
@@ -659,10 +971,9 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update cell,tablet_types,on_ddl",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
Cells: []string{"zone1", "zone2", "zone3"},
TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_REPLICA, topodatapb.TabletType_PRIMARY},
- OnDdl: binlogdatapb.OnDDLAction_EXEC_IGNORE,
+ OnDdl: &execIgnore,
},
query: fmt.Sprintf(`update _vt.vreplication set state = 'Running', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}} on_ddl:%s', cell = '%s', tablet_types = '%s' where id in (%d)`,
keyspace, shard, binlogdatapb.OnDDLAction_EXEC_IGNORE.String(), "zone1,zone2,zone3", "rdonly,replica,primary", vreplID),
@@ -671,10 +982,9 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update state",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState_Stopped,
+ State: &stopped,
Cells: textutil.SimulatedNullStringSlice,
- TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)},
- OnDdl: binlogdatapb.OnDDLAction(textutil.SimulatedNullInt),
+ TabletTypes: textutil.SimulatedNullTabletTypeSlice,
},
query: fmt.Sprintf(`update _vt.vreplication set state = '%s', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}}', cell = '%s', tablet_types = '%s' where id in (%d)`,
binlogdatapb.VReplicationWorkflowState_Stopped.String(), keyspace, shard, cells[0], tabletTypes[0], vreplID),
@@ -683,15 +993,27 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
name: "update to running while copying",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
Workflow: workflow,
- State: binlogdatapb.VReplicationWorkflowState_Running,
+ State: &running,
Cells: textutil.SimulatedNullStringSlice,
- TabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)},
- OnDdl: binlogdatapb.OnDDLAction(textutil.SimulatedNullInt),
+ TabletTypes: textutil.SimulatedNullTabletTypeSlice,
},
isCopying: true,
query: fmt.Sprintf(`update _vt.vreplication set state = 'Copying', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}}', cell = '%s', tablet_types = '%s' where id in (%d)`,
keyspace, shard, cells[0], tabletTypes[0], vreplID),
},
+ {
+ name: "update cells and options",
+ request: &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{
+ Workflow: workflow,
+ Cells: []string{"zone2"},
+ ConfigOverrides: map[string]string{
+ "user": "admin",
+ "password": "secret",
+ },
+ },
+ query: fmt.Sprintf(`update _vt.vreplication set state = 'Running', source = 'keyspace:"%s" shard:"%s" filter:{rules:{match:"corder" filter:"select * from corder"} rules:{match:"customer" filter:"select * from customer"}}', cell = '%s', tablet_types = '', options = json_set(options, '$.config', json_object(), '$.config."password"', 'secret', '$.config."user"', 'admin') where id in (%d)`,
+ keyspace, shard, "zone2", vreplID),
+ },
}
for _, tt := range tests {
@@ -700,7 +1022,9 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
// which doesn't play well with subtests.
defer func() {
if err := recover(); err != nil {
- t.Errorf("Recovered from panic: %v", err)
+ log.Infof("Got panic in test: %v", err)
+ log.Flush()
+ t.Errorf("Recovered from panic: %v, stack: %s", err, debug.Stack())
}
}()
@@ -710,8 +1034,7 @@ func TestUpdateVReplicationWorkflow(t *testing.T) {
// These are the same for each RPC call.
tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(selectQuery, selectRes, nil)
- if tt.request.State == binlogdatapb.VReplicationWorkflowState_Running ||
- tt.request.State == binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt) {
+ if tt.request.State == nil || *tt.request.State == binlogdatapb.VReplicationWorkflowState_Running {
tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
if tt.isCopying {
tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(getCopyStateQuery, copying, nil)
@@ -756,9 +1079,7 @@ func TestUpdateVReplicationWorkflows(t *testing.T) {
name: "update only state=running for all workflows",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
AllWorkflows: true,
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: textutil.SimulatedNullString,
- StopPosition: textutil.SimulatedNullString,
+ State: &running,
},
query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")),
},
@@ -766,9 +1087,7 @@ func TestUpdateVReplicationWorkflows(t *testing.T) {
name: "update only state=running for all but reverse workflows",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
ExcludeWorkflows: []string{workflow.ReverseWorkflowName("testwf")},
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: textutil.SimulatedNullString,
- StopPosition: textutil.SimulatedNullString,
+ State: &running,
},
query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")),
},
@@ -776,9 +1095,9 @@ func TestUpdateVReplicationWorkflows(t *testing.T) {
name: "update all vals for all workflows",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
AllWorkflows: true,
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: "hi",
- StopPosition: position,
+ State: &running,
+ Message: ptr.Of("hi"),
+ StopPosition: &position,
},
query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running', message = 'hi', stop_pos = '%s' where id in (%s)`, position, strings.Join(vreplIDs, ", ")),
},
@@ -786,9 +1105,7 @@ func TestUpdateVReplicationWorkflows(t *testing.T) {
name: "update state=stopped, messege=for vdiff for two workflows",
request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
IncludeWorkflows: []string{"testwf", "testwf2"},
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: textutil.SimulatedNullString,
- StopPosition: textutil.SimulatedNullString,
+ State: &running,
},
query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")),
},
@@ -1020,9 +1337,12 @@ func TestSourceShardSelection(t *testing.T) {
}
for uid, streams := range tt.streams {
- tt := targetTablets[uid]
- tt.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
- tt.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ targetTablet := targetTablets[uid]
+ targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range tt.schema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
for i, sourceShard := range streams {
var err error
if i == len(streams)-1 {
@@ -1031,25 +1351,26 @@ func TestSourceShardSelection(t *testing.T) {
// everything we wanted to in the test.
err = errShortCircuit
}
- tt.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
- tt.vrdbClient.ExpectRequest(
+ targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.ExpectRequest(
fmt.Sprintf(`%s values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0, '{}')`,
- insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, tt.tablet.Shard, tenv.cells[0], tenv.dbName),
+ insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, targetTablet.tablet.Shard, tenv.cells[0], tenv.dbName),
&sqltypes.Result{InsertID: uint64(i + 1)},
err,
)
if errors.Is(err, errShortCircuit) {
break
}
- tt.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
- tt.vrdbClient.ExpectRequest(
+ targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.ExpectRequest(
fmt.Sprintf("select * from _vt.vreplication where id = %d", uint64(i+1)),
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|state",
- "int64|varchar|varchar",
+ "id|source|state|options",
+ "int64|varchar|varchar|varchar",
),
- fmt.Sprintf("%d|%s|Stopped", uint64(i+1), fmt.Sprintf(`keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(id, '%s.hash', '%s')"}}`, sourceKs, sourceShard, targetKs, tt.tablet.Shard)),
+ fmt.Sprintf("%d|%s|Stopped|{}", uint64(i+1), fmt.Sprintf(`keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(id, '%s.hash', '%s')"}}`,
+ sourceKs, sourceShard, targetKs, targetTablet.tablet.Shard)),
),
nil,
)
@@ -1112,8 +1433,10 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) {
require.NoError(t, err, "failed to save routing rules")
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
-
tenv.tmc.tablets[targetTabletUID].vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range defaultSchema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
targetTablet.vrdbClient.ExpectRequest(
fmt.Sprintf("%s %s",
insertVReplicationPrefix,
@@ -1130,22 +1453,28 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) {
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source",
- "int64|varchar",
+ "id|source|options",
+ "int64|varchar|varchar",
),
- fmt.Sprintf("%d|%s", vreplID, bls),
+ fmt.Sprintf("%d|%s|{}", vreplID, bls),
),
nil,
)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult(
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys",
- "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64",
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
),
- fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs),
- ), nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(insertStreamsCreatedLog, bls), &sqltypes.Result{}, nil)
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
tenv.tmc.setVReplicationExecResults(targetTablet.tablet,
fmt.Sprintf("select convert_tz('2006-01-02 15:04:05', '%s', 'UTC')", invalidTimeZone),
@@ -1172,7 +1501,6 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) {
// Save the current target vschema.
vs, err := tenv.ts.GetVSchema(ctx, targetKs)
require.NoError(t, err, "failed to get target vschema")
-
_, err = ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{
Workflow: wf,
SourceKeyspace: sourceKs,
@@ -1394,8 +1722,11 @@ func TestReadVReplicationWorkflows(t *testing.T) {
require.NotNil(t, tt.req, "No request provided")
if !tt.wantErr { // Errors we're testing for occur before executing any queries.
- tablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ tablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
tablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, tt.wantPreds), &sqltypes.Result{}, nil)
+ for _, table := range defaultSchema.TableDefinitions {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.Name), &sqltypes.Result{})
+ }
}
_, err := tenv.tmc.ReadVReplicationWorkflows(ctx, tablet.tablet, tt.req)
@@ -1451,6 +1782,7 @@ func addInvariants(dbClient *binlogplayer.MockDBClient, vreplID, sourceTabletUID
"0",
))
dbClient.AddInvariant(fmt.Sprintf(updatePickedSourceTablet, cell, sourceTabletUID, vreplID), &sqltypes.Result{})
+ dbClient.AddInvariant("update _vt.vreplication set state='Running', message='' where id=1", &sqltypes.Result{})
}
func addMaterializeSettingsTablesToSchema(ms *vtctldatapb.MaterializeSettings, tenv *testEnv, venv *vtenv.Environment) {
@@ -1560,22 +1892,22 @@ func TestExternalizeLookupVindex(t *testing.T) {
trxTS := fmt.Sprintf("%d", time.Now().Unix())
fields := sqltypes.MakeTestFields(
- "id|state|message|source|workflow_type|workflow_sub_type|max_tps|max_replication_lag|time_updated|time_heartbeat|time_throttled|transaction_timestamp|rows_copied",
- "int64|varbinary|varbinary|blob|int64|int64|int64|int64|int64|int64|int64|int64|int64",
+ "id|state|message|source|workflow_type|workflow_sub_type|max_tps|max_replication_lag|time_updated|time_heartbeat|time_throttled|transaction_timestamp|rows_copied|options",
+ "int64|varbinary|varbinary|blob|int64|int64|int64|int64|int64|int64|int64|int64|int64|varchar",
)
wftype := fmt.Sprintf("%d", binlogdatapb.VReplicationWorkflowType_CreateLookupIndex)
ownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`,
ms.SourceKeyspace, ms.SourceKeyspace)
ownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`,
ms.SourceKeyspace, ms.SourceKeyspace)
- ownedRunning := sqltypes.MakeTestResult(fields, "1|Running|msg|"+ownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5")
- ownedStopped := sqltypes.MakeTestResult(fields, "1|Stopped|Stopped after copy|"+ownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5")
+ ownedRunning := sqltypes.MakeTestResult(fields, "1|Running|msg|"+ownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5|{}")
+ ownedStopped := sqltypes.MakeTestResult(fields, "1|Stopped|Stopped after copy|"+ownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5|{}")
unownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`,
ms.SourceKeyspace, ms.SourceKeyspace)
unownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`,
ms.SourceKeyspace, ms.SourceKeyspace)
- unownedRunning := sqltypes.MakeTestResult(fields, "2|Running|msg|"+unownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5")
- unownedStopped := sqltypes.MakeTestResult(fields, "2|Stopped|Stopped after copy|"+unownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5")
+ unownedRunning := sqltypes.MakeTestResult(fields, "2|Running|msg|"+unownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5|{}")
+ unownedStopped := sqltypes.MakeTestResult(fields, "2|Stopped|Stopped after copy|"+unownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5|{}")
testcases := []struct {
request *vtctldatapb.LookupVindexExternalizeRequest
@@ -1785,9 +2117,11 @@ func TestMaterializerOneToOne(t *testing.T) {
addMaterializeSettingsTablesToSchema(ms, tenv, vtenv)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
// This is our expected query, which will also short circuit
// the test with an error as at this point we've tested what
@@ -1849,6 +2183,9 @@ func TestMaterializerManyToOne(t *testing.T) {
addMaterializeSettingsTablesToSchema(ms, tenv, vtenv)
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
// This is our expected query, which will also short circuit
// the test with an error as at this point we've tested what
@@ -1861,6 +2198,13 @@ func TestMaterializerManyToOne(t *testing.T) {
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1"} rules:{match:"t2" filter:"select * from t3"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, tenv.cells[0], tenv.dbName)
if vreplID == 1 {
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil)
targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID),
@@ -1950,12 +2294,22 @@ func TestMaterializerOneToMany(t *testing.T) {
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}",
sourceKs, sourceShard, targetKs, targetShard)
insert := insertVReplicationPrefix +
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName)
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
if targetShard == "-80" {
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil)
targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
@@ -2048,14 +2402,25 @@ func TestMaterializerManyToMany(t *testing.T) {
targetTablet := targetShards[targetShard]
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
for i, sourceShard := range []string{"-40", "40-"} { // One insert per [binlog]source/stream
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID+(i*10), position, wf, tenv.cells[0])
bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}",
sourceKs, sourceShard, targetKs, targetShard)
+
insert := insertVReplicationPrefix +
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName)
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
if targetShard == "80-" && sourceShard == "40-" { // Last insert
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit)
} else { // Can't short circuit as we will do more inserts
@@ -2151,12 +2516,22 @@ func TestMaterializerMulticolumnVindex(t *testing.T) {
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}",
sourceKs, sourceShard, targetKs, targetShard)
insert := insertVReplicationPrefix +
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select * from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName)
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
if targetShard == "-80" {
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil)
targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
@@ -2230,9 +2605,11 @@ func TestMaterializerDeploySchema(t *testing.T) {
}
}
tenv.tmc.tabletSchemas[targetTabletUID] = schema
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
tenv.tmc.setVReplicationExecResults(targetTablet.tablet, `t2ddl`, &sqltypes.Result{}) // Execute the fake CreateDdl
// This is our expected query, which will also short circuit
@@ -2301,9 +2678,11 @@ func TestMaterializerCopySchema(t *testing.T) {
}
}
tenv.tmc.tabletSchemas[targetTabletUID] = schema
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
// This is our expected query, which will also short circuit
// the test with an error as at this point we've tested what
@@ -2391,12 +2770,22 @@ func TestMaterializerExplicitColumns(t *testing.T) {
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}",
sourceKs, sourceShard, targetKs, targetShard)
insert := insertVReplicationPrefix +
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName)
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
if targetShard == "-80" {
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil)
targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
@@ -2491,12 +2880,22 @@ func TestMaterializerRenamedColumns(t *testing.T) {
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ...
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, '%s.region', '%s')\"}}",
sourceKs, sourceShard, targetKs, targetShard)
insert := insertVReplicationPrefix +
fmt.Sprintf(` values ('%s', 'keyspace:"%s" shard:"%s" filter:{rules:{match:"t1" filter:"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, \'%s.region\', \'%s\')"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0, '{}')`,
wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName)
+ targetTablet.vrdbClient.AddInvariant(binlogplayer.TestGetWorkflowQueryId1, sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys|options",
+ "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64|varchar",
+ ),
+ fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1|{}", vreplID, bls, position, targetKs),
+ ))
if targetShard == "-80" {
targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil)
targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil)
@@ -2562,9 +2961,11 @@ func TestMaterializerStopAfterCopy(t *testing.T) {
addMaterializeSettingsTablesToSchema(ms, tenv, vtenv)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ targetTablet.vrdbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
- targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
// This is our expected query, which will also short circuit
// the test with an error as at this point we've tested what
@@ -2623,6 +3024,9 @@ func TestMaterializerNoTargetVSchema(t *testing.T) {
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err = ws.Materialize(ctx, ms)
targetTablet.vrdbClient.Wait()
@@ -2668,6 +3072,9 @@ func TestMaterializerNoDDL(t *testing.T) {
tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{}
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined")
@@ -2765,6 +3172,9 @@ func TestMaterializerTableMismatchNonCopy(t *testing.T) {
tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{}
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined")
@@ -2811,6 +3221,9 @@ func TestMaterializerTableMismatchCopy(t *testing.T) {
tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{}
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "source and target table names must match for copying schema: t2 vs t1")
@@ -2853,6 +3266,9 @@ func TestMaterializerNoSourceTable(t *testing.T) {
tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{}
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "source table t1 does not exist")
@@ -2901,6 +3317,9 @@ func TestMaterializerSyntaxError(t *testing.T) {
tenv.tmc.setVReplicationExecResults(targetTablet.tablet, ms.TableSettings[0].CreateDdl, &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "syntax error at position 4 near 'bad'")
@@ -2949,6 +3368,9 @@ func TestMaterializerNotASelect(t *testing.T) {
tenv.tmc.setVReplicationExecResults(targetTablet.tablet, ms.TableSettings[0].CreateDdl, &sqltypes.Result{})
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil)
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
err := ws.Materialize(ctx, ms)
require.EqualError(t, err, "unrecognized statement: update t1 set val=1")
@@ -3027,6 +3449,9 @@ func TestMaterializerNoGoodVindex(t *testing.T) {
targetTablet := targetShards[targetShard]
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
errs = append(errs, errNoVindex)
}
@@ -3102,6 +3527,9 @@ func TestMaterializerComplexVindexExpression(t *testing.T) {
targetTablet := targetShards[targetShard]
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
errs = append(errs, errNoVindex)
}
@@ -3177,6 +3605,9 @@ func TestMaterializerNoVindexInExpression(t *testing.T) {
targetTablet := targetShards[targetShard]
addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0])
targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil)
+ for _, table := range ms.TableSettings {
+ tenv.db.AddQuery(fmt.Sprintf(getNonEmptyTableQuery, table.TargetTable), &sqltypes.Result{})
+ }
errs = append(errs, errNoVindex)
}
@@ -3235,6 +3666,12 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
DBName: "vt_testks",
},
}
+ forVdiff := "for vdiff"
+ forPos := "for until position"
+ forTest := "test message"
+ stopPos1 := "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-20"
+ stopPos2 := "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-9999"
+
tests := []struct {
name string
req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest
@@ -3242,18 +3679,14 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
wantErr string
}{
{
- name: "nothing to update",
- req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt),
- Message: textutil.SimulatedNullString,
- StopPosition: textutil.SimulatedNullString,
- },
+ name: "nothing to update",
+ req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{},
wantErr: errNoFieldsToUpdate.Error(),
},
{
name: "mutually exclusive options",
req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState_Running,
+ State: &running,
AllWorkflows: true,
ExcludeWorkflows: []string{"wf1"},
},
@@ -3262,9 +3695,9 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
{
name: "all values and options",
req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: "test message",
- StopPosition: "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-20",
+ State: &running,
+ Message: &forTest,
+ StopPosition: &stopPos1,
IncludeWorkflows: []string{"wf2", "wf3"},
ExcludeWorkflows: []string{"1wf"},
},
@@ -3273,9 +3706,7 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
{
name: "state for all",
req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: textutil.SimulatedNullString,
- StopPosition: textutil.SimulatedNullString,
+ State: &running,
AllWorkflows: true,
},
want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where db_name = 'vt_testks'",
@@ -3283,9 +3714,8 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
{
name: "stop all for vdiff",
req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState_Stopped,
- Message: "for vdiff",
- StopPosition: textutil.SimulatedNullString,
+ State: &stopped,
+ Message: &forVdiff,
AllWorkflows: true,
},
want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Stopped', message = 'for vdiff' where db_name = 'vt_testks'",
@@ -3293,9 +3723,9 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
{
name: "start one until position",
req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{
- State: binlogdatapb.VReplicationWorkflowState_Running,
- Message: "for until position",
- StopPosition: "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-9999",
+ State: &running,
+ Message: &forPos,
+ StopPosition: &stopPos2,
IncludeWorkflows: []string{"wf1"},
},
want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running', message = 'for until position', stop_pos = 'MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-9999' where db_name = 'vt_testks' and workflow in ('wf1')",
@@ -3313,3 +3743,81 @@ func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) {
})
}
}
+
+func TestDeleteTableData(t *testing.T) {
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
+ defer cancel()
+ sourceKs := "sourceks"
+ sourceShard := "0"
+ sourceTabletUID := 200
+ tenv := newTestEnv(t, ctx, sourceKs, []string{shard})
+ defer tenv.close()
+
+ tablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard)
+ defer tenv.deleteTablet(tablet.tablet)
+
+ testCases := []struct {
+ name string
+ req *tabletmanagerdatapb.DeleteTableDataRequest
+ workflowType *binlogdatapb.VReplicationWorkflowType
+ expectedQueries []string
+ wantErr string
+ }{
+ {
+ name: "no request",
+ req: nil,
+ wantErr: "invalid nil request",
+ },
+ {
+ name: "one table",
+ req: &tabletmanagerdatapb.DeleteTableDataRequest{
+ TableFilters: map[string]string{
+ "t1": "where tenant_id = 1",
+ },
+ BatchSize: 100,
+ },
+ expectedQueries: []string{
+ "delete from t1 where tenant_id = 1 limit 100",
+ },
+ },
+ {
+ name: "one table without batch size",
+ req: &tabletmanagerdatapb.DeleteTableDataRequest{
+ TableFilters: map[string]string{
+ "t1": "where tenant_id = 1",
+ },
+ },
+ expectedQueries: []string{
+ "delete from t1 where tenant_id = 1 limit 1000", // Default batch size of 1,000
+ },
+ },
+ {
+ name: "multiple tables",
+ req: &tabletmanagerdatapb.DeleteTableDataRequest{
+ TableFilters: map[string]string{
+ "t1": "where tenant_id = 1",
+ "t2": "where foo = 2",
+ },
+ BatchSize: 500,
+ },
+ expectedQueries: []string{
+ "delete from t1 where tenant_id = 1 limit 500",
+ "delete from t2 where foo = 2 limit 500",
+ },
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ tenv.db.AddQuery(fmt.Sprintf("use `%s`", tenv.dbName), &sqltypes.Result{})
+ for _, query := range tc.expectedQueries {
+ tenv.db.AddQuery(query, &sqltypes.Result{})
+ }
+ _, err := tenv.tmc.DeleteTableData(ctx, tablet.tablet, tc.req)
+ if tc.wantErr != "" {
+ require.EqualError(t, err, tc.wantErr)
+ } else {
+ require.NoError(t, err)
+ }
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletmanager/shard_sync_test.go b/go/vt/vttablet/tabletmanager/shard_sync_test.go
index 24078efa977..8f7739e5341 100644
--- a/go/vt/vttablet/tabletmanager/shard_sync_test.go
+++ b/go/vt/vttablet/tabletmanager/shard_sync_test.go
@@ -18,6 +18,7 @@ package tabletmanager
import (
"context"
+ "errors"
"fmt"
"reflect"
"testing"
@@ -44,11 +45,11 @@ const (
)
func TestShardSync(t *testing.T) {
- ctx, cancel := context.WithCancel(context.Background())
+ ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
statsTabletTypeCount.ResetAll()
- tm := newTestTM(t, ts, 100, keyspace, shard)
+ tm := newTestTM(t, ts, 100, keyspace, shard, nil)
defer tm.Stop()
// update the primary info in the shard record and set it to nil
@@ -67,48 +68,77 @@ func TestShardSync(t *testing.T) {
// wait for syncing to work correctly
// this should also have updated the shard record since it is a more recent operation
// We check here that the shard record and the tablet record are in sync
- checkShardRecordWithTimeout(ctx, t, ts, ti.Alias, ti.PrimaryTermStartTime, 1*time.Second)
+ err = checkShardRecord(ctx, t, ts, ti.Alias, ti.PrimaryTermStartTime)
+ require.NoError(t, err)
+
+ // Shard sync loop runs asynchronously and starts a watch on the shard.
+ // We wait for the shard watch to start, otherwise the test is flaky
+ // because the update of the record can happen before the watch is started.
+ waitForShardWatchToStart(ctx, t, tm, originalTime, ti)
// even if try to update the shard record with the old timestamp, it should be reverted again
updatePrimaryInfoInShardRecord(ctx, t, tm, nil, originalTime)
// this should have also updated the shard record because of the timestamp.
- checkShardRecordWithTimeout(ctx, t, ts, ti.Alias, ti.PrimaryTermStartTime, 1*time.Second)
+ err = checkShardRecord(ctx, t, ts, ti.Alias, ti.PrimaryTermStartTime)
+ require.NoError(t, err)
// updating the shard record with the latest time should trigger an update in the tablet
newTime := time.Now()
updatePrimaryInfoInShardRecord(ctx, t, tm, nil, newTime)
// this should not have updated.
- checkShardRecordWithTimeout(ctx, t, ts, nil, protoutil.TimeToProto(newTime), 1*time.Second)
+ err = checkShardRecord(ctx, t, ts, nil, protoutil.TimeToProto(newTime))
+ require.NoError(t, err)
// verify that the tablet record has been updated
- checkTabletRecordWithTimeout(ctx, t, ts, tm.tabletAlias, topodata.TabletType_REPLICA, nil, 1*time.Second)
+ checkTabletRecordWithTimeout(ctx, t, ts, tm.tabletAlias, topodata.TabletType_REPLICA, nil)
+}
+
+// waitForShardWatchToStart waits for shard watch to have started.
+func waitForShardWatchToStart(ctx context.Context, t *testing.T, tm *TabletManager, originalTime time.Time, ti *topo.TabletInfo) {
+ // We wait for shard watch to start by
+ // updating the record and waiting to see
+ // the shard record is updated back by the tablet manager.
+ idx := 1
+ for {
+ select {
+ case <-ctx.Done():
+ require.FailNow(t, "timed out: waiting for shard watch to start")
+ default:
+ updatePrimaryInfoInShardRecord(ctx, t, tm, nil, originalTime.Add(-1*time.Duration(idx)*time.Second))
+ idx = idx + 1
+ checkCtx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
+ err := checkShardRecord(checkCtx, t, tm.TopoServer, ti.Alias, ti.PrimaryTermStartTime)
+ cancel()
+ if err == nil {
+ return
+ }
+ }
+ }
}
-func checkShardRecordWithTimeout(ctx context.Context, t *testing.T, ts *topo.Server, tabletAlias *topodata.TabletAlias, expectedStartTime *vttime.Time, timeToWait time.Duration) {
- timeOut := time.After(timeToWait)
+func checkShardRecord(ctx context.Context, t *testing.T, ts *topo.Server, tabletAlias *topodata.TabletAlias, expectedStartTime *vttime.Time) error {
for {
select {
- case <-timeOut:
- t.Fatalf("timed out: waiting for shard record to update")
+ case <-ctx.Done():
+ return errors.New("timed out: waiting for shard record to update")
default:
si, err := ts.GetShard(ctx, keyspace, shard)
require.NoError(t, err)
if reflect.DeepEqual(tabletAlias, si.PrimaryAlias) && reflect.DeepEqual(expectedStartTime, si.PrimaryTermStartTime) {
- return
+ return nil
}
time.Sleep(100 * time.Millisecond)
}
}
}
-func checkTabletRecordWithTimeout(ctx context.Context, t *testing.T, ts *topo.Server, tabletAlias *topodata.TabletAlias, tabletType topodata.TabletType, expectedStartTime *vttime.Time, timeToWait time.Duration) {
- timeOut := time.After(timeToWait)
+func checkTabletRecordWithTimeout(ctx context.Context, t *testing.T, ts *topo.Server, tabletAlias *topodata.TabletAlias, tabletType topodata.TabletType, expectedStartTime *vttime.Time) {
for {
select {
- case <-timeOut:
- t.Fatalf("timed out: waiting for tablet record to update")
+ case <-ctx.Done():
+ require.FailNow(t, "timed out: waiting for tablet record to update")
default:
ti, err := ts.GetTablet(ctx, tabletAlias)
require.NoError(t, err)
diff --git a/go/vt/vttablet/tabletmanager/tm_init.go b/go/vt/vttablet/tabletmanager/tm_init.go
index 6046ed99727..84150c82be8 100644
--- a/go/vt/vttablet/tabletmanager/tm_init.go
+++ b/go/vt/vttablet/tabletmanager/tm_init.go
@@ -50,6 +50,8 @@ import (
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/flagutil"
"vitess.io/vitess/go/mysql/collations"
+ "vitess.io/vitess/go/mysql/replication"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/netutil"
"vitess.io/vitess/go/protoutil"
"vitess.io/vitess/go/sets"
@@ -63,6 +65,7 @@ import (
"vitess.io/vitess/go/vt/mysqlctl"
querypb "vitess.io/vitess/go/vt/proto/query"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
@@ -74,6 +77,7 @@ import (
"vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
"vitess.io/vitess/go/vt/vttablet/tabletserver"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
+ "vitess.io/vitess/go/vt/vttablet/tmclient"
)
const (
@@ -122,6 +126,9 @@ var (
// statsIsInSrvKeyspace is set to 1 (true), 0 (false) whether the tablet is in the serving keyspace
statsIsInSrvKeyspace *stats.Gauge
+ // statsTabletTags is set to 1 (true) if a tablet tag exists.
+ statsTabletTags *stats.GaugesWithMultiLabels
+
statsKeyspace = stats.NewString("TabletKeyspace")
statsShard = stats.NewString("TabletShard")
statsKeyRangeStart = stats.NewString("TabletKeyRangeStart")
@@ -141,6 +148,7 @@ func init() {
statsTabletTypeCount = stats.NewCountersWithSingleLabel("TabletTypeCount", "Number of times the tablet changed to the labeled type", "type")
statsBackupIsRunning = stats.NewGaugesWithMultiLabels("BackupIsRunning", "Whether a backup is running", []string{"mode"})
statsIsInSrvKeyspace = stats.NewGauge("IsInSrvKeyspace", "Whether the vttablet is in the serving keyspace (1 = true / 0 = false)")
+ statsTabletTags = stats.NewGaugesWithMultiLabels("TabletTags", "Tablet tags key/values", []string{"key", "value"})
}
// TabletManager is the main class for the tablet manager.
@@ -157,6 +165,9 @@ type TabletManager struct {
VDiffEngine *vdiff.Engine
Env *vtenv.Environment
+ // tmc is used to run an RPC against other vttablets.
+ tmc tmclient.TabletManagerClient
+
// tmState manages the TabletManager state.
tmState *tmState
@@ -343,6 +354,13 @@ func mergeTags(a, b map[string]string) map[string]string {
return result
}
+func setTabletTagsStats(tablet *topodatapb.Tablet) {
+ statsTabletTags.ResetAll()
+ for key, val := range tablet.Tags {
+ statsTabletTags.Set([]string{key, val}, 1)
+ }
+}
+
// Start starts the TabletManager.
func (tm *TabletManager) Start(tablet *topodatapb.Tablet, config *tabletenv.TabletConfig) error {
defer func() {
@@ -351,6 +369,7 @@ func (tm *TabletManager) Start(tablet *topodatapb.Tablet, config *tabletenv.Tabl
log.Infof("TabletManager Start")
tm.DBConfigs.DBName = topoproto.TabletDbName(tablet)
tm.tabletAlias = tablet.Alias
+ tm.tmc = tmclient.NewTabletManagerClient()
tm.tmState = newTMState(tm, tablet)
tm.actionSema = semaphore.NewWeighted(1)
tm._waitForGrantsComplete = make(chan struct{})
@@ -544,13 +563,17 @@ func (tm *TabletManager) createKeyspaceShard(ctx context.Context) (*topo.ShardIn
return nil, err
}
- tm.tmState.RefreshFromTopoInfo(ctx, shardInfo, nil)
+ if err := tm.tmState.RefreshFromTopoInfo(ctx, shardInfo, nil); err != nil {
+ return nil, err
+ }
// Rebuild keyspace if this the first tablet in this keyspace/cell
srvKeyspace, err := tm.TopoServer.GetSrvKeyspace(ctx, tm.tabletAlias.Cell, tablet.Keyspace)
switch {
case err == nil:
- tm.tmState.RefreshFromTopoInfo(ctx, nil, srvKeyspace)
+ if err := tm.tmState.RefreshFromTopoInfo(ctx, nil, srvKeyspace); err != nil {
+ return nil, err
+ }
case topo.IsErrType(err, topo.NoNode):
var rebuildKsCtx context.Context
rebuildKsCtx, tm._rebuildKeyspaceCancel = context.WithCancel(tm.BatchCtx)
@@ -602,7 +625,10 @@ func (tm *TabletManager) rebuildKeyspace(ctx context.Context, done chan<- struct
defer func() {
log.Infof("Keyspace rebuilt: %v", keyspace)
if ctx.Err() == nil {
- tm.tmState.RefreshFromTopoInfo(tm.BatchCtx, nil, srvKeyspace)
+ err := tm.tmState.RefreshFromTopoInfo(tm.BatchCtx, nil, srvKeyspace)
+ if err != nil {
+ log.Errorf("Error refreshing topo information - %v", err)
+ }
}
close(done)
}()
@@ -751,6 +777,24 @@ func (tm *TabletManager) findMysqlPort(retryInterval time.Duration) {
}
}
+// redoPreparedTransactionsAndSetReadWrite redoes prepared transactions in read-only mode.
+// We turn off super read only mode, and then redo the transactions. Finally, we turn off read-only mode to allow for further traffic.
+func (tm *TabletManager) redoPreparedTransactionsAndSetReadWrite(ctx context.Context) error {
+ _, err := tm.MysqlDaemon.SetSuperReadOnly(ctx, false)
+ if err != nil {
+ // Ignore the error if the sever doesn't support super read only variable.
+ // We should just redo the preapred transactions before we set it to read-write.
+ if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERUnknownSystemVariable {
+ log.Warningf("server does not know about super_read_only, continuing anyway...")
+ } else {
+ return err
+ }
+ }
+ tm.QueryServiceControl.RedoPreparedTransactions()
+ err = tm.MysqlDaemon.SetReadOnly(ctx, false)
+ return err
+}
+
func (tm *TabletManager) initTablet(ctx context.Context) error {
tablet := tm.Tablet()
err := tm.TopoServer.CreateTablet(ctx, tablet)
@@ -822,9 +866,10 @@ func (tm *TabletManager) handleRestore(ctx context.Context, config *tabletenv.Ta
log.Exitf(fmt.Sprintf("RestoreFromBackup failed: unable to parse the --restore-to-timestamp value provided of '%s'. Error: %v", restoreToTimestampStr, err))
}
}
+
// restoreFromBackup will just be a regular action
// (same as if it was triggered remotely)
- if err := tm.RestoreData(ctx, logutil.NewConsoleLogger(), waitForBackupInterval, false /* deleteBeforeRestore */, backupTime, restoreToTimestamp, restoreToPos, mysqlShutdownTimeout); err != nil {
+ if err := tm.RestoreData(ctx, logutil.NewConsoleLogger(), waitForBackupInterval, false /* deleteBeforeRestore */, backupTime, restoreToTimestamp, restoreToPos, restoreFromBackupAllowedEngines, mysqlShutdownTimeout); err != nil {
log.Exitf("RestoreFromBackup failed: %v", err)
}
@@ -870,6 +915,7 @@ func (tm *TabletManager) exportStats() {
statsKeyRangeEnd.Set(hex.EncodeToString(tablet.KeyRange.End))
}
statsAlias.Set(topoproto.TabletAliasString(tablet.Alias))
+ setTabletTagsStats(tablet)
}
// withRetry will exponentially back off and retry a function upon
@@ -924,50 +970,50 @@ func (tm *TabletManager) hookExtraEnv() map[string]string {
// initializeReplication is used to initialize the replication when the tablet starts.
// It returns the current primary tablet for use externally
-func (tm *TabletManager) initializeReplication(ctx context.Context, tabletType topodatapb.TabletType) (primary *topo.TabletInfo, err error) {
+func (tm *TabletManager) initializeReplication(ctx context.Context, tabletType topodatapb.TabletType) (string, error) {
// If active reparents are disabled, we do not touch replication.
// There is nothing to do
if mysqlctl.DisableActiveReparents {
- return nil, nil
+ return "", nil
}
// If the desired tablet type is primary, then we shouldn't be setting our replication source.
// So there is nothing to do.
if tabletType == topodatapb.TabletType_PRIMARY {
- return nil, nil
+ return "", nil
}
// Read the shard to find the current primary, and its location.
tablet := tm.Tablet()
si, err := tm.TopoServer.GetShard(ctx, tablet.Keyspace, tablet.Shard)
if err != nil {
- return nil, vterrors.Wrap(err, "cannot read shard")
+ return "", vterrors.Wrap(err, "cannot read shard")
}
if si.PrimaryAlias == nil {
// There's no primary. This is fine, since there might be no primary currently
log.Warningf("cannot start replication during initialization: shard %v/%v has no primary.", tablet.Keyspace, tablet.Shard)
- return nil, nil
+ return "", nil
}
if topoproto.TabletAliasEqual(si.PrimaryAlias, tablet.Alias) {
// We used to be the primary before we got restarted,
// and no other primary has been elected in the meantime.
// There isn't anything to do here either.
log.Warningf("cannot start replication during initialization: primary in shard record still points to this tablet.")
- return nil, nil
+ return "", nil
}
currentPrimary, err := tm.TopoServer.GetTablet(ctx, si.PrimaryAlias)
if err != nil {
- return nil, vterrors.Wrapf(err, "cannot read primary tablet %v", si.PrimaryAlias)
+ return "", vterrors.Wrapf(err, "cannot read primary tablet %v", si.PrimaryAlias)
}
durabilityName, err := tm.TopoServer.GetKeyspaceDurability(ctx, tablet.Keyspace)
if err != nil {
- return nil, vterrors.Wrapf(err, "cannot read keyspace durability policy %v", tablet.Keyspace)
+ return "", vterrors.Wrapf(err, "cannot read keyspace durability policy %v", tablet.Keyspace)
}
log.Infof("Getting a new durability policy for %v", durabilityName)
durability, err := reparentutil.GetDurabilityPolicy(durabilityName)
if err != nil {
- return nil, vterrors.Wrapf(err, "cannot get durability policy %v", durabilityName)
+ return "", vterrors.Wrapf(err, "cannot get durability policy %v", durabilityName)
}
// If using semi-sync, we need to enable it before connecting to primary.
// We should set the correct type, since it is used in replica semi-sync
@@ -976,21 +1022,52 @@ func (tm *TabletManager) initializeReplication(ctx context.Context, tabletType t
semiSyncAction, err := tm.convertBoolToSemiSyncAction(ctx, reparentutil.IsReplicaSemiSync(durability, currentPrimary.Tablet, tablet))
if err != nil {
- return nil, err
+ return "", err
}
if err := tm.fixSemiSync(ctx, tabletType, semiSyncAction); err != nil {
- return nil, err
+ return "", err
}
// Set primary and start replication.
if currentPrimary.Tablet.MysqlHostname == "" {
log.Warningf("primary tablet in the shard record does not have mysql hostname specified, possibly because that tablet has been shut down.")
- return nil, nil
+ return "", nil
+ }
+
+ // Find our own executed GTID set and,
+ // the executed GTID set of the tablet that we are reparenting to.
+ // We will then compare our own position against it to verify that we don't
+ // have an errant GTID. If we find any GTID that we have, but the primary doesn't,
+ // we will not enter the replication graph and instead fail replication.
+ replicaPos, err := tm.MysqlDaemon.PrimaryPosition(ctx)
+ if err != nil {
+ return "", err
+ }
+
+ primaryStatus, err := tm.tmc.PrimaryStatus(ctx, currentPrimary.Tablet)
+ if err != nil {
+ return "", err
}
+ primaryPosition, err := replication.DecodePosition(primaryStatus.Position)
+ if err != nil {
+ return "", err
+ }
+ primarySid, err := replication.ParseSID(primaryStatus.ServerUuid)
+ if err != nil {
+ return "", err
+ }
+ errantGtid, err := replication.ErrantGTIDsOnReplica(replicaPos, primaryPosition, primarySid)
+ if err != nil {
+ return "", err
+ }
+ if errantGtid != "" {
+ return "", vterrors.New(vtrpc.Code_FAILED_PRECONDITION, fmt.Sprintf("Errant GTID detected - %s; Primary GTID - %s, Replica GTID - %s", errantGtid, primaryPosition, replicaPos.String()))
+ }
+
if err := tm.MysqlDaemon.SetReplicationSource(ctx, currentPrimary.Tablet.MysqlHostname, currentPrimary.Tablet.MysqlPort, 0, true, true); err != nil {
- return nil, vterrors.Wrap(err, "MysqlDaemon.SetReplicationSource failed")
+ return "", vterrors.Wrap(err, "MysqlDaemon.SetReplicationSource failed")
}
- return currentPrimary, nil
+ return primaryStatus.Position, nil
}
diff --git a/go/vt/vttablet/tabletmanager/tm_init_test.go b/go/vt/vttablet/tabletmanager/tm_init_test.go
index d0c0075eda3..b8c9c54dcc2 100644
--- a/go/vt/vttablet/tabletmanager/tm_init_test.go
+++ b/go/vt/vttablet/tabletmanager/tm_init_test.go
@@ -173,7 +173,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
statsTabletTypeCount.ResetAll()
cell := "cell1"
ts := memorytopo.NewServer(ctx, cell)
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
assert.Equal(t, "replica", statsTabletType.Get())
@@ -193,7 +193,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
// keyspace-shard already created.
_, err = ts.GetOrCreateShard(ctx, "ks1", "0")
require.NoError(t, err)
- tm = newTestTM(t, ts, 2, "ks1", "0")
+ tm = newTestTM(t, ts, 2, "ks1", "0", nil)
defer tm.Stop()
_, err = ts.GetShard(ctx, "ks1", "0")
require.NoError(t, err)
@@ -207,7 +207,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
require.NoError(t, err)
err = topotools.RebuildKeyspace(ctx, logutil.NewConsoleLogger(), ts, "ks2", []string{cell}, false)
require.NoError(t, err)
- tm = newTestTM(t, ts, 3, "ks2", "0")
+ tm = newTestTM(t, ts, 3, "ks2", "0", nil)
defer tm.Stop()
_, err = ts.GetShard(ctx, "ks2", "0")
require.NoError(t, err)
@@ -224,7 +224,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
require.NoError(t, err)
err = ts.RebuildSrvVSchema(ctx, []string{cell})
require.NoError(t, err)
- tm = newTestTM(t, ts, 4, "ks3", "0")
+ tm = newTestTM(t, ts, 4, "ks3", "0", nil)
defer tm.Stop()
_, err = ts.GetShard(ctx, "ks3", "0")
require.NoError(t, err)
@@ -235,7 +235,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
assert.Equal(t, wantVSchema, srvVSchema.Keyspaces["ks3"])
// Multi-shard
- tm1 := newTestTM(t, ts, 5, "ks4", "-80")
+ tm1 := newTestTM(t, ts, 5, "ks4", "-80", nil)
defer tm1.Stop()
// Wait a bit and make sure that srvKeyspace is still not created.
@@ -243,7 +243,7 @@ func TestStartCreateKeyspaceShard(t *testing.T) {
_, err = ts.GetSrvKeyspace(context.Background(), cell, "ks4")
require.True(t, topo.IsErrType(err, topo.NoNode), err)
- tm2 := newTestTM(t, ts, 6, "ks4", "80-")
+ tm2 := newTestTM(t, ts, 6, "ks4", "80-", nil)
defer tm2.Stop()
// Now that we've started the tablet for the other shard, srvKeyspace will succeed.
ensureSrvKeyspace(t, ctx, ts, cell, "ks4")
@@ -264,7 +264,7 @@ func TestCheckPrimaryShip(t *testing.T) {
// 1. Initialize the tablet as REPLICA.
// This will create the respective topology records.
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
tablet := tm.Tablet()
ensureSrvKeyspace(t, ctx, ts, cell, "ks")
ti, err := ts.GetTablet(ctx, alias)
@@ -398,7 +398,7 @@ func TestStartCheckMysql(t *testing.T) {
defer cancel()
cell := "cell1"
ts := memorytopo.NewServer(ctx, cell)
- tablet := newTestTablet(t, 1, "ks", "0")
+ tablet := newTestTablet(t, 1, "ks", "0", nil)
cp := mysql.ConnParams{
Host: "foo",
Port: 1,
@@ -429,7 +429,7 @@ func TestStartFindMysqlPort(t *testing.T) {
defer cancel()
cell := "cell1"
ts := memorytopo.NewServer(ctx, cell)
- tablet := newTestTablet(t, 1, "ks", "0")
+ tablet := newTestTablet(t, 1, "ks", "0", nil)
fmd := newTestMysqlDaemon(t, -1)
tm := &TabletManager{
BatchCtx: context.Background(),
@@ -472,7 +472,7 @@ func TestStartFixesReplicationData(t *testing.T) {
defer cancel()
cell := "cell1"
ts := memorytopo.NewServer(ctx, cell, "cell2")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
tabletAlias := tm.tabletAlias
@@ -505,14 +505,14 @@ func TestStartDoesNotUpdateReplicationDataForTabletInWrongShard(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1", "cell2")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
tm.Stop()
tabletAliases, err := ts.FindAllTabletAliasesInShard(ctx, "ks", "0")
require.NoError(t, err)
assert.Equal(t, uint32(1), tabletAliases[0].Uid)
- tablet := newTestTablet(t, 1, "ks", "-d0")
+ tablet := newTestTablet(t, 1, "ks", "-d0", nil)
require.NoError(t, err)
err = tm.Start(tablet, nil)
assert.Contains(t, err.Error(), "existing tablet keyspace and shard ks/0 differ")
@@ -537,7 +537,7 @@ func TestCheckTabletTypeResets(t *testing.T) {
// 1. Initialize the tablet as REPLICA.
// This will create the respective topology records.
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
tablet := tm.Tablet()
ensureSrvKeyspace(t, ctx, ts, cell, "ks")
ti, err := ts.GetTablet(ctx, alias)
@@ -648,6 +648,27 @@ func TestGetBuildTags(t *testing.T) {
}
}
+func TestStartExportStats(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+
+ ts := memorytopo.NewServer(ctx, "cell1")
+ _ = newTestTM(t, ts, 1, "ks", "0", map[string]string{
+ "test": t.Name(),
+ })
+
+ assert.Equal(t, "ks", statsKeyspace.Get())
+ assert.Equal(t, "0", statsShard.Get())
+ assert.Equal(t, "replica", statsTabletType.Get())
+ assert.Equal(t, map[string]int64{
+ "replica": 1,
+ }, statsTabletTypeCount.Counts())
+ assert.Equal(t, "cell1-0000000001", statsAlias.Get())
+ assert.Equal(t, map[string]int64{
+ "test." + t.Name(): 1,
+ }, statsTabletTags.Counts())
+}
+
func newTestMysqlDaemon(t *testing.T, port int32) *mysqlctl.FakeMysqlDaemon {
t.Helper()
@@ -662,10 +683,14 @@ func newTestMysqlDaemon(t *testing.T, port int32) *mysqlctl.FakeMysqlDaemon {
return mysqld
}
-func newTestTM(t *testing.T, ts *topo.Server, uid int, keyspace, shard string) *TabletManager {
+func newTestTM(t *testing.T, ts *topo.Server, uid int, keyspace, shard string, tags map[string]string) *TabletManager {
+ // reset stats
+ statsTabletTags.ResetAll()
+ statsTabletTypeCount.ResetAll()
+
t.Helper()
ctx := context.Background()
- tablet := newTestTablet(t, uid, keyspace, shard)
+ tablet := newTestTablet(t, uid, keyspace, shard, tags)
tm := &TabletManager{
BatchCtx: ctx,
TopoServer: ts,
@@ -701,7 +726,7 @@ func newTestTM(t *testing.T, ts *topo.Server, uid int, keyspace, shard string) *
}
}
-func newTestTablet(t *testing.T, uid int, keyspace, shard string) *topodatapb.Tablet {
+func newTestTablet(t *testing.T, uid int, keyspace, shard string, tags map[string]string) *topodatapb.Tablet {
shard, keyRange, err := topo.ValidateShardName(shard)
require.NoError(t, err)
return &topodatapb.Tablet{
@@ -718,6 +743,7 @@ func newTestTablet(t *testing.T, uid int, keyspace, shard string) *topodatapb.Ta
Shard: shard,
KeyRange: keyRange,
Type: topodatapb.TabletType_REPLICA,
+ Tags: tags,
}
}
diff --git a/go/vt/vttablet/tabletmanager/tm_state.go b/go/vt/vttablet/tabletmanager/tm_state.go
index 312c675fce7..026f2b62552 100644
--- a/go/vt/vttablet/tabletmanager/tm_state.go
+++ b/go/vt/vttablet/tabletmanager/tm_state.go
@@ -37,6 +37,7 @@ import (
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/topotools"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver"
"vitess.io/vitess/go/vt/vttablet/tabletserver/rules"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
@@ -135,11 +136,10 @@ func (ts *tmState) RefreshFromTopo(ctx context.Context) error {
if err != nil {
return err
}
- ts.RefreshFromTopoInfo(ctx, shardInfo, srvKeyspace)
- return nil
+ return ts.RefreshFromTopoInfo(ctx, shardInfo, srvKeyspace)
}
-func (ts *tmState) RefreshFromTopoInfo(ctx context.Context, shardInfo *topo.ShardInfo, srvKeyspace *topodatapb.SrvKeyspace) {
+func (ts *tmState) RefreshFromTopoInfo(ctx context.Context, shardInfo *topo.ShardInfo, srvKeyspace *topodatapb.SrvKeyspace) error {
ts.mu.Lock()
defer ts.mu.Unlock()
@@ -157,6 +157,7 @@ func (ts *tmState) RefreshFromTopoInfo(ctx context.Context, shardInfo *topo.Shar
if srvKeyspace != nil {
ts.isShardServing = make(map[topodatapb.TabletType]bool)
ts.tabletControls = make(map[topodatapb.TabletType]bool)
+ ts.tm.QueryServiceControl.SetTwoPCAllowed(tabletserver.TwoPCAllowed_TabletControls, true)
for _, partition := range srvKeyspace.GetPartitions() {
@@ -169,7 +170,10 @@ func (ts *tmState) RefreshFromTopoInfo(ctx context.Context, shardInfo *topo.Shar
for _, tabletControl := range partition.GetShardTabletControls() {
if key.KeyRangeEqual(tabletControl.GetKeyRange(), ts.KeyRange()) {
if tabletControl.QueryServiceDisabled {
- ts.tabletControls[partition.GetServedType()] = true
+ err := ts.prepareForDisableQueryService(ctx, partition.GetServedType())
+ if err != nil {
+ return err
+ }
}
break
}
@@ -177,7 +181,20 @@ func (ts *tmState) RefreshFromTopoInfo(ctx context.Context, shardInfo *topo.Shar
}
}
- _ = ts.updateLocked(ctx)
+ return ts.updateLocked(ctx)
+}
+
+// prepareForDisableQueryService prepares the tablet for disabling query service.
+func (ts *tmState) prepareForDisableQueryService(ctx context.Context, servType topodatapb.TabletType) error {
+ if servType == topodatapb.TabletType_PRIMARY {
+ ts.tm.QueryServiceControl.SetTwoPCAllowed(tabletserver.TwoPCAllowed_TabletControls, false)
+ err := ts.tm.QueryServiceControl.WaitForPreparedTwoPCTransactions(ctx)
+ if err != nil {
+ return err
+ }
+ }
+ ts.tabletControls[servType] = true
+ return nil
}
func (ts *tmState) ChangeTabletType(ctx context.Context, tabletType topodatapb.TabletType, action DBAction) error {
@@ -214,9 +231,10 @@ func (ts *tmState) ChangeTabletType(ctx context.Context, tabletType topodatapb.T
}
if action == DBActionSetReadWrite {
+ // We need to redo the prepared transactions in read only mode using the dba user to ensure we don't lose them.
// We call SetReadOnly only after the topo has been updated to avoid
// situations where two tablets are primary at the DB level but not at the vitess level
- if err := ts.tm.MysqlDaemon.SetReadOnly(ctx, false); err != nil {
+ if err = ts.tm.redoPreparedTransactionsAndSetReadWrite(ctx); err != nil {
return err
}
}
@@ -239,6 +257,17 @@ func (ts *tmState) ChangeTabletType(ctx context.Context, tabletType topodatapb.T
return err
}
+func (ts *tmState) ChangeTabletTags(ctx context.Context, tabletTags map[string]string) {
+ ts.mu.Lock()
+ defer ts.mu.Unlock()
+ log.Infof("Changing Tablet Tags: %v for %s", tabletTags, ts.tablet.Alias.String())
+
+ ts.tablet.Tags = tabletTags
+ ts.publishStateLocked(ctx)
+ ts.publishForDisplay()
+ setTabletTagsStats(ts.tablet)
+}
+
func (ts *tmState) SetMysqlPort(mport int32) {
ts.mu.Lock()
defer ts.mu.Unlock()
@@ -281,7 +310,7 @@ func (ts *tmState) updateLocked(ctx context.Context) error {
errStr := fmt.Sprintf("SetServingType(serving=false) failed: %v", err)
log.Errorf(errStr)
// No need to short circuit. Apply all steps and return error in the end.
- returnErr = vterrors.Wrapf(err, errStr)
+ returnErr = vterrors.Wrap(err, errStr)
}
}
@@ -289,7 +318,7 @@ func (ts *tmState) updateLocked(ctx context.Context) error {
errStr := fmt.Sprintf("Cannot update denied tables rule: %v", err)
log.Errorf(errStr)
// No need to short circuit. Apply all steps and return error in the end.
- returnErr = vterrors.Wrapf(err, errStr)
+ returnErr = vterrors.Wrap(err, errStr)
}
if ts.tm.UpdateStream != nil {
@@ -329,7 +358,7 @@ func (ts *tmState) updateLocked(ctx context.Context) error {
if err := ts.tm.QueryServiceControl.SetServingType(ts.tablet.Type, ptsTime, true, ""); err != nil {
errStr := fmt.Sprintf("Cannot start query service: %v", err)
log.Errorf(errStr)
- returnErr = vterrors.Wrapf(err, errStr)
+ returnErr = vterrors.Wrap(err, errStr)
}
}
diff --git a/go/vt/vttablet/tabletmanager/tm_state_test.go b/go/vt/vttablet/tabletmanager/tm_state_test.go
index 8bd98edefff..7a507248e18 100644
--- a/go/vt/vttablet/tabletmanager/tm_state_test.go
+++ b/go/vt/vttablet/tabletmanager/tm_state_test.go
@@ -46,7 +46,7 @@ func TestStateOpenClose(t *testing.T) {
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
// Re-Open should be a no-op
tm.tmState.mu.Lock()
@@ -69,7 +69,7 @@ func TestStateRefreshFromTopo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
err := tm.RefreshState(ctx)
@@ -80,7 +80,7 @@ func TestStateResharding(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
tm.tmState.mu.Lock()
@@ -108,7 +108,7 @@ func TestStateDenyList(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
fmd := tm.MysqlDaemon.(*mysqlctl.FakeMysqlDaemon)
@@ -140,7 +140,7 @@ func TestStateTabletControls(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
ks := &topodatapb.SrvKeyspace{
@@ -169,7 +169,7 @@ func TestStateIsShardServingisInSrvKeyspace(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
tm.tmState.mu.Lock()
@@ -341,7 +341,7 @@ func TestStateNonServing(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 1, "ks", "0")
+ tm := newTestTM(t, ts, 1, "ks", "0", nil)
defer tm.Stop()
tm.tmState.mu.Lock()
@@ -359,7 +359,7 @@ func TestStateChangeTabletType(t *testing.T) {
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
statsTabletTypeCount.ResetAll()
- tm := newTestTM(t, ts, 2, "ks", "0")
+ tm := newTestTM(t, ts, 2, "ks", "0", nil)
defer tm.Stop()
assert.Equal(t, 1, len(statsTabletTypeCount.Counts()))
@@ -402,7 +402,7 @@ func TestStateChangeTabletTypeWithFailure(t *testing.T) {
ts := memorytopo.NewServer(ctx, "cell1")
statsTabletTypeCount.ResetAll()
// create TM with replica and put a hook to return error during SetServingType
- tm := newTestTM(t, ts, 2, "ks", "0")
+ tm := newTestTM(t, ts, 2, "ks", "0", nil)
qsc := tm.QueryServiceControl.(*tabletservermock.Controller)
qsc.SetServingTypeError = vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "mocking resource exhaustion error ")
defer tm.Stop()
@@ -485,7 +485,7 @@ func TestChangeTypeErrorWhileWritingToTopo(t *testing.T) {
fakeConn := factory.AddCell("cell1")
ts := faketopo.NewFakeTopoServer(context.TODO(), factory)
statsTabletTypeCount.ResetAll()
- tm := newTestTM(t, ts, 2, "ks", "0")
+ tm := newTestTM(t, ts, 2, "ks", "0", nil)
defer tm.Stop()
// ChangeTabletType calls topotools.ChangeType which in-turn issues
@@ -533,7 +533,7 @@ func TestPublishStateNew(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 42, "ks", "0")
+ tm := newTestTM(t, ts, 42, "ks", "0", nil)
ttablet, err := tm.TopoServer.GetTablet(ctx, tm.tabletAlias)
require.NoError(t, err)
utils.MustMatch(t, tm.Tablet(), ttablet.Tablet)
@@ -580,7 +580,7 @@ func TestPublishDeleted(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ts := memorytopo.NewServer(ctx, "cell1")
- tm := newTestTM(t, ts, 2, "ks", "0")
+ tm := newTestTM(t, ts, 2, "ks", "0", nil)
defer tm.Stop()
alias := &topodatapb.TabletAlias{
diff --git a/go/vt/vttablet/tabletmanager/vdiff/action.go b/go/vt/vttablet/tabletmanager/vdiff/action.go
index 0b9dd6f45ed..9cfb145c1e0 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/action.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/action.go
@@ -37,7 +37,7 @@ import (
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
-type VDiffAction string //nolint
+type VDiffAction string // nolint
const (
CreateAction VDiffAction = "create"
@@ -97,15 +97,15 @@ func (vde *Engine) PerformVDiffAction(ctx context.Context, req *tabletmanagerdat
return nil, err
}
case ShowAction:
- if err := vde.handleShowAction(ctx, dbClient, action, req, resp); err != nil {
+ if err := vde.handleShowAction(ctx, dbClient, req, resp); err != nil {
return nil, err
}
case StopAction:
- if err := vde.handleStopAction(ctx, dbClient, action, req, resp); err != nil {
+ if err := vde.handleStopAction(ctx, dbClient, req, resp); err != nil {
return nil, err
}
case DeleteAction:
- if err := vde.handleDeleteAction(ctx, dbClient, action, req, resp); err != nil {
+ if err := vde.handleDeleteAction(ctx, dbClient, req, resp); err != nil {
return nil, err
}
default:
@@ -174,7 +174,7 @@ func (vde *Engine) getDefaultCell() (string, error) {
func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlogplayer.DBClient, action VDiffAction, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
var qr *sqltypes.Result
- options := req.Options
+ options := req.GetOptions()
query, err := sqlparser.ParseAndBind(sqlGetVDiffID, sqltypes.StringBindVariable(req.VdiffUuid))
if err != nil {
@@ -184,20 +184,12 @@ func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlog
return err
}
recordFound := len(qr.Rows) == 1
- if recordFound && action == CreateAction {
- return fmt.Errorf("vdiff with UUID %s already exists on tablet %v",
- req.VdiffUuid, vde.thisTablet.Alias)
- } else if action == ResumeAction {
- if !recordFound {
- return fmt.Errorf("vdiff with UUID %s not found on tablet %v",
- req.VdiffUuid, vde.thisTablet.Alias)
- }
- if resp.Id, err = qr.Named().Row().ToInt64("id"); err != nil {
- return fmt.Errorf("vdiff found with invalid id on tablet %v: %w",
- vde.thisTablet.Alias, err)
- }
- }
+
if action == CreateAction {
+ if recordFound {
+ return fmt.Errorf("vdiff with UUID %s already exists on tablet %s",
+ req.VdiffUuid, topoproto.TabletAliasString(vde.thisTablet.Alias))
+ }
// Use the options specified via the vdiff create client
// command, which we'll then store in the vdiff record.
if options, err = vde.fixupOptions(options); err != nil {
@@ -207,10 +199,14 @@ func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlog
if err != nil {
return err
}
+ state := PendingState
+ if options.CoreOptions != nil && options.CoreOptions.AutoStart != nil && !options.CoreOptions.GetAutoStart() {
+ state = StoppedState
+ }
query, err := sqlparser.ParseAndBind(sqlNewVDiff,
sqltypes.StringBindVariable(req.Keyspace),
sqltypes.StringBindVariable(req.Workflow),
- sqltypes.StringBindVariable("pending"),
+ sqltypes.StringBindVariable(string(state)),
sqltypes.StringBindVariable(string(optionsJSON)),
sqltypes.StringBindVariable(vde.thisTablet.Shard),
sqltypes.StringBindVariable(topoproto.TabletDbName(vde.thisTablet)),
@@ -223,24 +219,44 @@ func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlog
return err
}
if qr.InsertID == 0 {
- return fmt.Errorf("unable to create vdiff for UUID %s on tablet %v (%w)",
- req.VdiffUuid, vde.thisTablet.Alias, err)
+ return fmt.Errorf("unable to create vdiff for UUID %s on tablet %s (%w)",
+ req.VdiffUuid, topoproto.TabletAliasString(vde.thisTablet.Alias), err)
}
resp.Id = int64(qr.InsertID)
} else {
- query, err := sqlparser.ParseAndBind(sqlResumeVDiff,
- sqltypes.StringBindVariable(req.VdiffUuid),
- )
- if err != nil {
- return err
+ if !recordFound {
+ return fmt.Errorf("vdiff with UUID %s not found on tablet %s",
+ req.VdiffUuid, topoproto.TabletAliasString(vde.thisTablet.Alias))
}
- if qr, err = dbClient.ExecuteFetch(query, 1); err != nil {
+ if resp.Id, err = qr.Named().Row().ToInt64("id"); err != nil {
+ return fmt.Errorf("vdiff found with invalid id on tablet %s: %w",
+ topoproto.TabletAliasString(vde.thisTablet.Alias), err)
+ }
+ execResume := func(query string) (rowsAffected uint64, err error) {
+ query, err = sqlparser.ParseAndBind(query,
+ sqltypes.StringBindVariable(req.VdiffUuid),
+ )
+ if err != nil {
+ return 0, err
+ }
+ if qr, err = dbClient.ExecuteFetch(query, 1); err != nil {
+ return 0, err
+ }
+ return qr.RowsAffected, nil
+ }
+ rowsAffected, err := execResume(sqlResumeVDiff)
+ if err != nil {
return err
}
- if qr.RowsAffected == 0 {
- msg := fmt.Sprintf("no completed or stopped vdiff found for UUID %s on tablet %v",
- req.VdiffUuid, vde.thisTablet.Alias)
- return fmt.Errorf(msg)
+ if rowsAffected == 0 { // See if it's a vdiff that was never started
+ rowsAffected, err := execResume(sqlStartVDiff)
+ if err != nil {
+ return err
+ }
+ if rowsAffected == 0 {
+ return fmt.Errorf("no completed or stopped vdiff found for UUID %s on tablet %s",
+ req.VdiffUuid, topoproto.TabletAliasString(vde.thisTablet.Alias))
+ }
}
}
@@ -252,7 +268,7 @@ func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlog
vdiffRecord := qr.Named().Row()
if vdiffRecord == nil {
return fmt.Errorf("unable to %s vdiff for UUID %s as it was not found on tablet %v (%w)",
- action, req.VdiffUuid, vde.thisTablet.Alias, err)
+ action, req.VdiffUuid, topoproto.TabletAliasString(vde.thisTablet.Alias), err)
}
if action == ResumeAction {
// Use the existing options from the vdiff record.
@@ -272,7 +288,7 @@ func (vde *Engine) handleCreateResumeAction(ctx context.Context, dbClient binlog
return nil
}
-func (vde *Engine) handleShowAction(ctx context.Context, dbClient binlogplayer.DBClient, action VDiffAction, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
+func (vde *Engine) handleShowAction(ctx context.Context, dbClient binlogplayer.DBClient, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
var qr *sqltypes.Result
vdiffUUID := ""
@@ -312,8 +328,8 @@ func (vde *Engine) handleShowAction(ctx context.Context, dbClient binlogplayer.D
}
switch len(qr.Rows) {
case 0:
- return fmt.Errorf("no vdiff found for UUID %s keyspace %s and workflow %s on tablet %v",
- vdiffUUID, req.Keyspace, req.Workflow, vde.thisTablet.Alias)
+ return fmt.Errorf("no vdiff found for UUID %s keyspace %s and workflow %s on tablet %s",
+ vdiffUUID, req.Keyspace, req.Workflow, topoproto.TabletAliasString(vde.thisTablet.Alias))
case 1:
row := qr.Named().Row()
vdiffID, _ := row["id"].ToInt64()
@@ -323,8 +339,8 @@ func (vde *Engine) handleShowAction(ctx context.Context, dbClient binlogplayer.D
return err
}
default:
- return fmt.Errorf("too many vdiffs found (%d) for UUID %s keyspace %s and workflow %s on tablet %v",
- len(qr.Rows), vdiffUUID, req.Keyspace, req.Workflow, vde.thisTablet.Alias)
+ return fmt.Errorf("too many vdiffs found (%d) for UUID %s keyspace %s and workflow %s on tablet %s",
+ len(qr.Rows), vdiffUUID, req.Keyspace, req.Workflow, topoproto.TabletAliasString(vde.thisTablet.Alias))
}
}
switch req.ActionArg {
@@ -351,7 +367,7 @@ func (vde *Engine) handleShowAction(ctx context.Context, dbClient binlogplayer.D
return nil
}
-func (vde *Engine) handleStopAction(ctx context.Context, dbClient binlogplayer.DBClient, action VDiffAction, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
+func (vde *Engine) handleStopAction(ctx context.Context, dbClient binlogplayer.DBClient, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
vde.mu.Lock()
defer vde.mu.Unlock()
for _, controller := range vde.controllers {
@@ -366,7 +382,7 @@ func (vde *Engine) handleStopAction(ctx context.Context, dbClient binlogplayer.D
return nil
}
-func (vde *Engine) handleDeleteAction(ctx context.Context, dbClient binlogplayer.DBClient, action VDiffAction, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
+func (vde *Engine) handleDeleteAction(ctx context.Context, dbClient binlogplayer.DBClient, req *tabletmanagerdatapb.VDiffRequest, resp *tabletmanagerdatapb.VDiffResponse) error {
vde.mu.Lock()
defer vde.mu.Unlock()
var deleteQuery string
@@ -425,8 +441,8 @@ func (vde *Engine) handleDeleteAction(ctx context.Context, dbClient binlogplayer
}
row := res.Named().Row() // Must only be one
if row == nil {
- return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no vdiff found for UUID %s on tablet %v",
- uuid, vde.thisTablet.Alias)
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "no vdiff found for UUID %s on tablet %s",
+ uuid, topoproto.TabletAliasString(vde.thisTablet.Alias))
}
cleanupController(vde.controllers[row.AsInt64("id", -1)])
deleteQuery, err = sqlparser.ParseAndBind(sqlDeleteVDiffByUUID,
diff --git a/go/vt/vttablet/tabletmanager/vdiff/action_test.go b/go/vt/vttablet/tabletmanager/vdiff/action_test.go
index 6949b411f13..6bdc7044878 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/action_test.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/action_test.go
@@ -26,6 +26,7 @@ import (
"github.com/google/uuid"
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/ptr"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/vterrors"
@@ -46,6 +47,7 @@ func TestPerformVDiffAction(t *testing.T) {
query string
result *sqltypes.Result // Optional if you need a non-empty result
}
+
tests := []struct {
name string
vde *Engine
@@ -93,6 +95,27 @@ func TestPerformVDiffAction(t *testing.T) {
return tstenv.TopoServ.DeleteCellInfo(ctx, "zone100_test", true)
},
},
+ {
+ name: "create without starting",
+ req: &tabletmanagerdatapb.VDiffRequest{
+ Action: string(CreateAction),
+ VdiffUuid: uuid,
+ Options: &tabletmanagerdatapb.VDiffOptions{
+ PickerOptions: &tabletmanagerdatapb.VDiffPickerOptions{},
+ CoreOptions: &tabletmanagerdatapb.VDiffCoreOptions{
+ AutoStart: ptr.Of(false),
+ },
+ },
+ },
+ expectQueries: []queryAndResult{
+ {
+ query: fmt.Sprintf("select id as id from _vt.vdiff where vdiff_uuid = %s", encodeString(uuid)),
+ },
+ {
+ query: fmt.Sprintf(`insert into _vt.vdiff(keyspace, workflow, state, options, shard, db_name, vdiff_uuid) values('', '', 'stopped', '{"picker_options":{"source_cell":"cell1","target_cell":"cell1"},"core_options":{"auto_start":false}}', '0', 'vt_vttest', %s)`, encodeString(uuid)),
+ },
+ },
+ },
{
name: "create with cell alias",
req: &tabletmanagerdatapb.VDiffRequest{
@@ -130,6 +153,79 @@ func TestPerformVDiffAction(t *testing.T) {
return tstenv.TopoServ.DeleteCellsAlias(ctx, "all")
},
},
+ {
+ name: "resume never started vdiff",
+ req: &tabletmanagerdatapb.VDiffRequest{
+ Action: string(ResumeAction),
+ VdiffUuid: uuid,
+ Keyspace: keyspace,
+ Workflow: workflow,
+ },
+ expectQueries: []queryAndResult{
+ {
+ query: fmt.Sprintf("select id as id from _vt.vdiff where vdiff_uuid = %s", encodeString(uuid)),
+ result: sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id",
+ "int64",
+ ),
+ "1",
+ ),
+ },
+ {
+ query: fmt.Sprintf(`update _vt.vdiff as vd, _vt.vdiff_table as vdt set vd.started_at = NULL, vd.completed_at = NULL, vd.state = 'pending',
+ vdt.state = 'pending' where vd.vdiff_uuid = %s and vd.id = vdt.vdiff_id and vd.state in ('completed', 'stopped')
+ and vdt.state in ('completed', 'stopped')`, encodeString(uuid)),
+ result: &sqltypes.Result{
+ RowsAffected: 0, // No _vt.vdiff_table records
+ },
+ },
+ {
+ query: fmt.Sprintf(`update _vt.vdiff as vd set vd.state = 'pending' where vd.vdiff_uuid = %s and vd.state = 'stopped' and
+ vd.started_at is NULL and vd.completed_at is NULL and
+ (select count(*) as cnt from _vt.vdiff_table as vdt where vd.id = vdt.vdiff_id) = 0`,
+ encodeString(uuid)),
+ result: &sqltypes.Result{
+ RowsAffected: 1,
+ },
+ },
+ {
+ query: "select * from _vt.vdiff where id = 1",
+ },
+ },
+ },
+ {
+ name: "resume completed vdiff",
+ req: &tabletmanagerdatapb.VDiffRequest{
+ Action: string(ResumeAction),
+ VdiffUuid: uuid,
+ Keyspace: keyspace,
+ Workflow: workflow,
+ },
+ expectQueries: []queryAndResult{
+ {
+ query: fmt.Sprintf("select id as id from _vt.vdiff where vdiff_uuid = %s", encodeString(uuid)),
+ result: sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "id",
+ "int64",
+ ),
+ "1",
+ ),
+ },
+ {
+ query: fmt.Sprintf(`update _vt.vdiff as vd, _vt.vdiff_table as vdt set vd.started_at = NULL, vd.completed_at = NULL, vd.state = 'pending',
+ vdt.state = 'pending' where vd.vdiff_uuid = %s and vd.id = vdt.vdiff_id and vd.state in ('completed', 'stopped')
+ and vdt.state in ('completed', 'stopped')`, encodeString(uuid)),
+ result: &sqltypes.Result{
+ RowsAffected: 1,
+ },
+ },
+ {
+ query: "select * from _vt.vdiff where id = 1",
+ },
+ },
+ },
{
name: "delete by uuid",
req: &tabletmanagerdatapb.VDiffRequest{
@@ -213,6 +309,7 @@ func TestPerformVDiffAction(t *testing.T) {
},
},
}
+
errCount := int64(0)
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
diff --git a/go/vt/vttablet/tabletmanager/vdiff/engine_test.go b/go/vt/vttablet/tabletmanager/vdiff/engine_test.go
index ef3c673cc8f..61d713a8e58 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/engine_test.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/engine_test.go
@@ -77,10 +77,10 @@ func TestEngineOpen(t *testing.T) {
fmt.Sprintf("1|%s|%s|%s|%s|%s|%s|%s|", UUID, vdiffenv.workflow, tstenv.KeyspaceName, tstenv.ShardName, vdiffDBName, tt.state, optionsJS),
), nil)
vdenv.dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vreplication where workflow = '%s' and db_name = '%s'", vdiffenv.workflow, vdiffDBName), sqltypes.MakeTestResult(sqltypes.MakeTestFields(
- "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type",
- "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64",
+ "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|options",
+ "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64|varchar",
),
- fmt.Sprintf("1|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1", vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
+ fmt.Sprintf("1|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1|{}", vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
), nil)
// Now let's short circuit the vdiff as we know that the open has worked as expected.
@@ -126,10 +126,10 @@ func TestVDiff(t *testing.T) {
vdenv.dbClient.ExpectRequest("select * from _vt.vdiff where id = 1", controllerQR, nil)
vdenv.dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vreplication where workflow = '%s' and db_name = '%s'", vdiffenv.workflow, vdiffDBName), sqltypes.MakeTestResult(sqltypes.MakeTestFields(
- "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type",
- "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64",
+ "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|options",
+ "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64|varchar",
),
- fmt.Sprintf("1|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1", vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
+ fmt.Sprintf("1|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1|{}", vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
), nil)
vdenv.dbClient.ExpectRequest("update _vt.vdiff set state = 'started', last_error = left('', 1024) , started_at = utc_timestamp() where id = 1", singleRowAffected, nil)
vdenv.dbClient.ExpectRequest("insert into _vt.vdiff_log(vdiff_id, message) values (1, 'State changed to: started')", singleRowAffected, nil)
@@ -263,10 +263,10 @@ func TestEngineRetryErroredVDiffs(t *testing.T) {
fmt.Sprintf("%s|%s|%s|%s|%s|%s|pending|%s|", id, UUID, vdiffenv.workflow, tstenv.KeyspaceName, tstenv.ShardName, vdiffDBName, optionsJS),
), nil)
vdiffenv.dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vreplication where workflow = '%s' and db_name = '%s'", vdiffenv.workflow, vdiffDBName), sqltypes.MakeTestResult(sqltypes.MakeTestFields(
- "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type",
- "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64",
+ "id|workflow|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|options",
+ "int64|varbinary|blob|varbinary|varbinary|int64|int64|varbinary|varbinary|int64|int64|varbinary|varbinary|varbinary|int64|varbinary|int64|int64|int64|varchar|int64|varchar",
),
- fmt.Sprintf("%s|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1", id, vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
+ fmt.Sprintf("%s|%s|%s|%s||9223372036854775807|9223372036854775807||PRIMARY,REPLICA|1669511347|0|Running||%s|200||1669511347|1|0||1|{}", id, vdiffenv.workflow, vreplSource, vdiffSourceGtid, vdiffDBName),
), nil)
// At this point we know that we kicked off the expected retry so we can short circuit the vdiff.
diff --git a/go/vt/vttablet/tabletmanager/vdiff/framework_test.go b/go/vt/vttablet/tabletmanager/vdiff/framework_test.go
index 8df060e4170..7d4cdb78c20 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/framework_test.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/framework_test.go
@@ -26,9 +26,11 @@ import (
"sync"
"testing"
+ "github.com/google/uuid"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ _flag "vitess.io/vitess/go/internal/flag"
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/mysql/capabilities"
"vitess.io/vitess/go/sqltypes"
@@ -183,6 +185,7 @@ func init() {
}
func TestMain(m *testing.M) {
+ _flag.ParseFlagsForTest()
exitCode := func() int {
var err error
ctx, cancel := context.WithCancel(context.Background())
@@ -213,7 +216,7 @@ func shortCircuitTestAfterQuery(query string, dbClient *binlogplayer.MockDBClien
dbClient.ExpectRequest("insert into _vt.vdiff_log(vdiff_id, message) values (1, 'Error: Short circuiting test')", singleRowAffected, nil)
}
-//--------------------------------------
+// --------------------------------------
// Topos and tablets
// fakeTabletConn implement TabletConn interface. We only care about the
@@ -249,7 +252,7 @@ func (ftc *fakeTabletConn) VStream(ctx context.Context, request *binlogdatapb.VS
if vstreamHook != nil {
vstreamHook(ctx)
}
- return vdiffenv.vse.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send)
+ return vdiffenv.vse.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send, nil)
}
// vstreamRowsHook allows you to do work just before calling VStreamRows.
@@ -276,14 +279,14 @@ func (ftc *fakeTabletConn) VStreamRows(ctx context.Context, request *binlogdatap
vstreamRowsSendHook(ctx)
}
return send(rows)
- })
+ }, nil)
}
func (ftc *fakeTabletConn) Close(ctx context.Context) error {
return nil
}
-//--------------------------------------
+// --------------------------------------
// Binlog Client to TabletManager
// fakeBinlogClient satisfies binlogplayer.Client.
@@ -344,7 +347,7 @@ func (bts *btStream) Recv() (*binlogdatapb.BinlogTransaction, error) {
return nil, bts.ctx.Err()
}
-//--------------------------------------
+// --------------------------------------
// DBCLient wrapper
func realDBClientFactory() binlogplayer.DBClient {
@@ -393,6 +396,10 @@ func (dbc *realDBClient) Close() {
dbc.conn.Close()
}
+func (dbc *realDBClient) IsClosed() bool {
+ return dbc.conn.IsClosed()
+}
+
func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) {
// Use Clone() because the contents of memory region referenced by
// string can change when clients (e.g. vcopier) use unsafe string methods.
@@ -427,7 +434,7 @@ func (dbc *realDBClient) SupportsCapability(capability capabilities.FlavorCapabi
return dbc.conn.SupportsCapability(capability)
}
-//----------------------------------------------
+// ----------------------------------------------
// fakeTMClient
type fakeTMClient struct {
@@ -622,10 +629,10 @@ func (tvde *testVDiffEnv) close() {
tstenv.SchemaEngine.Reload(context.Background())
}
tvde.tablets = nil
- vdiffenv.vse.Close()
- vdiffenv.vre.Close()
- vdiffenv.vde.Close()
- vdiffenv.dbClient.Close()
+ tvde.vse.Close()
+ tvde.vre.Close()
+ tvde.vde.Close()
+ tvde.dbClient.Close()
}
func (tvde *testVDiffEnv) addTablet(id int, keyspace, shard string, tabletType topodatapb.TabletType) *fakeTabletConn {
@@ -660,3 +667,16 @@ func (tvde *testVDiffEnv) addTablet(id int, keyspace, shard string, tabletType t
tstenv.SchemaEngine.Reload(context.Background())
return tvde.tablets[id]
}
+
+func (tvde *testVDiffEnv) createController(t *testing.T, id int) *controller {
+ controllerQR := sqltypes.MakeTestResult(sqltypes.MakeTestFields(
+ vdiffTestCols,
+ vdiffTestColTypes,
+ ),
+ fmt.Sprintf("%d|%s|%s|%s|%s|%s|%s|%s|", id, uuid.New(), tvde.workflow, tstenv.KeyspaceName, tstenv.ShardName, vdiffDBName, PendingState, optionsJS),
+ )
+ tvde.dbClient.ExpectRequest(fmt.Sprintf("select * from _vt.vdiff where id = %d", id), noResults, nil)
+ ct, err := newController(context.Background(), controllerQR.Named().Row(), tvde.dbClientFactory, tstenv.TopoServ, tvde.vde, tvde.opts)
+ require.NoError(t, err)
+ return ct
+}
diff --git a/go/vt/vttablet/tabletmanager/vdiff/report.go b/go/vt/vttablet/tabletmanager/vdiff/report.go
index 62ce6d24585..b53288b3019 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/report.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/report.go
@@ -18,11 +18,11 @@ package vdiff
import (
"fmt"
- "sort"
- "strings"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/sqlparser"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
)
const (
@@ -63,9 +63,9 @@ type RowDiff struct {
Query string `json:"Query,omitempty"`
}
-func (td *tableDiffer) genRowDiff(queryStmt string, row []sqltypes.Value, debug, onlyPks bool) (*RowDiff, error) {
- drp := &RowDiff{}
- drp.Row = make(map[string]string)
+func (td *tableDiffer) genRowDiff(queryStmt string, row []sqltypes.Value, opts *tabletmanagerdatapb.VDiffReportOptions) (*RowDiff, error) {
+ rd := &RowDiff{}
+ rd.Row = make(map[string]string)
statement, err := td.wd.ct.vde.parser.Parse(queryStmt)
if err != nil {
return nil, err
@@ -75,30 +75,44 @@ func (td *tableDiffer) genRowDiff(queryStmt string, row []sqltypes.Value, debug,
return nil, fmt.Errorf("unexpected: %+v", sqlparser.String(statement))
}
- if debug {
- drp.Query = td.genDebugQueryDiff(sel, row, onlyPks)
+ if opts.GetDebugQuery() {
+ rd.Query = td.genDebugQueryDiff(sel, row, opts.GetOnlyPks())
}
- setVal := func(index int) {
+ addVal := func(index int, truncateAt int) {
buf := sqlparser.NewTrackedBuffer(nil)
sel.SelectExprs[index].Format(buf)
col := buf.String()
- drp.Row[col] = row[index].ToString()
+ // Let's truncate if it's really worth it to avoid losing
+ // value for a few chars.
+ if truncateAt > 0 && row[index].Len() >= truncateAt+len(truncatedNotation)+20 {
+ rd.Row[col] = row[index].ToString()[:truncateAt] + truncatedNotation
+ } else {
+ rd.Row[col] = row[index].ToString()
+ }
}
- if onlyPks {
- for _, pkI := range td.tablePlan.selectPks {
- setVal(pkI)
- }
- return drp, nil
+ // Include PK columns first and do not truncate them so that
+ // the user can always at a minimum identify the row for
+ // further investigation.
+ pks := make(map[int]struct{}, len(td.tablePlan.selectPks))
+ for _, pkI := range td.tablePlan.selectPks {
+ addVal(pkI, 0)
+ pks[pkI] = struct{}{}
}
+ if opts.GetOnlyPks() {
+ return rd, nil
+ }
+
+ truncateAt := int(opts.GetRowDiffColumnTruncateAt())
for i := range sel.SelectExprs {
- setVal(i)
+ if _, pk := pks[i]; !pk {
+ addVal(i, truncateAt)
+ }
}
- formatSampleRow(drp)
- return drp, nil
+ return rd, nil
}
func (td *tableDiffer) genDebugQueryDiff(sel *sqlparser.Select, row []sqltypes.Value, onlyPks bool) string {
@@ -130,22 +144,3 @@ func (td *tableDiffer) genDebugQueryDiff(sel *sqlparser.Select, row []sqltypes.V
buf.Myprintf(";")
return buf.String()
}
-
-// formatSampleRow returns a formatted string representing a sample
-// extra/mismatched row
-func formatSampleRow(rd *RowDiff) {
- keys := make([]string, 0, len(rd.Row))
- rowString := strings.Builder{}
- for k := range rd.Row {
- keys = append(keys, k)
- }
-
- sort.Strings(keys)
- for _, k := range keys {
- // Let's truncate if it's really worth it to avoid losing value for a few chars
- if len(rd.Row[k]) >= 30+len(truncatedNotation)+20 {
- rd.Row[k] = rd.Row[k][:30] + truncatedNotation
- }
- rowString.WriteString(fmt.Sprintf("%s: %s\n", k, rd.Row[k]))
- }
-}
diff --git a/go/vt/vttablet/tabletmanager/vdiff/report_test.go b/go/vt/vttablet/tabletmanager/vdiff/report_test.go
new file mode 100644
index 00000000000..9b6d840f751
--- /dev/null
+++ b/go/vt/vttablet/tabletmanager/vdiff/report_test.go
@@ -0,0 +1,186 @@
+/*
+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 vdiff
+
+import (
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/mysql/collations"
+ "vitess.io/vitess/go/sqltypes"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+)
+
+func TestGenRowDiff(t *testing.T) {
+ vdenv := newTestVDiffEnv(t)
+ defer vdenv.close()
+
+ testCases := []struct {
+ name string
+ schema *tabletmanagerdatapb.SchemaDefinition
+ query string
+ tablePlan *tablePlan
+ row []sqltypes.Value
+ reportOptions *tabletmanagerdatapb.VDiffReportOptions
+ want *RowDiff
+ }{
+ {
+ name: "defaults",
+ schema: &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: "t1",
+ Columns: []string{"c1", "c2", "c3", "c4", "c5"},
+ PrimaryKeyColumns: []string{"c1", "c5"},
+ Fields: sqltypes.MakeTestFields("c1|c2|c3|c4|c5", "int64|int64|varchar|varchar|int64"),
+ },
+ },
+ },
+ query: "select c1,c2,c3,c4,c5 from t1",
+ tablePlan: &tablePlan{
+ selectPks: []int{0, 4},
+ },
+ row: []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ sqltypes.NewVarChar("hi3"),
+ sqltypes.NewVarChar("hi4"),
+ sqltypes.NewInt64(5),
+ },
+ reportOptions: &tabletmanagerdatapb.VDiffReportOptions{},
+ want: &RowDiff{
+ Row: map[string]string{ // The two PK cols should be first
+ "c1": "1", "c5": "5", "c2": "2", "c3": "hi3", "c4": "hi4",
+ },
+ },
+ },
+ {
+ name: "only PKs",
+ schema: &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: "t1",
+ Columns: []string{"c1", "c2"},
+ PrimaryKeyColumns: []string{"c1"},
+ Fields: sqltypes.MakeTestFields("c1|c2", "int64|int64"),
+ },
+ },
+ },
+ query: "select c1,c2 from t1",
+ tablePlan: &tablePlan{
+ selectPks: []int{0},
+ },
+ row: []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ },
+ reportOptions: &tabletmanagerdatapb.VDiffReportOptions{
+ OnlyPks: true,
+ },
+ want: &RowDiff{
+ Row: map[string]string{
+ "c1": "1",
+ },
+ },
+ },
+ {
+ name: "debug query",
+ schema: &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: "t1",
+ Columns: []string{"c1", "c2"},
+ PrimaryKeyColumns: []string{"c1"},
+ Fields: sqltypes.MakeTestFields("c1|c2", "int64|int64"),
+ },
+ },
+ },
+ query: "select c1,c2 from t1",
+ tablePlan: &tablePlan{
+ selectPks: []int{0},
+ },
+ row: []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ },
+ reportOptions: &tabletmanagerdatapb.VDiffReportOptions{
+ DebugQuery: true,
+ },
+ want: &RowDiff{
+ Row: map[string]string{
+ "c1": "1",
+ "c2": "2",
+ },
+ Query: "select c1, c2 from t1 where c1=1;",
+ },
+ },
+ {
+ name: "column truncation",
+ schema: &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{
+ {
+ Name: "t1",
+ Columns: []string{"c1", "c2"},
+ PrimaryKeyColumns: []string{"c1"},
+ Fields: sqltypes.MakeTestFields("c1|c2", "varchar|varchar"),
+ },
+ },
+ },
+ query: "select c1,c2 from t1",
+ tablePlan: &tablePlan{
+ selectPks: []int{0},
+ },
+ row: []sqltypes.Value{
+ sqltypes.NewVarChar(strings.Repeat("a", 100)),
+ sqltypes.NewVarChar(strings.Repeat("b", 100)),
+ },
+ reportOptions: &tabletmanagerdatapb.VDiffReportOptions{
+ RowDiffColumnTruncateAt: 5,
+ },
+ want: &RowDiff{
+ Row: map[string]string{
+ "c1": strings.Repeat("a", 100), // PK fields are not truncated
+ "c2": strings.Repeat("b", 5) + truncatedNotation,
+ },
+ },
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ require.NotNil(t, tc.tablePlan)
+ require.NotNil(t, tc.row)
+ require.NotNil(t, tc.reportOptions)
+
+ vdenv.tmc.schema = tc.schema
+ ct := vdenv.createController(t, 1)
+ wd, err := newWorkflowDiffer(ct, vdenv.opts, collations.MySQL8())
+ require.NoError(t, err)
+ td := &tableDiffer{
+ wd: wd,
+ sourceQuery: tc.query,
+ tablePlan: tc.tablePlan,
+ }
+
+ got, err := td.genRowDiff(tc.query, tc.row, tc.reportOptions)
+ require.NoError(t, err)
+ require.EqualValues(t, tc.want, got)
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletmanager/vdiff/schema.go b/go/vt/vttablet/tabletmanager/vdiff/schema.go
index afb79b4e4b3..9fef975e1e0 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/schema.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/schema.go
@@ -22,6 +22,9 @@ const (
sqlResumeVDiff = `update _vt.vdiff as vd, _vt.vdiff_table as vdt set vd.started_at = NULL, vd.completed_at = NULL, vd.state = 'pending',
vdt.state = 'pending' where vd.vdiff_uuid = %a and vd.id = vdt.vdiff_id and vd.state in ('completed', 'stopped')
and vdt.state in ('completed', 'stopped')`
+ sqlStartVDiff = `update _vt.vdiff as vd set vd.state = 'pending' where vd.vdiff_uuid = %a and vd.state = 'stopped' and
+ vd.started_at is NULL and vd.completed_at is NULL and
+ (select count(*) as cnt from _vt.vdiff_table as vdt where vd.id = vdt.vdiff_id) = 0`
sqlRetryVDiff = `update _vt.vdiff as vd left join _vt.vdiff_table as vdt on (vd.id = vdt.vdiff_id) set vd.state = 'pending',
vd.last_error = '', vdt.state = 'pending' where vd.id = %a and (vd.state = 'error' or vdt.state = 'error')`
sqlGetVDiffByKeyspaceWorkflowUUID = "select * from _vt.vdiff where keyspace = %a and workflow = %a and vdiff_uuid = %a"
diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go
index f91a82b9d2c..102d7535af9 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/table_differ.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/table_differ.go
@@ -486,7 +486,7 @@ func (td *tableDiffer) setupRowSorters() {
}
}
-func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onlyPks bool, maxExtraRowsToCompare int64, maxReportSampleRows int64, stop <-chan time.Time) (*DiffReport, error) {
+func (td *tableDiffer) diff(ctx context.Context, coreOpts *tabletmanagerdatapb.VDiffCoreOptions, reportOpts *tabletmanagerdatapb.VDiffReportOptions, stop <-chan time.Time) (*DiffReport, error) {
defer td.wd.ct.TableDiffPhaseTimings.Record(fmt.Sprintf("%s.%s", td.table.Name, diffingTable), time.Now())
dbClient := td.wd.ct.dbClientFactory()
if err := dbClient.Connect(); err != nil {
@@ -539,6 +539,10 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
globalStats.RowsDiffedCount.Add(dr.ProcessedRows)
}()
+ rowsToCompare := coreOpts.GetMaxRows()
+ maxExtraRowsToCompare := coreOpts.GetMaxExtraRowsToCompare()
+ maxReportSampleRows := reportOpts.GetMaxSampleRows()
+
for {
lastProcessedRow = sourceRow
@@ -560,6 +564,7 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
return nil, err
}
}
+
rowsToCompare--
if rowsToCompare < 0 {
log.Infof("Stopping vdiff, specified row limit reached")
@@ -587,7 +592,7 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
advanceSource = true
advanceTarget = true
if sourceRow == nil {
- diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, targetRow, debug, onlyPks)
+ diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, targetRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
@@ -605,7 +610,7 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
if targetRow == nil {
// No more rows from the target but we know we have more rows from
// source, so drain them and update the counts.
- diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, sourceRow, debug, onlyPks)
+ diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, sourceRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
@@ -628,7 +633,7 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
return nil, err
case c < 0:
if dr.ExtraRowsSource < maxExtraRowsToCompare {
- diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, sourceRow, debug, onlyPks)
+ diffRow, err := td.genRowDiff(td.tablePlan.sourceQuery, sourceRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
@@ -639,7 +644,7 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
continue
case c > 0:
if dr.ExtraRowsTarget < maxExtraRowsToCompare {
- diffRow, err := td.genRowDiff(td.tablePlan.targetQuery, targetRow, debug, onlyPks)
+ diffRow, err := td.genRowDiff(td.tablePlan.targetQuery, targetRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
@@ -659,11 +664,11 @@ func (td *tableDiffer) diff(ctx context.Context, rowsToCompare int64, debug, onl
case c != 0:
// We don't do a second pass to compare mismatched rows so we can cap the slice here.
if maxReportSampleRows == 0 || dr.MismatchedRows < maxReportSampleRows {
- sourceDiffRow, err := td.genRowDiff(td.tablePlan.targetQuery, sourceRow, debug, onlyPks)
+ sourceDiffRow, err := td.genRowDiff(td.tablePlan.targetQuery, sourceRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
- targetDiffRow, err := td.genRowDiff(td.tablePlan.targetQuery, targetRow, debug, onlyPks)
+ targetDiffRow, err := td.genRowDiff(td.tablePlan.targetQuery, targetRow, reportOpts)
if err != nil {
return nil, vterrors.Wrap(err, "unexpected error generating diff")
}
diff --git a/go/vt/vttablet/tabletmanager/vdiff/table_plan.go b/go/vt/vttablet/tabletmanager/vdiff/table_plan.go
index becce6f90e6..836df8ffe94 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/table_plan.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/table_plan.go
@@ -30,6 +30,7 @@ import (
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/engine"
"vitess.io/vitess/go/vt/vtgate/engine/opcode"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
querypb "vitess.io/vitess/go/vt/proto/query"
tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
@@ -54,17 +55,19 @@ type tablePlan struct {
pkCols []int
// selectPks is the list of pk columns as they appear in the select clause for the diff.
- selectPks []int
- dbName string
- table *tabletmanagerdatapb.TableDefinition
- orderBy sqlparser.OrderBy
- aggregates []*engine.AggregateParams
+ selectPks []int
+ dbName string
+ table *tabletmanagerdatapb.TableDefinition
+ orderBy sqlparser.OrderBy
+ aggregates []*engine.AggregateParams
+ WorkflowConfig **vttablet.VReplicationConfig
}
func (td *tableDiffer) buildTablePlan(dbClient binlogplayer.DBClient, dbName string, collationEnv *collations.Environment) (*tablePlan, error) {
tp := &tablePlan{
- table: td.table,
- dbName: dbName,
+ table: td.table,
+ dbName: dbName,
+ WorkflowConfig: td.wd.WorkflowConfig,
}
statement, err := td.wd.ct.vde.parser.Parse(td.sourceQuery)
if err != nil {
diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go
index 8c00b61b784..ef30d8f14b0 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ.go
@@ -24,8 +24,6 @@ import (
"strings"
"time"
- "vitess.io/vitess/go/vt/schema"
-
"google.golang.org/protobuf/encoding/prototext"
"vitess.io/vitess/go/mysql/collations"
@@ -33,10 +31,12 @@ import (
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/log"
+ "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"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -53,15 +53,18 @@ type workflowDiffer struct {
tableDiffers map[string]*tableDiffer // key is table name
opts *tabletmanagerdatapb.VDiffOptions
- collationEnv *collations.Environment
+ collationEnv *collations.Environment
+ WorkflowConfig **vttablet.VReplicationConfig
}
func newWorkflowDiffer(ct *controller, opts *tabletmanagerdatapb.VDiffOptions, collationEnv *collations.Environment) (*workflowDiffer, error) {
+ vttablet.InitVReplicationConfigDefaults()
wd := &workflowDiffer{
- ct: ct,
- opts: opts,
- tableDiffers: make(map[string]*tableDiffer, 1),
- collationEnv: collationEnv,
+ ct: ct,
+ opts: opts,
+ tableDiffers: make(map[string]*tableDiffer, 1),
+ collationEnv: collationEnv,
+ WorkflowConfig: &vttablet.DefaultVReplicationConfig,
}
return wd, nil
}
@@ -205,7 +208,7 @@ func (wd *workflowDiffer) diffTable(ctx context.Context, dbClient binlogplayer.D
}
log.Infof("Table initialization done on table %s for vdiff %s", td.table.Name, wd.ct.uuid)
diffTimer = time.NewTimer(maxDiffRuntime)
- diffReport, diffErr = td.diff(ctx, wd.opts.CoreOptions.MaxRows, wd.opts.ReportOptions.DebugQuery, wd.opts.ReportOptions.OnlyPks, wd.opts.CoreOptions.MaxExtraRowsToCompare, wd.opts.ReportOptions.MaxSampleRows, diffTimer.C)
+ diffReport, diffErr = td.diff(ctx, wd.opts.CoreOptions, wd.opts.ReportOptions, diffTimer.C)
if diffErr == nil { // We finished the diff successfully
break
}
diff --git a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go
index d4f9ddb001d..c51b6fdf281 100644
--- a/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go
+++ b/go/vt/vttablet/tabletmanager/vdiff/workflow_differ_test.go
@@ -574,6 +574,7 @@ func TestBuildPlanSuccess(t *testing.T) {
err = wd.buildPlan(dbc, filter, testSchema)
require.NoError(t, err, tcase.input)
require.Equal(t, 1, len(wd.tableDiffers), tcase.input)
+ wd.tableDiffers[tcase.table].tablePlan.WorkflowConfig = nil
assert.Equal(t, tcase.tablePlan, wd.tableDiffers[tcase.table].tablePlan, tcase.input)
// Confirm that the options are passed through.
@@ -588,15 +589,7 @@ func TestBuildPlanInclude(t *testing.T) {
vdenv := newTestVDiffEnv(t)
defer vdenv.close()
- controllerQR := sqltypes.MakeTestResult(sqltypes.MakeTestFields(
- vdiffTestCols,
- vdiffTestColTypes,
- ),
- fmt.Sprintf("1|%s|%s|%s|%s|%s|%s|%s|", uuid.New(), vdiffenv.workflow, tstenv.KeyspaceName, tstenv.ShardName, vdiffDBName, PendingState, optionsJS),
- )
- vdiffenv.dbClient.ExpectRequest("select * from _vt.vdiff where id = 1", noResults, nil)
- ct, err := newController(context.Background(), controllerQR.Named().Row(), vdiffenv.dbClientFactory, tstenv.TopoServ, vdiffenv.vde, vdiffenv.opts)
- require.NoError(t, err)
+ ct := vdenv.createController(t, 1)
schm := &tabletmanagerdatapb.SchemaDefinition{
TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{
diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go
index 581244eebb3..7067211ff10 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/controller.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go
@@ -18,6 +18,7 @@ package vreplication
import (
"context"
+ "encoding/json"
"fmt"
"strconv"
"strings"
@@ -31,12 +32,14 @@ import (
"vitess.io/vitess/go/vt/discovery"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/mysqlctl"
+ "vitess.io/vitess/go/vt/proto/vtctldata"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
const (
@@ -44,6 +47,9 @@ const (
// give up and return an error message that the user
// can see and act upon if needed.
tabletPickerRetries = 5
+
+ // Prepended to the message to indicate that it is a terminal error.
+ TerminalErrorIndicator = "terminal error"
)
// controller is created by Engine. Members are initialized upfront.
@@ -68,15 +74,36 @@ type controller struct {
sourceTablet atomic.Value
lastWorkflowError *vterrors.LastError
+ WorkflowConfig *vttablet.VReplicationConfig
+}
+
+func processWorkflowOptions(params map[string]string) (*vttablet.VReplicationConfig, error) {
+ options, ok := params["options"]
+ if !ok {
+ options = "{}"
+ }
+ var workflowOptions vtctldata.WorkflowOptions
+ if err := json.Unmarshal([]byte(options), &workflowOptions); err != nil {
+ return nil, fmt.Errorf("failed to parse options column: %v", err)
+ }
+ workflowConfig, err := vttablet.NewVReplicationConfig(workflowOptions.Config)
+ if err != nil {
+ return nil, fmt.Errorf("failed to process config options: %v", err)
+ }
+ return workflowConfig, nil
}
// newController creates a new controller. Unless a stream is explicitly 'Stopped',
// this function launches a goroutine to perform continuous vreplication.
-func newController(ctx context.Context, params map[string]string, dbClientFactory func() binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon, ts *topo.Server, cell, tabletTypesStr string, blpStats *binlogplayer.Stats, vre *Engine, tpo discovery.TabletPickerOptions) (*controller, error) {
+func newController(ctx context.Context, params map[string]string, dbClientFactory func() binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon, ts *topo.Server, cell string, blpStats *binlogplayer.Stats, vre *Engine, tpo discovery.TabletPickerOptions) (*controller, error) {
if blpStats == nil {
blpStats = binlogplayer.NewStats()
}
-
+ workflowConfig, err := processWorkflowOptions(params)
+ if err != nil {
+ return nil, err
+ }
+ tabletTypesStr := workflowConfig.TabletTypesStr
ct := &controller{
vre: vre,
dbClientFactory: dbClientFactory,
@@ -84,7 +111,9 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor
blpStats: blpStats,
done: make(chan struct{}),
source: &binlogdatapb.BinlogSource{},
+ WorkflowConfig: workflowConfig,
}
+ blpStats.WorkflowConfig = workflowConfig.String()
ct.sourceTablet.Store(&topodatapb.TabletAlias{})
log.Infof("creating controller with cell: %v, tabletTypes: %v, and params: %v", cell, tabletTypesStr, params)
@@ -94,7 +123,7 @@ func newController(ctx context.Context, params map[string]string, dbClientFactor
}
ct.id = int32(id)
ct.workflow = params["workflow"]
- ct.lastWorkflowError = vterrors.NewLastError(fmt.Sprintf("VReplication controller %d for workflow %q", ct.id, ct.workflow), maxTimeToRetryError)
+ ct.lastWorkflowError = vterrors.NewLastError(fmt.Sprintf("VReplication controller %d for workflow %q", ct.id, ct.workflow), workflowConfig.MaxTimeToRetryError)
state := params["state"]
blpStats.State.Store(state)
@@ -164,8 +193,8 @@ func (ct *controller) run(ctx context.Context) {
}
ct.blpStats.ErrorCounts.Add([]string{"Stream Error"}, 1)
- binlogplayer.LogError(fmt.Sprintf("error in stream %v, will retry after %v", ct.id, retryDelay), err)
- timer := time.NewTimer(retryDelay)
+ binlogplayer.LogError(fmt.Sprintf("error in stream %v, will retry after %v", ct.id, ct.WorkflowConfig.RetryDelay), err)
+ timer := time.NewTimer(ct.WorkflowConfig.RetryDelay)
select {
case <-ctx.Done():
log.Warningf("context canceled: %s", err.Error())
@@ -176,6 +205,38 @@ func (ct *controller) run(ctx context.Context) {
}
}
+func setDBClientSettings(dbClient binlogplayer.DBClient, workflowConfig *vttablet.VReplicationConfig) error {
+ if workflowConfig == nil {
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "vreplication controller: workflowConfig is nil")
+ }
+ const maxRows = 10000
+ // Timestamp fields from binlogs are always sent as UTC.
+ // So, we should set the timezone to be UTC for those values to be correctly inserted.
+ if _, err := dbClient.ExecuteFetch("set @@session.time_zone = '+00:00'", maxRows); err != nil {
+ return err
+ }
+ // Tables may have varying character sets. To ship the bits without interpreting them
+ // we set the character set to be binary.
+ if _, err := dbClient.ExecuteFetch("set names 'binary'", maxRows); err != nil {
+ return err
+ }
+ if _, err := dbClient.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v",
+ workflowConfig.NetReadTimeout), maxRows); err != nil {
+ return err
+ }
+ if _, err := dbClient.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v",
+ workflowConfig.NetWriteTimeout), maxRows); err != nil {
+ return err
+ }
+ // We must apply AUTO_INCREMENT values precisely as we got them. This include the 0 value, which is
+ // not recommended in AUTO_INCREMENT, and yet is valid.
+ if _, err := dbClient.ExecuteFetch("set @@session.sql_mode = CONCAT(@@session.sql_mode, ',NO_AUTO_VALUE_ON_ZERO')",
+ maxRows); err != nil {
+ return err
+ }
+ return nil
+}
+
func (ct *controller) runBlp(ctx context.Context) (err error) {
defer func() {
ct.sourceTablet.Store(&topodatapb.TabletAlias{})
@@ -217,27 +278,9 @@ func (ct *controller) runBlp(ctx context.Context) (err error) {
player := binlogplayer.NewBinlogPlayerKeyRange(dbClient, tablet, ct.source.KeyRange, ct.id, ct.blpStats)
return player.ApplyBinlogEvents(ctx)
case ct.source.Filter != nil:
- // Timestamp fields from binlogs are always sent as UTC.
- // So, we should set the timezone to be UTC for those values to be correctly inserted.
- if _, err := dbClient.ExecuteFetch("set @@session.time_zone = '+00:00'", 10000); err != nil {
- return err
- }
- // Tables may have varying character sets. To ship the bits without interpreting them
- // we set the character set to be binary.
- if _, err := dbClient.ExecuteFetch("set names 'binary'", 10000); err != nil {
+ if err := setDBClientSettings(dbClient, ct.WorkflowConfig); err != nil {
return err
}
- if _, err := dbClient.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v", vttablet.VReplicationNetReadTimeout), 10000); err != nil {
- return err
- }
- if _, err := dbClient.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v", vttablet.VReplicationNetWriteTimeout), 10000); err != nil {
- return err
- }
- // We must apply AUTO_INCREMENT values precisely as we got them. This include the 0 value, which is not recommended in AUTO_INCREMENT, and yet is valid.
- if _, err := dbClient.ExecuteFetch("set @@session.sql_mode = CONCAT(@@session.sql_mode, ',NO_AUTO_VALUE_ON_ZERO')", 10000); err != nil {
- return err
- }
-
var vsClient VStreamerClient
var err error
if name := ct.source.GetExternalMysql(); name != "" {
@@ -253,7 +296,7 @@ func (ct *controller) runBlp(ctx context.Context) (err error) {
}
defer vsClient.Close(ctx)
- vr := newVReplicator(ct.id, ct.source, vsClient, ct.blpStats, dbClient, ct.mysqld, ct.vre)
+ vr := newVReplicator(ct.id, ct.source, vsClient, ct.blpStats, dbClient, ct.mysqld, ct.vre, ct.WorkflowConfig)
err = vr.Replicate(ctx)
ct.lastWorkflowError.Record(err)
@@ -265,7 +308,7 @@ func (ct *controller) runBlp(ctx context.Context) (err error) {
if (err != nil && vr.WorkflowSubType == int32(binlogdatapb.VReplicationWorkflowSubType_AtomicCopy)) ||
isUnrecoverableError(err) ||
!ct.lastWorkflowError.ShouldRetry() {
-
+ err = vterrors.Wrapf(err, TerminalErrorIndicator)
if errSetState := vr.setState(binlogdatapb.VReplicationWorkflowState_Error, err.Error()); errSetState != nil {
log.Errorf("INTERNAL: unable to setState() in controller: %v. Could not set error text to: %v.", errSetState, err)
return err // yes, err and not errSetState.
diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go
index 57cb60384c6..9a0b6ca833c 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go
@@ -23,6 +23,8 @@ import (
"testing"
"time"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
+
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/discovery"
@@ -70,20 +72,29 @@ var (
defaultTabletPickerOptions = discovery.TabletPickerOptions{}
)
+func setTabletTypesStr(tabletTypesStr string) func() {
+ oldTabletTypesStr := vttablet.DefaultVReplicationConfig.TabletTypesStr
+ vttablet.DefaultVReplicationConfig.TabletTypesStr = tabletTypesStr
+ return func() {
+ vttablet.DefaultVReplicationConfig.TabletTypesStr = oldTabletTypesStr
+ }
+}
+
func TestControllerKeyRange(t *testing.T) {
resetBinlogClient()
wantTablet := addTablet(100)
defer deleteTablet(wantTablet)
params := map[string]string{
- "id": "1",
- "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
- "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "id": "1",
+ "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
+ "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "options": "{}",
}
dbClient := binlogplayer.NewMockDBClient(t)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -94,7 +105,8 @@ func TestControllerKeyRange(t *testing.T) {
mysqld.MysqlPort.Store(3306)
vre := NewTestEngine(nil, wantTablet.GetAlias().Cell, mysqld, dbClientFactory, dbClientFactory, dbClient.DBName(), nil)
- ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil, vre, defaultTabletPickerOptions)
+ defer setTabletTypesStr("replica")()
+ ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -113,15 +125,16 @@ func TestControllerTables(t *testing.T) {
resetBinlogClient()
params := map[string]string{
- "id": "1",
- "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
- "source": fmt.Sprintf(`keyspace:"%s" shard:"0" tables:"table1" tables:"/funtables_/" `, env.KeyspaceName),
+ "id": "1",
+ "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
+ "source": fmt.Sprintf(`keyspace:"%s" shard:"0" tables:"table1" tables:"/funtables_/" `, env.KeyspaceName),
+ "options": "{}",
}
dbClient := binlogplayer.NewMockDBClient(t)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -155,8 +168,8 @@ func TestControllerTables(t *testing.T) {
}
mysqld.MysqlPort.Store(3306)
vre := NewTestEngine(nil, wantTablet.GetAlias().Cell, mysqld, dbClientFactory, dbClientFactory, dbClient.DBName(), nil)
-
- ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil, vre, defaultTabletPickerOptions)
+ defer setTabletTypesStr("replica")()
+ ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -171,9 +184,10 @@ func TestControllerTables(t *testing.T) {
func TestControllerBadID(t *testing.T) {
params := map[string]string{
- "id": "bad",
+ "id": "bad",
+ "options": "{}",
}
- _, err := newController(context.Background(), params, nil, nil, nil, "", "", nil, nil, defaultTabletPickerOptions)
+ _, err := newController(context.Background(), params, nil, nil, nil, "", nil, nil, defaultTabletPickerOptions)
want := `strconv.ParseInt: parsing "bad": invalid syntax`
if err == nil || err.Error() != want {
t.Errorf("newController err: %v, want %v", err, want)
@@ -182,11 +196,12 @@ func TestControllerBadID(t *testing.T) {
func TestControllerStopped(t *testing.T) {
params := map[string]string{
- "id": "1",
- "state": binlogdatapb.VReplicationWorkflowState_Stopped.String(),
+ "id": "1",
+ "state": binlogdatapb.VReplicationWorkflowState_Stopped.String(),
+ "options": "{}",
}
- ct, err := newController(context.Background(), params, nil, nil, nil, "", "", nil, nil, defaultTabletPickerOptions)
+ ct, err := newController(context.Background(), params, nil, nil, nil, "", nil, nil, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -210,12 +225,13 @@ func TestControllerOverrides(t *testing.T) {
"source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
"cell": env.Cells[0],
"tablet_types": "replica",
+ "options": "{}",
}
dbClient := binlogplayer.NewMockDBClient(t)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -226,7 +242,8 @@ func TestControllerOverrides(t *testing.T) {
mysqld.MysqlPort.Store(3306)
vre := NewTestEngine(nil, wantTablet.GetAlias().Cell, mysqld, dbClientFactory, dbClientFactory, dbClient.DBName(), nil)
- ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "rdonly", nil, vre, defaultTabletPickerOptions)
+ defer setTabletTypesStr("rdonly")()
+ ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -244,16 +261,17 @@ func TestControllerCanceledContext(t *testing.T) {
defer deleteTablet(wantTablet)
params := map[string]string{
- "id": "1",
- "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
- "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "id": "1",
+ "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
+ "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "options": "{}",
}
ctx, cancel := context.WithCancel(context.Background())
cancel()
vre := NewTestEngine(nil, wantTablet.GetAlias().Cell, nil, nil, nil, "", nil)
- ct, err := newController(ctx, params, nil, nil, env.TopoServ, env.Cells[0], "rdonly", nil, vre, defaultTabletPickerOptions)
+ ct, err := newController(ctx, params, nil, nil, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -267,9 +285,9 @@ func TestControllerCanceledContext(t *testing.T) {
}
func TestControllerRetry(t *testing.T) {
- savedDelay := retryDelay
- defer func() { retryDelay = savedDelay }()
- retryDelay = 10 * time.Millisecond
+ savedDelay := vttablet.DefaultVReplicationConfig.RetryDelay
+ defer func() { vttablet.DefaultVReplicationConfig.RetryDelay = savedDelay }()
+ vttablet.DefaultVReplicationConfig.RetryDelay = 10 * time.Millisecond
resetBinlogClient()
defer deleteTablet(addTablet(100))
@@ -280,16 +298,17 @@ func TestControllerRetry(t *testing.T) {
"source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
"cell": env.Cells[0],
"tablet_types": "replica",
+ "options": "{}",
}
dbClient := binlogplayer.NewMockDBClient(t)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", nil, errors.New("(expected error)"))
- dbClient.ExpectRequest("update _vt.vreplication set state='Error', message='error (expected error) in selecting vreplication settings select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1' where id=1", testDMLResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, nil, errors.New("(expected error)"))
+ dbClient.ExpectRequest("update _vt.vreplication set state='Error', message='error (expected error) in selecting vreplication settings select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys, options from _vt.vreplication where id=1' where id=1", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -299,7 +318,8 @@ func TestControllerRetry(t *testing.T) {
mysqld.MysqlPort.Store(3306)
vre := NewTestEngine(nil, env.Cells[0], mysqld, dbClientFactory, dbClientFactory, dbClient.DBName(), nil)
- ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "rdonly", nil, vre, defaultTabletPickerOptions)
+ defer setTabletTypesStr("rdonly")()
+ ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
@@ -314,9 +334,10 @@ func TestControllerStopPosition(t *testing.T) {
defer deleteTablet(wantTablet)
params := map[string]string{
- "id": "1",
- "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
- "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "id": "1",
+ "state": binlogdatapb.VReplicationWorkflowState_Running.String(),
+ "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ "options": "{}",
}
dbClient := binlogplayer.NewMockDBClient(t)
@@ -349,7 +370,7 @@ func TestControllerStopPosition(t *testing.T) {
},
},
}
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", withStop, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, withStop, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -361,7 +382,7 @@ func TestControllerStopPosition(t *testing.T) {
mysqld.MysqlPort.Store(3306)
vre := NewTestEngine(nil, wantTablet.GetAlias().Cell, mysqld, dbClientFactory, dbClientFactory, dbClient.DBName(), nil)
- ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil, vre, defaultTabletPickerOptions)
+ ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], nil, vre, defaultTabletPickerOptions)
if err != nil {
t.Fatal(err)
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go
index d407bfe403b..e692781538a 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/engine.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go
@@ -229,6 +229,10 @@ func (vre *Engine) Open(ctx context.Context) {
log.Infof("VReplication engine opened successfully")
}
+func (vre *Engine) ThrottlerClient() *throttle.Client {
+ return vre.throttlerClient
+}
+
func (vre *Engine) openLocked(ctx context.Context) error {
rows, err := vre.readAllRows(ctx)
if err != nil {
@@ -282,9 +286,9 @@ func (vre *Engine) retry(ctx context.Context, err error) {
func (vre *Engine) initControllers(rows []map[string]string) {
for _, row := range rows {
- ct, err := newController(vre.ctx, row, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, tabletTypesStr, nil, vre, discovery.TabletPickerOptions{})
+ ct, err := newController(vre.ctx, row, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, nil, vre, discovery.TabletPickerOptions{})
if err != nil {
- log.Errorf("Controller could not be initialized for stream: %v", row)
+ log.Errorf("Controller could not be initialized for stream: %v: %v", row, err)
continue
}
vre.controllers[ct.id] = ct
@@ -405,8 +409,6 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error)
return nil, fmt.Errorf("insert id %v out of range", qr.InsertID)
}
- vdbc := newVDBClient(dbClient, stats)
-
// If we are creating multiple streams, for example in a
// merge workflow going from 2 shards to 1 shard, we
// will be inserting multiple rows. To get the ids of
@@ -430,11 +432,12 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error)
if err != nil {
return nil, err
}
- ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, tabletTypesStr, nil, vre, plan.tabletPickerOptions)
+ ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, nil, vre, plan.tabletPickerOptions)
if err != nil {
return nil, err
}
vre.controllers[id] = ct
+ vdbc := newVDBClient(dbClient, stats, ct.WorkflowConfig.RelayLogMaxSize)
insertLogWithParams(vdbc, LogStreamCreate, id, params)
}
return qr, nil
@@ -462,7 +465,6 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error)
if err != nil {
return nil, err
}
- vdbc := newVDBClient(dbClient, stats)
for _, id := range ids {
params, err := readRow(dbClient, id)
if err != nil {
@@ -470,11 +472,12 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error)
}
// Create a new controller in place of the old one.
// For continuity, the new controller inherits the previous stats.
- ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, tabletTypesStr, blpStats[id], vre, plan.tabletPickerOptions)
+ ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, blpStats[id], vre, plan.tabletPickerOptions)
if err != nil {
return nil, err
}
vre.controllers[id] = ct
+ vdbc := newVDBClient(dbClient, stats, ct.WorkflowConfig.RelayLogMaxSize)
insertLog(vdbc, LogStateChange, id, params["state"], "")
}
return qr, nil
@@ -487,13 +490,13 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error)
return &sqltypes.Result{}, nil
}
// Stop and delete the current controllers.
- vdbc := newVDBClient(dbClient, stats)
for _, id := range ids {
if ct := vre.controllers[id]; ct != nil {
+ vdbc := newVDBClient(dbClient, stats, ct.WorkflowConfig.RelayLogMaxSize)
ct.Stop()
delete(vre.controllers, id)
+ insertLogWithParams(vdbc, LogStreamDelete, id, nil)
}
- insertLogWithParams(vdbc, LogStreamDelete, id, nil)
}
if err := dbClient.Begin(); err != nil {
return nil, err
@@ -640,7 +643,7 @@ func (vre *Engine) transitionJournal(je *journalEvent) {
log.Infof("Transitioning for journal:workload %v", je)
- //sort both participants and shardgtids
+ // sort both participants and shardgtids
participants := make([]string, 0)
for ks := range je.participants {
participants = append(participants, ks)
@@ -691,7 +694,7 @@ func (vre *Engine) transitionJournal(je *journalEvent) {
deferSecondaryKeys, _ := strconv.ParseBool(params["defer_secondary_keys"])
ig := NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Running, vre.dbName)
ig.AddRow(params["workflow"], bls, sgtid.Gtid, params["cell"], params["tablet_types"],
- binlogdatapb.VReplicationWorkflowType(workflowType), binlogdatapb.VReplicationWorkflowSubType(workflowSubType), deferSecondaryKeys)
+ binlogdatapb.VReplicationWorkflowType(workflowType), binlogdatapb.VReplicationWorkflowSubType(workflowSubType), deferSecondaryKeys, "")
qr, err := dbClient.ExecuteFetch(ig.String(), maxRows)
if err != nil {
log.Errorf("transitionJournal: %v", err)
@@ -730,7 +733,7 @@ func (vre *Engine) transitionJournal(je *journalEvent) {
log.Errorf("transitionJournal: %v", err)
return
}
- ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, tabletTypesStr, nil, vre, discovery.TabletPickerOptions{})
+ ct, err := newController(vre.ctx, params, vre.dbClientFactoryFiltered, vre.mysqld, vre.ts, vre.cell, nil, vre, discovery.TabletPickerOptions{})
if err != nil {
log.Errorf("transitionJournal: %v", err)
return
diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go
index ea46e126895..af054932c89 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go
@@ -50,14 +50,14 @@ func TestEngineOpen(t *testing.T) {
dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|state|source|tablet_types",
- "int64|varchar|varchar|varbinary",
+ "id|state|source|tablet_types|options",
+ "int64|varchar|varchar|varbinary|varchar",
),
- fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}|PRIMARY,REPLICA`, env.KeyspaceName),
+ fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}|PRIMARY,REPLICA|{}`, env.KeyspaceName),
), nil)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -93,8 +93,8 @@ func TestEngineOpenRetry(t *testing.T) {
dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", nil, errors.New("err"))
dbClient.ExpectRequest("select * from _vt.vreplication where db_name='db'", sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|state|source",
- "int64|varchar|varchar",
+ "id|state|source|options",
+ "int64|varchar|varchar|varchar",
),
), nil)
@@ -161,14 +161,14 @@ func TestEngineExec(t *testing.T) {
dbClient.ExpectRequest("select @@session.auto_increment_increment", &sqltypes.Result{}, nil)
dbClient.ExpectRequest("select * from _vt.vreplication where id = 1", sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|state|source|tablet_types",
- "int64|varchar|varchar|varbinary",
+ "id|state|source|tablet_types|options",
+ "int64|varchar|varchar|varbinary|varchar",
),
- fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}|PRIMARY,REPLICA`, env.KeyspaceName),
+ fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}|PRIMARY,REPLICA|{}`, env.KeyspaceName),
), nil)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
@@ -187,6 +187,7 @@ func TestEngineExec(t *testing.T) {
ct := vre.controllers[1]
if ct == nil || ct.id != 1 {
t.Errorf("ct: %v, id should be 1", ct)
+ return
}
// Verify stats
@@ -203,14 +204,14 @@ func TestEngineExec(t *testing.T) {
dbClient.ExpectRequest("update _vt.vreplication set pos = 'MariaDB/0-1-1084', state = 'Running' where id in (1)", testDMLResponse, nil)
dbClient.ExpectRequest("select * from _vt.vreplication where id = 1", sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
- "id|state|source",
- "int64|varchar|varchar",
+ "id|state|source|options",
+ "int64|varchar|varchar|varchar",
),
- fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}`, env.KeyspaceName),
+ fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range:{end:"\x80"}|{}`, env.KeyspaceName),
), nil)
dbClient.ExpectRequestRE("update _vt.vreplication set message='Picked source tablet.*", testDMLResponse, nil)
dbClient.ExpectRequest("update _vt.vreplication set state='Running', message='' where id=1", testDMLResponse, nil)
- dbClient.ExpectRequest("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1", testSettingsResponse, nil)
+ dbClient.ExpectRequest(binlogplayer.TestGetWorkflowQueryId1, testSettingsResponse, nil)
dbClient.ExpectRequest("begin", nil, nil)
dbClient.ExpectRequest("insert into t values(1)", testDMLResponse, nil)
dbClient.ExpectRequestRE("update _vt.vreplication set pos='MariaDB/0-1-1235', time_updated=.*", testDMLResponse, nil)
diff --git a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go
index c53bfd2a584..c14d792da04 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/external_connector.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/external_connector.go
@@ -48,13 +48,16 @@ type VStreamerClient interface {
Close(context.Context) error
// VStream streams VReplication events based on the specified filter.
- VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error
+ VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter,
+ send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error
// VStreamRows streams rows of a table from the specified starting point.
- VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error
+ VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult,
+ send func(*binlogdatapb.VStreamRowsResponse) error, options *binlogdatapb.VStreamOptions) error
// VStreamTables streams rows of a table from the specified starting point.
- VStreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error
+ VStreamTables(ctx context.Context,
+ send func(*binlogdatapb.VStreamTablesResponse) error, options *binlogdatapb.VStreamOptions) error
}
type externalConnector struct {
@@ -110,7 +113,7 @@ func (ec *externalConnector) Get(name string) (*mysqlConnector, error) {
return c, nil
}
-//-----------------------------------------------------------
+// -----------------------------------------------------------
type mysqlConnector struct {
env tabletenv.Env
@@ -131,11 +134,13 @@ func (c *mysqlConnector) Close(ctx context.Context) error {
return nil
}
-func (c *mysqlConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error {
- return c.vstreamer.Stream(ctx, startPos, tablePKs, filter, throttlerapp.ExternalConnectorName, send)
+func (c *mysqlConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error {
+ return c.vstreamer.Stream(ctx, startPos, tablePKs, filter, throttlerapp.ExternalConnectorName, send, options)
}
-func (c *mysqlConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error {
+func (c *mysqlConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult,
+ send func(*binlogdatapb.VStreamRowsResponse) error, options *binlogdatapb.VStreamOptions) error {
var row []sqltypes.Value
if lastpk != nil {
r := sqltypes.Proto3ToResult(lastpk)
@@ -144,14 +149,15 @@ func (c *mysqlConnector) VStreamRows(ctx context.Context, query string, lastpk *
}
row = r.Rows[0]
}
- return c.vstreamer.StreamRows(ctx, query, row, send)
+ return c.vstreamer.StreamRows(ctx, query, row, send, options)
}
-func (c *mysqlConnector) VStreamTables(ctx context.Context, send func(response *binlogdatapb.VStreamTablesResponse) error) error {
- return c.vstreamer.StreamTables(ctx, send)
+func (c *mysqlConnector) VStreamTables(ctx context.Context,
+ send func(response *binlogdatapb.VStreamTablesResponse) error, options *binlogdatapb.VStreamOptions) error {
+ return c.vstreamer.StreamTables(ctx, send, options)
}
-//-----------------------------------------------------------
+// -----------------------------------------------------------
type tabletConnector struct {
tablet *topodatapb.Tablet
@@ -180,17 +186,20 @@ func (tc *tabletConnector) Close(ctx context.Context) error {
return tc.qs.Close(ctx)
}
-func (tc *tabletConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error {
- req := &binlogdatapb.VStreamRequest{Target: tc.target, Position: startPos, TableLastPKs: tablePKs, Filter: filter}
+func (tc *tabletConnector) VStream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error {
+ req := &binlogdatapb.VStreamRequest{Target: tc.target, Position: startPos, TableLastPKs: tablePKs, Filter: filter, Options: options}
return tc.qs.VStream(ctx, req, send)
}
-func (tc *tabletConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult, send func(*binlogdatapb.VStreamRowsResponse) error) error {
- req := &binlogdatapb.VStreamRowsRequest{Target: tc.target, Query: query, Lastpk: lastpk}
+func (tc *tabletConnector) VStreamRows(ctx context.Context, query string, lastpk *querypb.QueryResult,
+ send func(*binlogdatapb.VStreamRowsResponse) error, options *binlogdatapb.VStreamOptions) error {
+ req := &binlogdatapb.VStreamRowsRequest{Target: tc.target, Query: query, Lastpk: lastpk, Options: options}
return tc.qs.VStreamRows(ctx, req, send)
}
-func (tc *tabletConnector) VStreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error {
- req := &binlogdatapb.VStreamTablesRequest{Target: tc.target}
+func (tc *tabletConnector) VStreamTables(ctx context.Context,
+ send func(*binlogdatapb.VStreamTablesResponse) error, options *binlogdatapb.VStreamOptions) error {
+ req := &binlogdatapb.VStreamTablesRequest{Target: tc.target, Options: options}
return tc.qs.VStreamTables(ctx, req, send)
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/external_connector_test.go b/go/vt/vttablet/tabletmanager/vreplication/external_connector_test.go
index 24e6cb0dbc3..c671d2a086d 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/external_connector_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/external_connector_test.go
@@ -64,7 +64,7 @@ func TestExternalConnectorCopy(t *testing.T) {
expectDBClientAndVreplicationQueries(t, []string{
"begin",
- "insert into tab1(id,val) values (1,'a'), (2,'b')",
+ "insert into tab1(id,val) values (1,_binary'a'), (2,_binary'b')",
"/insert into _vt.copy_state",
"commit",
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name",
@@ -73,7 +73,7 @@ func TestExternalConnectorCopy(t *testing.T) {
execStatements(t, []string{"insert into tab1 values(3, 'c')"})
expectDBClientQueries(t, qh.Expect(
"begin",
- "insert into tab1(id,val) values (3,'c')",
+ "insert into tab1(id,val) values (3,_binary'c')",
"/update _vt.vreplication set pos=",
"commit",
))
@@ -96,7 +96,7 @@ func TestExternalConnectorCopy(t *testing.T) {
expectDBClientAndVreplicationQueries(t, []string{
"begin",
- "insert into tab2(id,val) values (1,'a'), (2,'b')",
+ "insert into tab2(id,val) values (1,_binary'a'), (2,_binary'b')",
"/insert into _vt.copy_state",
"commit",
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name",
@@ -121,7 +121,7 @@ func TestExternalConnectorCopy(t *testing.T) {
expectDBClientAndVreplicationQueries(t, []string{
"begin",
- "insert into tab3(id,val) values (1,'a'), (2,'b')",
+ "insert into tab3(id,val) values (1,_binary'a'), (2,_binary'b')",
"/insert into _vt.copy_state",
"commit",
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name",
@@ -163,8 +163,7 @@ func TestExternalConnectorPlay(t *testing.T) {
expectDBClientAndVreplicationQueries(t, []string{
"begin",
- "insert into tab1(id,val) values (1,'a')",
- "insert into tab1(id,val) values (2,'b')",
+ "insert into tab1(id,val) values (1,_binary'a'), (2,_binary'b')",
"/update _vt.vreplication set pos=",
"commit",
}, pos)
diff --git a/go/vt/vttablet/tabletmanager/vreplication/flags.go b/go/vt/vttablet/tabletmanager/vreplication/flags.go
deleted file mode 100644
index e45158ab99a..00000000000
--- a/go/vt/vttablet/tabletmanager/vreplication/flags.go
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
-Copyright 2022 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 vreplication
-
-import (
- "time"
-
- "github.com/spf13/pflag"
-
- "vitess.io/vitess/go/vt/servenv"
-)
-
-var (
- retryDelay = 5 * time.Second
- maxTimeToRetryError time.Duration // Default behavior is to keep retrying, for backward compatibility
-
- tabletTypesStr = "in_order:REPLICA,PRIMARY" // Default value
-
- relayLogMaxSize = 250000
- relayLogMaxItems = 5000
-
- replicaLagTolerance = 1 * time.Minute
-
- vreplicationHeartbeatUpdateInterval = 1
-
- vreplicationStoreCompressedGTID = false
- vreplicationParallelInsertWorkers = 1
-)
-
-func registerVReplicationFlags(fs *pflag.FlagSet) {
- fs.DurationVar(&retryDelay, "vreplication_retry_delay", retryDelay, "delay before retrying a failed workflow event in the replication phase")
- fs.DurationVar(&maxTimeToRetryError, "vreplication_max_time_to_retry_on_error", maxTimeToRetryError, "stop automatically retrying when we've had consecutive failures with the same error for this long after the first occurrence")
-
- fs.IntVar(&relayLogMaxSize, "relay_log_max_size", relayLogMaxSize, "Maximum buffer size (in bytes) for VReplication target buffering. If single rows are larger than this, a single row is buffered at a time.")
- fs.IntVar(&relayLogMaxItems, "relay_log_max_items", relayLogMaxItems, "Maximum number of rows for VReplication target buffering.")
-
- fs.DurationVar(&replicaLagTolerance, "vreplication_replica_lag_tolerance", replicaLagTolerance, "Replica lag threshold duration: once lag is below this we switch from copy phase to the replication (streaming) phase")
-
- // vreplicationHeartbeatUpdateInterval determines how often the time_updated column is updated if there are no real events on the source and the source
- // vstream is only sending heartbeats for this long. Keep this low if you expect high QPS and are monitoring this column to alert about potential
- // outages. Keep this high if
- // you have too many streams the extra write qps or cpu load due to these updates are unacceptable
- // you have too many streams and/or a large source field (lot of participating tables) which generates unacceptable increase in your binlog size
- fs.IntVar(&vreplicationHeartbeatUpdateInterval, "vreplication_heartbeat_update_interval", vreplicationHeartbeatUpdateInterval, "Frequency (in seconds, default 1, max 60) at which the time_updated column of a vreplication stream when idling")
- fs.BoolVar(&vreplicationStoreCompressedGTID, "vreplication_store_compressed_gtid", vreplicationStoreCompressedGTID, "Store compressed gtids in the pos column of the sidecar database's vreplication table")
-
- fs.IntVar(&vreplicationParallelInsertWorkers, "vreplication-parallel-insert-workers", vreplicationParallelInsertWorkers, "Number of parallel insertion workers to use during copy phase. Set <= 1 to disable parallelism, or > 1 to enable concurrent insertion during copy phase.")
-}
-
-func init() {
- servenv.OnParseFor("vtcombo", registerVReplicationFlags)
- servenv.OnParseFor("vttablet", registerVReplicationFlags)
-}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go
index ef80dd49757..12a05a69dbc 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go
@@ -47,7 +47,7 @@ import (
"vitess.io/vitess/go/vt/sidecardb"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/queryservice"
"vitess.io/vitess/go/vt/vttablet/queryservice/fakes"
"vitess.io/vitess/go/vt/vttablet/tabletconn"
@@ -140,7 +140,7 @@ func setup(ctx context.Context) (func(), int) {
globalDBQueries = make(chan string, 1000)
resetBinlogClient()
- vttablet.VReplicationExperimentalFlags = 0
+ vttablet.InitVReplicationConfigDefaults()
// Engines cannot be initialized in testenv because it introduces circular dependencies.
streamerEngine = vstreamer.NewEngine(env.TabletEnv, env.SrvTopo, env.SchemaEngine, nil, env.Cells[0])
@@ -237,7 +237,7 @@ func execConnStatements(t *testing.T, conn *dbconnpool.DBConnection, queries []s
}
}
-//--------------------------------------
+// --------------------------------------
// Topos and tablets
func addTablet(id int) *topodatapb.Tablet {
@@ -319,7 +319,7 @@ func (ftc *fakeTabletConn) VStream(ctx context.Context, request *binlogdatapb.VS
if vstreamHook != nil {
vstreamHook(ctx)
}
- return streamerEngine.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send)
+ return streamerEngine.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send, nil)
}
// vstreamRowsHook allows you to do work just before calling VStreamRows.
@@ -341,15 +341,18 @@ func (ftc *fakeTabletConn) VStreamRows(ctx context.Context, request *binlogdatap
}
row = r.Rows[0]
}
+ vstreamOptions := &binlogdatapb.VStreamOptions{
+ ConfigOverrides: vttablet.GetVReplicationConfigDefaults(false).Map(),
+ }
return streamerEngine.StreamRows(ctx, request.Query, row, func(rows *binlogdatapb.VStreamRowsResponse) error {
if vstreamRowsSendHook != nil {
vstreamRowsSendHook(ctx)
}
return send(rows)
- })
+ }, vstreamOptions)
}
-//--------------------------------------
+// --------------------------------------
// Binlog Client to TabletManager
// fakeBinlogClient satisfies binlogplayer.Client.
@@ -426,7 +429,7 @@ func expectFBCRequest(t *testing.T, tablet *topodatapb.Tablet, pos string, table
}
}
-//--------------------------------------
+// --------------------------------------
// DBCLient wrapper
func realDBClientFactory() binlogplayer.DBClient {
@@ -475,6 +478,10 @@ func (dbc *realDBClient) Close() {
dbc.conn.Close()
}
+func (dbc *realDBClient) IsClosed() bool {
+ return dbc.conn.IsClosed()
+}
+
func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) {
// Use Clone() because the contents of memory region referenced by
// string can change when clients (e.g. vcopier) use unsafe string methods.
@@ -487,7 +494,7 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resu
globalDBQueries <- query
} else if testSetForeignKeyQueries && strings.Contains(query, "set foreign_key_checks") {
globalDBQueries <- query
- } else if testForeignKeyQueries && strings.Contains(query, "foreign_key_checks") { //allow select/set for foreign_key_checks
+ } else if testForeignKeyQueries && strings.Contains(query, "foreign_key_checks") { // allow select/set for foreign_key_checks
globalDBQueries <- query
}
return qr, err
@@ -684,7 +691,7 @@ func expectNontxQueries(t *testing.T, expectations qh.ExpectationSequence, recvT
}
result := validator.AcceptQuery(got)
-
+ require.NotNil(t, result)
require.True(t, result.Accepted, fmt.Sprintf(
"query:%q\nmessage:%s\nexpectation:%s\nmatched:%t\nerror:%v\nhistory:%s",
got, result.Message, result.Expectation, result.Matched, result.Error, validator.History(),
diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go
index 6a127b084b5..a43278d783c 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator.go
@@ -51,7 +51,10 @@ func NewInsertGenerator(state binlogdatapb.VReplicationWorkflowState, dbname str
// AddRow adds a row to the insert statement.
func (ig *InsertGenerator) AddRow(workflow string, bls *binlogdatapb.BinlogSource, pos, cell, tabletTypes string,
- workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType, deferSecondaryKeys bool) {
+ workflowType binlogdatapb.VReplicationWorkflowType, workflowSubType binlogdatapb.VReplicationWorkflowSubType, deferSecondaryKeys bool, options string) {
+ if options == "" {
+ options = "'{}'"
+ }
protoutil.SortBinlogSourceTables(bls)
fmt.Fprintf(ig.buf, "%s(%v, %v, %v, %v, %v, %v, %v, %v, 0, '%v', %v, %d, %d, %v, %v)",
ig.prefix,
@@ -68,7 +71,7 @@ func (ig *InsertGenerator) AddRow(workflow string, bls *binlogdatapb.BinlogSourc
workflowType,
workflowSubType,
deferSecondaryKeys,
- "'{}'",
+ options,
)
ig.prefix = ", "
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go
index 92100429963..30c626ec02e 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/insert_generator_test.go
@@ -27,12 +27,12 @@ import (
func TestInsertGenerator(t *testing.T) {
ig := NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, "a")
ig.now = 111
- ig.AddRow("b", &binlogdatapb.BinlogSource{Keyspace: "c"}, "d", "e", "f", binlogdatapb.VReplicationWorkflowType_Materialize, binlogdatapb.VReplicationWorkflowSubType_None, false)
+ ig.AddRow("b", &binlogdatapb.BinlogSource{Keyspace: "c"}, "d", "e", "f", binlogdatapb.VReplicationWorkflowType_Materialize, binlogdatapb.VReplicationWorkflowSubType_None, false, "")
want := `insert into _vt.vreplication(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys, options) values ` +
`('b', 'keyspace:"c"', 'd', 9223372036854775807, 9223372036854775807, 'e', 'f', 111, 0, 'Stopped', 'a', 0, 0, false, '{}')`
assert.Equal(t, ig.String(), want)
- ig.AddRow("g", &binlogdatapb.BinlogSource{Keyspace: "h"}, "i", "j", "k", binlogdatapb.VReplicationWorkflowType_Reshard, binlogdatapb.VReplicationWorkflowSubType_Partial, true)
+ ig.AddRow("g", &binlogdatapb.BinlogSource{Keyspace: "h"}, "i", "j", "k", binlogdatapb.VReplicationWorkflowType_Reshard, binlogdatapb.VReplicationWorkflowSubType_Partial, true, "")
want += `, ('g', 'keyspace:"h"', 'i', 9223372036854775807, 9223372036854775807, 'j', 'k', 111, 0, 'Stopped', 'a', 4, 1, true, '{}')`
assert.Equal(t, ig.String(), want)
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/queryhistory/verifier.go b/go/vt/vttablet/tabletmanager/vreplication/queryhistory/verifier.go
index ebe145461d7..aeddf5344d5 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/queryhistory/verifier.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/queryhistory/verifier.go
@@ -72,7 +72,7 @@ func (v *Verifier) AcceptQuery(query string) *Result {
break
}
}
-
+ result.Message += fmt.Sprintf(" (query: %q) (pending: %q)", query, v.Pending())
return result
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/replica_connector.go b/go/vt/vttablet/tabletmanager/vreplication/replica_connector.go
index c3cd073f0bf..f26b5054d01 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/replica_connector.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/replica_connector.go
@@ -58,7 +58,7 @@ func NewReplicaConnector(venv *vtenv.Environment, connParams *mysql.ConnParams)
return c
}
-//-----------------------------------------------------------
+// -----------------------------------------------------------
type ReplicaConnector struct {
conn *mysql.ConnParams
@@ -73,5 +73,5 @@ func (c *ReplicaConnector) Close() error {
}
func (c *ReplicaConnector) VStream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error {
- return c.vstreamer.Stream(ctx, startPos, nil, filter, throttlerapp.ReplicaConnectorName, send)
+ return c.vstreamer.Stream(ctx, startPos, nil, filter, throttlerapp.ReplicaConnectorName, send, nil)
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
index 3bef997d0be..62d6166b5ca 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan.go
@@ -27,12 +27,13 @@ import (
"vitess.io/vitess/go/mysql/collations/charset"
"vitess.io/vitess/go/mysql/collations/colldata"
vjson "vitess.io/vitess/go/mysql/json"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/evalengine"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -53,13 +54,14 @@ import (
// of the members, leaving the original plan unchanged.
// The constructor is buildReplicatorPlan in table_plan_builder.go
type ReplicatorPlan struct {
- VStreamFilter *binlogdatapb.Filter
- TargetTables map[string]*TablePlan
- TablePlans map[string]*TablePlan
- ColInfoMap map[string][]*ColumnInfo
- stats *binlogplayer.Stats
- Source *binlogdatapb.BinlogSource
- collationEnv *collations.Environment
+ VStreamFilter *binlogdatapb.Filter
+ TargetTables map[string]*TablePlan
+ TablePlans map[string]*TablePlan
+ ColInfoMap map[string][]*ColumnInfo
+ stats *binlogplayer.Stats
+ Source *binlogdatapb.BinlogSource
+ collationEnv *collations.Environment
+ workflowConfig *vttablet.VReplicationConfig
}
// buildExecution plan uses the field info as input and the partially built
@@ -88,7 +90,7 @@ func (rp *ReplicatorPlan) buildExecutionPlan(fieldEvent *binlogdatapb.FieldEvent
// select * construct was used. We need to use the field names.
tplan, err := rp.buildFromFields(prelim.TargetName, prelim.Lastpk, fieldEvent.Fields)
if err != nil {
- return nil, err
+ return nil, vterrors.Wrapf(err, "failed to build replication plan for %s table", fieldEvent.TableName)
}
tplan.Fields = fieldEvent.Fields
return tplan, nil
@@ -99,28 +101,28 @@ func (rp *ReplicatorPlan) buildExecutionPlan(fieldEvent *binlogdatapb.FieldEvent
// requires us to wait for the field info sent by the source.
func (rp *ReplicatorPlan) buildFromFields(tableName string, lastpk *sqltypes.Result, fields []*querypb.Field) (*TablePlan, error) {
tpb := &tablePlanBuilder{
- name: sqlparser.NewIdentifierCS(tableName),
- lastpk: lastpk,
- colInfos: rp.ColInfoMap[tableName],
- stats: rp.stats,
- source: rp.Source,
- collationEnv: rp.collationEnv,
+ name: sqlparser.NewIdentifierCS(tableName),
+ lastpk: lastpk,
+ colInfos: rp.ColInfoMap[tableName],
+ stats: rp.stats,
+ source: rp.Source,
+ collationEnv: rp.collationEnv,
+ workflowConfig: rp.workflowConfig,
}
for _, field := range fields {
colName := sqlparser.NewIdentifierCI(field.Name)
- isGenerated := false
+ generated := false
+ // We have to loop over the columns in the plan as the columns between the
+ // source and target are not always 1 to 1.
for _, colInfo := range tpb.colInfos {
if !strings.EqualFold(colInfo.Name, field.Name) {
continue
}
if colInfo.IsGenerated {
- isGenerated = true
+ generated = true
}
break
}
- if isGenerated {
- continue
- }
cexpr := &colExpr{
colName: colName,
colType: field.Type,
@@ -130,6 +132,7 @@ func (rp *ReplicatorPlan) buildFromFields(tableName string, lastpk *sqltypes.Res
references: map[string]bool{
field.Name: true,
},
+ isGenerated: generated,
}
tpb.colExprs = append(tpb.colExprs, cexpr)
}
@@ -219,7 +222,8 @@ type TablePlan struct {
// PartialUpdates are same as PartialInserts, but for update statements
PartialUpdates map[string]*sqlparser.ParsedQuery
- CollationEnv *collations.Environment
+ CollationEnv *collations.Environment
+ WorkflowConfig *vttablet.VReplicationConfig
}
// MarshalJSON performs a custom JSON Marshalling.
@@ -257,7 +261,7 @@ func (tp *TablePlan) applyBulkInsert(sqlbuffer *bytes2.Buffer, rows []*querypb.R
if i > 0 {
sqlbuffer.WriteString(", ")
}
- if err := appendFromRow(tp.BulkInsertValues, sqlbuffer, tp.Fields, row, tp.FieldsToSkip); err != nil {
+ if err := tp.appendFromRow(sqlbuffer, row); err != nil {
return nil, err
}
}
@@ -285,7 +289,7 @@ func (tp *TablePlan) applyBulkInsert(sqlbuffer *bytes2.Buffer, rows []*querypb.R
// now and punt on the others.
func (tp *TablePlan) isOutsidePKRange(bindvars map[string]*querypb.BindVariable, before, after bool, stmtType string) bool {
// added empty comments below, otherwise gofmt removes the spaces between the bitwise & and obfuscates this check!
- if vttablet.VReplicationExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagOptimizeInserts == 0 {
+ if tp.WorkflowConfig.ExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagOptimizeInserts == 0 {
return false
}
// Ensure there is one and only one value in lastpk and pkrefs.
@@ -296,7 +300,7 @@ func (tp *TablePlan) isOutsidePKRange(bindvars map[string]*querypb.BindVariable,
case !before && after:
bindvar = bindvars["a_"+tp.PKReferences[0]]
}
- if bindvar == nil { //should never happen
+ if bindvar == nil { // should never happen
return false
}
@@ -312,6 +316,30 @@ func (tp *TablePlan) isOutsidePKRange(bindvars map[string]*querypb.BindVariable,
return false
}
+// convertStringCharset does a charset conversion given raw data and an applicable conversion rule.
+// In case of a conversion error, it returns an equivalent of MySQL error 1366, which is what you'd
+// get in a failed `CONVERT()` function, e.g.:
+//
+// > create table tascii(v varchar(100) charset ascii);
+// > insert into tascii values ('€');
+// ERROR 1366 (HY000): Incorrect string value: '\xE2\x82\xAC' for column 'v' at row 1
+func (tp *TablePlan) convertStringCharset(raw []byte, conversion *binlogdatapb.CharsetConversion, fieldName string) ([]byte, error) {
+ fromCollation := tp.CollationEnv.DefaultCollationForCharset(conversion.FromCharset)
+ if fromCollation == collations.Unknown {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "character set %s not supported for column %s", conversion.FromCharset, fieldName)
+ }
+ toCollation := tp.CollationEnv.DefaultCollationForCharset(conversion.ToCharset)
+ if toCollation == collations.Unknown {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "character set %s not supported for column %s", conversion.ToCharset, fieldName)
+ }
+
+ out, err := charset.Convert(nil, colldata.Lookup(toCollation).Charset(), raw, colldata.Lookup(fromCollation).Charset())
+ if err != nil {
+ return nil, sqlerror.NewSQLErrorf(sqlerror.ERTruncatedWrongValueForField, sqlerror.SSUnknownSQLState, "Incorrect string value: %s", err.Error())
+ }
+ return out, nil
+}
+
// bindFieldVal returns a bind variable based on given field and value.
// Most values will just bind directly. But some values may need manipulation:
// - text values with charset conversion
@@ -320,11 +348,7 @@ func (tp *TablePlan) isOutsidePKRange(bindvars map[string]*querypb.BindVariable,
func (tp *TablePlan) bindFieldVal(field *querypb.Field, val *sqltypes.Value) (*querypb.BindVariable, error) {
if conversion, ok := tp.ConvertCharset[field.Name]; ok && !val.IsNull() {
// Non-null string value, for which we have a charset conversion instruction
- fromCollation := tp.CollationEnv.DefaultCollationForCharset(conversion.FromCharset)
- if fromCollation == collations.Unknown {
- return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "Character set %s not supported for column %s", conversion.FromCharset, field.Name)
- }
- out, err := charset.Convert(nil, charset.Charset_utf8mb4{}, val.Raw(), colldata.Lookup(fromCollation).Charset())
+ out, err := tp.convertStringCharset(val.Raw(), conversion, field.Name)
if err != nil {
return nil, err
}
@@ -584,56 +608,50 @@ func valsEqual(v1, v2 sqltypes.Value) bool {
return v1.ToString() == v2.ToString()
}
-// AppendFromRow behaves like Append but takes a querypb.Row directly, assuming that
-// the fields in the row are in the same order as the placeholders in this query. The fields might include generated
-// columns which are dropped, by checking against skipFields, before binding the variables
-// note: there can be more fields than bind locations since extra columns might be requested from the source if not all
-// primary keys columns are present in the target table, for example. Also some values in the row may not correspond for
-// values from the database on the source: sum/count for aggregation queries, for example
-func appendFromRow(pq *sqlparser.ParsedQuery, buf *bytes2.Buffer, fields []*querypb.Field, row *querypb.Row, skipFields map[string]bool) error {
- bindLocations := pq.BindLocations()
- if len(fields) < len(bindLocations) {
- return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "wrong number of fields: got %d fields for %d bind locations ",
- len(fields), len(bindLocations))
- }
-
- type colInfo struct {
- typ querypb.Type
- length int64
- offset int64
- }
- rowInfo := make([]*colInfo, 0)
-
- offset := int64(0)
- for i, field := range fields { // collect info required for fields to be bound
- length := row.Lengths[i]
- if !skipFields[strings.ToLower(field.Name)] {
- rowInfo = append(rowInfo, &colInfo{
- typ: field.Type,
- length: length,
- offset: offset,
- })
- }
- if length > 0 {
- offset += row.Lengths[i]
+// AppendFromRow behaves like Append but takes a querypb.Row directly, assuming that the
+// fields in the row are in the same order as the placeholders in this query. The fields
+// might include generated columns which are dropped before binding the variables note:
+// there can be more fields than bind locations since extra columns might be requested
+// from the source if not all primary keys columns are present in the target table, for
+// example. Also some values in the row may not correspond for values from the database
+// on the source: sum/count for aggregation queries, for example.
+func (tp *TablePlan) appendFromRow(buf *bytes2.Buffer, row *querypb.Row) error {
+ bindLocations := tp.BulkInsertValues.BindLocations()
+ if len(tp.Fields) < len(bindLocations) {
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "wrong number of fields: got %d fields for %d bind locations",
+ len(tp.Fields), len(bindLocations))
+ }
+
+ // Bind field values to locations.
+ var (
+ offset int64
+ offsetQuery int
+ fieldsIndex int
+ field *querypb.Field
+ )
+ for i, loc := range bindLocations {
+ field = tp.Fields[fieldsIndex]
+ length := row.Lengths[fieldsIndex]
+ for tp.FieldsToSkip[strings.ToLower(field.Name)] {
+ if length > 0 {
+ offset += length
+ }
+ fieldsIndex++
+ field = tp.Fields[fieldsIndex]
+ length = row.Lengths[fieldsIndex]
}
- }
- // bind field values to locations
- var offsetQuery int
- for i, loc := range bindLocations {
- col := rowInfo[i]
- buf.WriteString(pq.Query[offsetQuery:loc.Offset])
- typ := col.typ
+ buf.WriteString(tp.BulkInsertValues.Query[offsetQuery:loc.Offset])
+ typ := field.Type
switch typ {
case querypb.Type_TUPLE:
return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected Type_TUPLE for value %d", i)
case querypb.Type_JSON:
- if col.length < 0 { // An SQL NULL and not an actual JSON value
+ if length < 0 { // An SQL NULL and not an actual JSON value
buf.WriteString(sqltypes.NullStr)
} else { // A JSON value (which may be a JSON null literal value)
- buf2 := row.Values[col.offset : col.offset+col.length]
+ buf2 := row.Values[offset : offset+length]
vv, err := vjson.MarshalSQLValue(buf2)
if err != nil {
return err
@@ -641,16 +659,33 @@ func appendFromRow(pq *sqlparser.ParsedQuery, buf *bytes2.Buffer, fields []*quer
buf.WriteString(vv.RawStr())
}
default:
- if col.length < 0 {
+ if length < 0 {
// -1 means a null variable; serialize it directly
buf.WriteString(sqltypes.NullStr)
} else {
- vv := sqltypes.MakeTrusted(typ, row.Values[col.offset:col.offset+col.length])
+ raw := row.Values[offset : offset+length]
+ var vv sqltypes.Value
+
+ if conversion, ok := tp.ConvertCharset[field.Name]; ok && length > 0 {
+ // Non-null string value, for which we have a charset conversion instruction
+ out, err := tp.convertStringCharset(raw, conversion, field.Name)
+ if err != nil {
+ return err
+ }
+ vv = sqltypes.MakeTrusted(typ, out)
+ } else {
+ vv = sqltypes.MakeTrusted(typ, raw)
+ }
+
vv.EncodeSQLBytes2(buf)
}
}
offsetQuery = loc.Offset + loc.Length
+ if length > 0 {
+ offset += length
+ }
+ fieldsIndex++
}
- buf.WriteString(pq.Query[offsetQuery:])
+ buf.WriteString(tp.BulkInsertValues.Query[offsetQuery:])
return nil
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go
index 6c9f92128ac..09ace916f11 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/replicator_plan_test.go
@@ -24,12 +24,15 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/bytes2"
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/sqlparser"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
)
type TestReplicatorPlan struct {
@@ -574,7 +577,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "bad query",
}},
},
- err: "syntax error at position 4 near 'bad' in query: bad query",
+ err: "failed to build table replication plan for t1 table: syntax error at position 4 near 'bad' in query: bad query",
}, {
// not a select
input: &binlogdatapb.Filter{
@@ -583,7 +586,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "update t1 set val = 1",
}},
},
- err: "unsupported non-select statement in query: update t1 set val = 1",
+ err: "failed to build table replication plan for t1 table: unsupported non-select statement in query: update t1 set val = 1",
}, {
// no distinct
input: &binlogdatapb.Filter{
@@ -592,7 +595,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select distinct c1 from t1",
}},
},
- err: "unsupported distinct clause in query: select distinct c1 from t1",
+ err: "failed to build table replication plan for t1 table: unsupported distinct clause in query: select distinct c1 from t1",
}, {
// no ',' join
input: &binlogdatapb.Filter{
@@ -601,7 +604,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select * from t1, t2",
}},
},
- err: "unsupported multi-table usage in query: select * from t1, t2",
+ err: "failed to build table replication plan for t1 table: unsupported multi-table usage in query: select * from t1, t2",
}, {
// no join
input: &binlogdatapb.Filter{
@@ -610,7 +613,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select * from t1 join t2",
}},
},
- err: "unsupported from expression (*sqlparser.JoinTableExpr) in query: select * from t1 join t2",
+ err: "failed to build table replication plan for t1 table: unsupported from expression (*sqlparser.JoinTableExpr) in query: select * from t1 join t2",
}, {
// no subqueries
input: &binlogdatapb.Filter{
@@ -619,7 +622,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select * from (select * from t2) as a",
}},
},
- err: "unsupported from source (*sqlparser.DerivedTable) in query: select * from (select * from t2) as a",
+ err: "failed to build table replication plan for t1 table: unsupported from source (*sqlparser.DerivedTable) in query: select * from (select * from t2) as a",
}, {
// cannot combine '*' with other
input: &binlogdatapb.Filter{
@@ -628,7 +631,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select *, c1 from t1",
}},
},
- err: "unsupported mix of '*' and columns in query: select *, c1 from t1",
+ err: "failed to build table replication plan for t1 table: unsupported mix of '*' and columns in query: select *, c1 from t1",
}, {
// cannot combine '*' with other (different code path)
input: &binlogdatapb.Filter{
@@ -637,7 +640,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select c1, * from t1",
}},
},
- err: "invalid expression: * in query: select c1, * from t1",
+ err: "failed to build table replication plan for t1 table: invalid expression: * in query: select c1, * from t1",
}, {
// no distinct in func
input: &binlogdatapb.Filter{
@@ -646,7 +649,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select hour(distinct c1) as a from t1",
}},
},
- err: "syntax error at position 21 near 'distinct' in query: select hour(distinct c1) as a from t1",
+ err: "failed to build table replication plan for t1 table: syntax error at position 21 near 'distinct' in query: select hour(distinct c1) as a from t1",
}, {
// funcs need alias
input: &binlogdatapb.Filter{
@@ -655,7 +658,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select hour(c1) from t1",
}},
},
- err: "expression needs an alias: hour(c1) in query: select hour(c1) from t1",
+ err: "failed to build table replication plan for t1 table: expression needs an alias: hour(c1) in query: select hour(c1) from t1",
}, {
// only count(*)
input: &binlogdatapb.Filter{
@@ -664,7 +667,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select count(c1) as c from t1",
}},
},
- err: "only count(*) is supported: count(c1) in query: select count(c1) as c from t1",
+ err: "failed to build table replication plan for t1 table: only count(*) is supported: count(c1) in query: select count(c1) as c from t1",
}, {
// no sum(*)
input: &binlogdatapb.Filter{
@@ -673,7 +676,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select sum(*) as c from t1",
}},
},
- err: "syntax error at position 13 in query: select sum(*) as c from t1",
+ err: "failed to build table replication plan for t1 table: syntax error at position 13 in query: select sum(*) as c from t1",
}, {
// sum should have only one argument
input: &binlogdatapb.Filter{
@@ -682,7 +685,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select sum(a, b) as c from t1",
}},
},
- err: "syntax error at position 14 in query: select sum(a, b) as c from t1",
+ err: "failed to build table replication plan for t1 table: syntax error at position 14 in query: select sum(a, b) as c from t1",
}, {
// no complex expr in sum
input: &binlogdatapb.Filter{
@@ -691,7 +694,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select sum(a + b) as c from t1",
}},
},
- err: "unsupported non-column name in sum clause: sum(a + b) in query: select sum(a + b) as c from t1",
+ err: "failed to build table replication plan for t1 table: unsupported non-column name in sum clause: sum(a + b) in query: select sum(a + b) as c from t1",
}, {
// no complex expr in group by
input: &binlogdatapb.Filter{
@@ -700,7 +703,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select a from t1 group by a + 1",
}},
},
- err: "unsupported non-column name or alias in group by clause: a + 1 in query: select a from t1 group by a + 1",
+ err: "failed to build table replication plan for t1 table: unsupported non-column name or alias in group by clause: a + 1 in query: select a from t1 group by a + 1",
}, {
// group by does not reference alias
input: &binlogdatapb.Filter{
@@ -709,7 +712,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select a as b from t1 group by a",
}},
},
- err: "group by expression does not reference an alias in the select list: a in query: select a as b from t1 group by a",
+ err: "failed to build table replication plan for t1 table: group by expression does not reference an alias in the select list: a in query: select a as b from t1 group by a",
}, {
// cannot group by aggr
input: &binlogdatapb.Filter{
@@ -718,7 +721,7 @@ func TestBuildPlayerPlan(t *testing.T) {
Filter: "select count(*) as a from t1 group by a",
}},
},
- err: "group by expression is not allowed to reference an aggregate expression: a in query: select count(*) as a from t1 group by a",
+ err: "failed to build table replication plan for t1 table: group by expression is not allowed to reference an aggregate expression: a in query: select count(*) as a from t1 group by a",
}}
PrimaryKeyInfos := map[string][]*ColumnInfo{
@@ -735,8 +738,12 @@ func TestBuildPlayerPlan(t *testing.T) {
),
}
+ vttablet.InitVReplicationConfigDefaults()
for _, tcase := range testcases {
- plan, err := buildReplicatorPlan(getSource(tcase.input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
+ vr := &vreplicator{
+ workflowConfig: vttablet.DefaultVReplicationConfig,
+ }
+ plan, err := vr.buildReplicatorPlan(getSource(tcase.input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
gotErr := ""
if err != nil {
gotErr = err.Error()
@@ -745,8 +752,7 @@ func TestBuildPlayerPlan(t *testing.T) {
gotPlan, _ := json.Marshal(plan)
wantPlan, _ := json.Marshal(tcase.plan)
require.Equal(t, string(wantPlan), string(gotPlan), "Filter(%v):\n%s, want\n%s", tcase.input, gotPlan, wantPlan)
-
- plan, err = buildReplicatorPlan(getSource(tcase.input), PrimaryKeyInfos, copyState, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
+ plan, err = vr.buildReplicatorPlan(getSource(tcase.input), PrimaryKeyInfos, copyState, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
if err != nil {
continue
}
@@ -774,7 +780,10 @@ func TestBuildPlayerPlanNoDup(t *testing.T) {
Filter: "select * from t",
}},
}
- _, err := buildReplicatorPlan(getSource(input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
+ vr := &vreplicator{
+ workflowConfig: vttablet.DefaultVReplicationConfig,
+ }
+ _, err := vr.buildReplicatorPlan(getSource(input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
want := "more than one target for source table t"
if err == nil || !strings.Contains(err.Error(), want) {
t.Errorf("buildReplicatorPlan err: %v, must contain: %v", err, want)
@@ -795,7 +804,10 @@ func TestBuildPlayerPlanExclude(t *testing.T) {
Filter: "",
}},
}
- plan, err := buildReplicatorPlan(getSource(input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
+ vr := &vreplicator{
+ workflowConfig: vttablet.DefaultVReplicationConfig,
+ }
+ plan, err := vr.buildReplicatorPlan(getSource(input), PrimaryKeyInfos, nil, binlogplayer.NewStats(), collations.MySQL8(), sqlparser.NewTestParser())
assert.NoError(t, err)
want := &TestReplicatorPlan{
@@ -818,3 +830,137 @@ func TestBuildPlayerPlanExclude(t *testing.T) {
wantPlan, _ := json.Marshal(want)
assert.Equal(t, string(gotPlan), string(wantPlan))
}
+
+func TestAppendFromRow(t *testing.T) {
+ testCases := []struct {
+ name string
+ tp *TablePlan
+ row *querypb.Row
+ want string
+ wantErr string
+ }{
+ {
+ name: "simple",
+ tp: &TablePlan{
+ BulkInsertValues: sqlparser.BuildParsedQuery("values (%a, %a, %a)",
+ ":c1", ":c2", ":c3",
+ ),
+ Fields: []*querypb.Field{
+ {Name: "c1", Type: querypb.Type_INT32},
+ {Name: "c2", Type: querypb.Type_INT32},
+ {Name: "c3", Type: querypb.Type_INT32},
+ },
+ },
+ row: sqltypes.RowToProto3(
+ []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ sqltypes.NewInt64(3),
+ },
+ ),
+ want: "values (1, 2, 3)",
+ },
+ {
+ name: "too few fields",
+ tp: &TablePlan{
+ BulkInsertValues: sqlparser.BuildParsedQuery("values (%a, %a, %a)",
+ ":c1", ":c2", ":c3",
+ ),
+ Fields: []*querypb.Field{
+ {Name: "c1", Type: querypb.Type_INT32},
+ {Name: "c2", Type: querypb.Type_INT32},
+ },
+ },
+ wantErr: "wrong number of fields: got 2 fields for 3 bind locations",
+ },
+ {
+ name: "skip half",
+ tp: &TablePlan{
+ BulkInsertValues: sqlparser.BuildParsedQuery("values (%a, %a, %a, %a)",
+ ":c1", ":c2", ":c4", ":c8",
+ ),
+ Fields: []*querypb.Field{
+ {Name: "c1", Type: querypb.Type_INT32},
+ {Name: "c2", Type: querypb.Type_INT32},
+ {Name: "c3", Type: querypb.Type_INT32},
+ {Name: "c4", Type: querypb.Type_INT32},
+ {Name: "c5", Type: querypb.Type_INT32},
+ {Name: "c6", Type: querypb.Type_INT32},
+ {Name: "c7", Type: querypb.Type_INT32},
+ {Name: "c8", Type: querypb.Type_INT32},
+ },
+ FieldsToSkip: map[string]bool{
+ "c3": true,
+ "c5": true,
+ "c6": true,
+ "c7": true,
+ },
+ },
+ row: sqltypes.RowToProto3(
+ []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ sqltypes.NewInt64(3),
+ sqltypes.NewInt64(4),
+ sqltypes.NewInt64(5),
+ sqltypes.NewInt64(6),
+ sqltypes.NewInt64(7),
+ sqltypes.NewInt64(8),
+ },
+ ),
+ want: "values (1, 2, 4, 8)",
+ },
+ {
+ name: "skip all but one",
+ tp: &TablePlan{
+ BulkInsertValues: sqlparser.BuildParsedQuery("values (%a)",
+ ":c4",
+ ),
+ Fields: []*querypb.Field{
+ {Name: "c1", Type: querypb.Type_INT32},
+ {Name: "c2", Type: querypb.Type_INT32},
+ {Name: "c3", Type: querypb.Type_INT32},
+ {Name: "c4", Type: querypb.Type_INT32},
+ {Name: "c5", Type: querypb.Type_INT32},
+ {Name: "c6", Type: querypb.Type_INT32},
+ {Name: "c7", Type: querypb.Type_INT32},
+ {Name: "c8", Type: querypb.Type_INT32},
+ },
+ FieldsToSkip: map[string]bool{
+ "c1": true,
+ "c2": true,
+ "c3": true,
+ "c5": true,
+ "c6": true,
+ "c7": true,
+ "c8": true,
+ },
+ },
+ row: sqltypes.RowToProto3(
+ []sqltypes.Value{
+ sqltypes.NewInt64(1),
+ sqltypes.NewInt64(2),
+ sqltypes.NewInt64(3),
+ sqltypes.NewInt64(4),
+ sqltypes.NewInt64(5),
+ sqltypes.NewInt64(6),
+ sqltypes.NewInt64(7),
+ sqltypes.NewInt64(8),
+ },
+ ),
+ want: "values (4)",
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ bb := &bytes2.Buffer{}
+ err := tc.tp.appendFromRow(bb, tc.row)
+ if tc.wantErr != "" {
+ require.EqualError(t, err, tc.wantErr)
+ } else {
+ require.NoError(t, err)
+ require.Equal(t, tc.want, bb.String())
+ }
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/stats.go b/go/vt/vttablet/tabletmanager/vreplication/stats.go
index 11f458d9541..1aaae3b84ec 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/stats.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/stats.go
@@ -543,6 +543,15 @@ func (st *vrStats) register() {
}
return result
})
+ stats.Publish("VReplicationConfig", stats.StringMapFunc(func() map[string]string {
+ st.mu.Lock()
+ defer st.mu.Unlock()
+ result := make(map[string]string, len(st.controllers))
+ for _, ct := range st.controllers {
+ result[fmt.Sprintf("%s.%d", ct.workflow, ct.id)] = ct.WorkflowConfig.String()
+ }
+ return result
+ }))
}
func (st *vrStats) numControllers() int64 {
diff --git a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
index 2e44fd49e9b..b8a86b94de5 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/table_plan_builder.go
@@ -29,11 +29,11 @@ import (
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
)
// This file contains just the builders for ReplicatorPlan and TablePlan.
@@ -61,7 +61,8 @@ type tablePlanBuilder struct {
source *binlogdatapb.BinlogSource
pkIndices []bool
- collationEnv *collations.Environment
+ collationEnv *collations.Environment
+ workflowConfig *vttablet.VReplicationConfig
}
// colExpr describes the processing to be performed to
@@ -79,10 +80,11 @@ type colExpr struct {
// references contains all the column names referenced in the expression.
references map[string]bool
- isGrouped bool
- isPK bool
- dataType string
- columnType string
+ isGrouped bool
+ isPK bool
+ isGenerated bool
+ dataType string
+ columnType string
}
// operation is the opcode for the colExpr.
@@ -131,16 +133,17 @@ const (
// The TablePlan built is a partial plan. The full plan for a table is built
// when we receive field information from events or rows sent by the source.
// buildExecutionPlan is the function that builds the full plan.
-func buildReplicatorPlan(source *binlogdatapb.BinlogSource, colInfoMap map[string][]*ColumnInfo, copyState map[string]*sqltypes.Result, stats *binlogplayer.Stats, collationEnv *collations.Environment, parser *sqlparser.Parser) (*ReplicatorPlan, error) {
+func (vr *vreplicator) buildReplicatorPlan(source *binlogdatapb.BinlogSource, colInfoMap map[string][]*ColumnInfo, copyState map[string]*sqltypes.Result, stats *binlogplayer.Stats, collationEnv *collations.Environment, parser *sqlparser.Parser) (*ReplicatorPlan, error) {
filter := source.Filter
plan := &ReplicatorPlan{
- VStreamFilter: &binlogdatapb.Filter{FieldEventMode: filter.FieldEventMode},
- TargetTables: make(map[string]*TablePlan),
- TablePlans: make(map[string]*TablePlan),
- ColInfoMap: colInfoMap,
- stats: stats,
- Source: source,
- collationEnv: collationEnv,
+ VStreamFilter: &binlogdatapb.Filter{FieldEventMode: filter.FieldEventMode},
+ TargetTables: make(map[string]*TablePlan),
+ TablePlans: make(map[string]*TablePlan),
+ ColInfoMap: colInfoMap,
+ stats: stats,
+ Source: source,
+ collationEnv: collationEnv,
+ workflowConfig: vr.workflowConfig,
}
for tableName := range colInfoMap {
lastpk, ok := copyState[tableName]
@@ -159,9 +162,9 @@ func buildReplicatorPlan(source *binlogdatapb.BinlogSource, colInfoMap map[strin
if !ok {
return nil, fmt.Errorf("table %s not found in schema", tableName)
}
- tablePlan, err := buildTablePlan(tableName, rule, colInfos, lastpk, stats, source, collationEnv, parser)
+ tablePlan, err := buildTablePlan(tableName, rule, colInfos, lastpk, stats, source, collationEnv, parser, vr.workflowConfig)
if err != nil {
- return nil, err
+ return nil, vterrors.Wrapf(err, "failed to build table replication plan for %s table", tableName)
}
if tablePlan == nil {
// Table was excluded.
@@ -199,7 +202,8 @@ func MatchTable(tableName string, filter *binlogdatapb.Filter) (*binlogdatapb.Ru
}
func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*ColumnInfo, lastpk *sqltypes.Result,
- stats *binlogplayer.Stats, source *binlogdatapb.BinlogSource, collationEnv *collations.Environment, parser *sqlparser.Parser) (*TablePlan, error) {
+ stats *binlogplayer.Stats, source *binlogdatapb.BinlogSource, collationEnv *collations.Environment,
+ parser *sqlparser.Parser, workflowConfig *vttablet.VReplicationConfig) (*TablePlan, error) {
planError := func(err error, query string) error {
// Use the error string here to ensure things are uniform across
@@ -248,6 +252,7 @@ func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*Colum
ConvertCharset: rule.ConvertCharset,
ConvertIntToEnum: rule.ConvertIntToEnum,
CollationEnv: collationEnv,
+ WorkflowConfig: workflowConfig,
}
return tablePlan, nil
@@ -259,11 +264,12 @@ func buildTablePlan(tableName string, rule *binlogdatapb.Rule, colInfos []*Colum
From: sel.From,
Where: sel.Where,
},
- lastpk: lastpk,
- colInfos: colInfos,
- stats: stats,
- source: source,
- collationEnv: collationEnv,
+ lastpk: lastpk,
+ colInfos: colInfos,
+ stats: stats,
+ source: source,
+ collationEnv: collationEnv,
+ workflowConfig: workflowConfig,
}
if err := tpb.analyzeExprs(sel.SelectExprs); err != nil {
@@ -355,10 +361,9 @@ func (tpb *tablePlanBuilder) generate() *TablePlan {
fieldsToSkip := make(map[string]bool)
for _, colInfo := range tpb.colInfos {
if colInfo.IsGenerated {
- fieldsToSkip[colInfo.Name] = true
+ fieldsToSkip[strings.ToLower(colInfo.Name)] = true
}
}
-
return &TablePlan{
TargetName: tpb.name.String(),
Lastpk: tpb.lastpk,
@@ -378,6 +383,7 @@ func (tpb *tablePlanBuilder) generate() *TablePlan {
PartialInserts: make(map[string]*sqlparser.ParsedQuery, 0),
PartialUpdates: make(map[string]*sqlparser.ParsedQuery, 0),
CollationEnv: tpb.collationEnv,
+ WorkflowConfig: tpb.workflowConfig,
}
}
@@ -598,7 +604,7 @@ func (tpb *tablePlanBuilder) analyzePK(cols []*ColumnInfo) error {
// TODO(shlomi): at some point in the futue we want to make this check stricter.
// We could be reading a generated column c1 which in turn selects some other column c2.
// We will want t oensure that `c2` is found in select list...
- return fmt.Errorf("primary key column %v not found in select list", col)
+ return fmt.Errorf("primary key column %v not found in table's select filter or the TableMap event within the GTID", col)
}
if cexpr.operation != opExpr {
return fmt.Errorf("primary key column %v is not allowed to reference an aggregate expression", col)
@@ -689,7 +695,7 @@ func (tpb *tablePlanBuilder) generateInsertPart(buf *sqlparser.TrackedBuffer) *s
}
separator := ""
for _, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
buf.Myprintf("%s%v", separator, cexpr.colName)
@@ -703,7 +709,7 @@ func (tpb *tablePlanBuilder) generateValuesPart(buf *sqlparser.TrackedBuffer, bv
bvf.mode = bvAfter
separator := "("
for _, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
buf.Myprintf("%s", separator)
@@ -740,7 +746,7 @@ func (tpb *tablePlanBuilder) generateSelectPart(buf *sqlparser.TrackedBuffer, bv
buf.WriteString(" select ")
separator := ""
for _, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
buf.Myprintf("%s", separator)
@@ -776,7 +782,7 @@ func (tpb *tablePlanBuilder) generateOnDupPart(buf *sqlparser.TrackedBuffer) *sq
if cexpr.isGrouped || cexpr.isPK {
continue
}
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
buf.Myprintf("%s%v=", separator, cexpr.colName)
@@ -807,10 +813,7 @@ func (tpb *tablePlanBuilder) generateUpdateStatement() *sqlparser.ParsedQuery {
if cexpr.isPK {
tpb.pkIndices[i] = true
}
- if cexpr.isGrouped || cexpr.isPK {
- continue
- }
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGrouped || cexpr.isPK || cexpr.isGenerated {
continue
}
buf.Myprintf("%s%v=", separator, cexpr.colName)
@@ -880,7 +883,7 @@ func (tpb *tablePlanBuilder) generateDeleteStatement() *sqlparser.ParsedQuery {
}
func (tpb *tablePlanBuilder) generateMultiDeleteStatement() *sqlparser.ParsedQuery {
- if vttablet.VReplicationExperimentalFlags&vttablet.VReplicationExperimentalFlagVPlayerBatching == 0 ||
+ if tpb.workflowConfig.ExperimentalFlags&vttablet.VReplicationExperimentalFlagVPlayerBatching == 0 ||
(len(tpb.pkCols)+len(tpb.extraSourcePkCols)) != 1 {
return nil
}
@@ -956,15 +959,6 @@ func (tpb *tablePlanBuilder) generatePKConstraint(buf *sqlparser.TrackedBuffer,
buf.WriteString(")")
}
-func (tpb *tablePlanBuilder) isColumnGenerated(col sqlparser.IdentifierCI) bool {
- for _, colInfo := range tpb.colInfos {
- if col.EqualString(colInfo.Name) && colInfo.IsGenerated {
- return true
- }
- }
- return false
-}
-
// bindvarFormatter is a dual mode formatter. Its behavior
// can be changed dynamically changed to generate bind vars
// for the 'before' row or 'after' row by setting its mode
diff --git a/go/vt/vttablet/tabletmanager/vreplication/table_plan_partial.go b/go/vt/vttablet/tabletmanager/vreplication/table_plan_partial.go
index c6ccb898996..85e0fd8e50f 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/table_plan_partial.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/table_plan_partial.go
@@ -26,7 +26,7 @@ import (
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
)
// isBitSet returns true if the bit at index is set
@@ -37,7 +37,7 @@ func isBitSet(data []byte, index int) bool {
}
func (tp *TablePlan) isPartial(rowChange *binlogdatapb.RowChange) bool {
- if (vttablet.VReplicationExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage) == 0 ||
+ if (tp.WorkflowConfig.ExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage) == 0 ||
rowChange.DataColumns == nil ||
rowChange.DataColumns.Count == 0 {
@@ -50,10 +50,7 @@ func (tpb *tablePlanBuilder) generatePartialValuesPart(buf *sqlparser.TrackedBuf
bvf.mode = bvAfter
separator := "("
for ind, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
- continue
- }
- if !isBitSet(dataColumns.Cols, ind) {
+ if cexpr.isGenerated || !isBitSet(dataColumns.Cols, ind) {
continue
}
buf.Myprintf("%s", separator)
@@ -84,7 +81,7 @@ func (tpb *tablePlanBuilder) generatePartialInsertPart(buf *sqlparser.TrackedBuf
buf.Myprintf("insert into %v(", tpb.name)
separator := ""
for ind, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
if !isBitSet(dataColumns.Cols, ind) {
@@ -102,7 +99,7 @@ func (tpb *tablePlanBuilder) generatePartialSelectPart(buf *sqlparser.TrackedBuf
buf.WriteString(" select ")
separator := ""
for ind, cexpr := range tpb.colExprs {
- if tpb.isColumnGenerated(cexpr.colName) {
+ if cexpr.isGenerated {
continue
}
if !isBitSet(dataColumns.Cols, ind) {
@@ -141,17 +138,11 @@ func (tpb *tablePlanBuilder) createPartialUpdateQuery(dataColumns *binlogdatapb.
buf.Myprintf("update %v set ", tpb.name)
separator := ""
for i, cexpr := range tpb.colExprs {
- if cexpr.isPK {
- continue
- }
- if tpb.isColumnGenerated(cexpr.colName) {
- continue
- }
if int64(i) >= dataColumns.Count {
log.Errorf("Ran out of columns trying to generate query for %s", tpb.name.CompliantName())
return nil
}
- if !isBitSet(dataColumns.Cols, i) {
+ if cexpr.isPK || cexpr.isGenerated || !isBitSet(dataColumns.Cols, i) {
continue
}
buf.Myprintf("%s%v=", separator, cexpr.colName)
diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go
index 537041907a7..67b52c56261 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/utils.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go
@@ -134,7 +134,12 @@ func isUnrecoverableError(err error) bool {
if err == nil {
return false
}
- if vterrors.Code(err) == vtrpcpb.Code_FAILED_PRECONDITION {
+ switch vterrors.Code(err) {
+ case vtrpcpb.Code_FAILED_PRECONDITION:
+ if vterrors.RxWrongTablet.MatchString(err.Error()) {
+ // If the chosen tablet type picked changes, say due to PRS/ERS, we should retry.
+ return false
+ }
return true
}
sqlErr, isSQLErr := sqlerror.NewSQLErrorFromError(err).(*sqlerror.SQLError)
@@ -222,10 +227,29 @@ func isUnrecoverableError(err error) bool {
sqlerror.ERWrongParametersToProcedure,
sqlerror.ERWrongUsage,
sqlerror.ERWrongValue,
+ sqlerror.ERWrongParamcountToNativeFct,
sqlerror.ERVectorConversion,
sqlerror.ERWrongValueCountOnRow:
log.Errorf("Got unrecoverable error: %v", sqlErr)
return true
+ case sqlerror.ERErrorDuringCommit:
+ switch sqlErr.HaErrorCode() {
+ case
+ 0, // Not really a HA error.
+ sqlerror.HaErrLockDeadlock,
+ sqlerror.HaErrLockTableFull,
+ sqlerror.HaErrLockWaitTimeout,
+ sqlerror.HaErrNotInLockPartitions,
+ sqlerror.HaErrQueryInterrupted,
+ sqlerror.HaErrRolledBack,
+ sqlerror.HaErrTooManyConcurrentTrxs,
+ sqlerror.HaErrUndoRecTooBig:
+ // These are recoverable errors.
+ return false
+ default:
+ log.Errorf("Got unrecoverable error: %v", sqlErr)
+ return true
+ }
}
return false
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go
index bfe79036f3c..15093e299fc 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go
@@ -17,17 +17,23 @@ limitations under the License.
package vreplication
import (
+ "errors"
"fmt"
"strings"
"testing"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
+
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
+ "vitess.io/vitess/go/vt/vterrors"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
func TestInsertLogTruncation(t *testing.T) {
@@ -36,7 +42,7 @@ func TestInsertLogTruncation(t *testing.T) {
dbClient.RemoveInvariant("insert into _vt.vreplication_log") // Otherwise the insert will be ignored
stats := binlogplayer.NewStats()
defer stats.Stop()
- vdbClient := newVDBClient(dbClient, stats)
+ vdbClient := newVDBClient(dbClient, stats, vttablet.DefaultVReplicationConfig.RelayLogMaxItems)
defer vdbClient.Close()
vrID := int32(1)
typ := "Testing"
@@ -97,3 +103,70 @@ func TestInsertLogTruncation(t *testing.T) {
})
}
}
+
+// TestIsUnrecoverableError tests the different error cases for isUnrecoverableError().
+func TestIsUnrecoverableError(t *testing.T) {
+ if runNoBlobTest {
+ t.Skip()
+ }
+
+ type testCase struct {
+ name string
+ err error
+ expected bool
+ }
+
+ testCases := []testCase{
+ {
+ name: "Nil error",
+ err: nil,
+ expected: false,
+ },
+ {
+ name: "vterrors.Code_FAILED_PRECONDITION",
+ err: vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "test error"),
+ expected: true,
+ },
+ {
+ name: "vterrors.Code_FAILED_PRECONDITION, WrongTablet",
+ err: vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s: %v, want: %v or %v", vterrors.WrongTablet, "PRIMARY", "REPLICA", nil),
+ expected: false,
+ },
+ {
+ name: "Non-SQL error",
+ err: errors.New("non-SQL error"),
+ expected: false,
+ },
+ {
+ name: "SQL error with ERUnknownError",
+ err: sqlerror.NewSQLError(sqlerror.ERUnknownError, "test SQL error", "test"),
+ expected: false,
+ },
+ {
+ name: "SQL error with ERAccessDeniedError",
+ err: sqlerror.NewSQLError(sqlerror.ERAccessDeniedError, "access denied", "test"),
+ expected: true,
+ },
+ {
+ name: "SQL error with ERDataOutOfRange",
+ err: sqlerror.NewSQLError(sqlerror.ERDataOutOfRange, "data out of range", "test"),
+ expected: true,
+ },
+ {
+ name: "SQL error with HaErrDiskFullNowait error",
+ err: sqlerror.NewSQLError(sqlerror.ERErrorDuringCommit, "unknown", "ERROR HY000: Got error 204 - 'No more room in disk' during COMMIT"),
+ expected: true,
+ },
+ {
+ name: "SQL error with HaErrLockDeadlock error",
+ err: sqlerror.NewSQLError(sqlerror.ERErrorDuringCommit, "unknown", "ERROR HY000: Got error 149 - 'Lock deadlock; Retry transaction' during COMMIT"),
+ expected: false,
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ result := isUnrecoverableError(tc.err)
+ require.Equal(t, tc.expected, result)
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go
index 47e3798acd0..e3a12258691 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go
@@ -37,7 +37,6 @@ import (
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -221,7 +220,7 @@ func newVCopierCopyWorker(
func (vc *vcopier) initTablesForCopy(ctx context.Context) error {
defer vc.vr.dbClient.Rollback()
- plan, err := buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
+ plan, err := vc.vr.buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
if err != nil {
return err
}
@@ -348,7 +347,7 @@ func (vc *vcopier) catchup(ctx context.Context, copyState map[string]*sqltypes.R
// Wait for catchup.
tkr := time.NewTicker(waitRetryTime)
defer tkr.Stop()
- seconds := int64(replicaLagTolerance / time.Second)
+ seconds := int64(vc.vr.workflowConfig.ReplicaLagTolerance / time.Second)
for {
sbm := vc.vr.stats.ReplicationLagSeconds.Load()
if sbm < seconds {
@@ -382,7 +381,7 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma
log.Infof("Copying table %s, lastpk: %v", tableName, copyState[tableName])
- plan, err := buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
+ plan, err := vc.vr.buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
if err != nil {
return err
}
@@ -392,7 +391,7 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma
return fmt.Errorf("plan not found for table: %s, current plans are: %#v", tableName, plan.TargetTables)
}
- ctx, cancel := context.WithTimeout(ctx, vttablet.CopyPhaseDuration)
+ ctx, cancel := context.WithTimeout(ctx, vc.vr.workflowConfig.CopyPhaseDuration)
defer cancel()
var lastpkpb *querypb.QueryResult
@@ -405,7 +404,7 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma
copyStateGCTicker := time.NewTicker(copyStateGCInterval)
defer copyStateGCTicker.Stop()
- parallelism := getInsertParallelism()
+ parallelism := int(math.Max(1, float64(vc.vr.workflowConfig.ParallelInsertWorkers)))
copyWorkerFactory := vc.newCopyWorkerFactory(parallelism)
copyWorkQueue := vc.newCopyWorkQueue(parallelism, copyWorkerFactory)
defer copyWorkQueue.close()
@@ -420,6 +419,9 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma
// Use this for task sequencing.
var prevCh <-chan *vcopierCopyTaskResult
+ vstreamOptions := &binlogdatapb.VStreamOptions{
+ ConfigOverrides: vc.vr.workflowConfig.Overrides,
+ }
serr := vc.vr.sourceVStreamer.VStreamRows(ctx, initialPlan.SendRule.Filter, lastpkpb, func(rows *binlogdatapb.VStreamRowsResponse) error {
for {
select {
@@ -593,7 +595,7 @@ func (vc *vcopier) copyTable(ctx context.Context, tableName string, copyState ma
}
return nil
- })
+ }, vstreamOptions)
// Close the work queue. This will prevent new tasks from being enqueued,
// and will wait until all workers are returned to the worker pool.
@@ -683,7 +685,7 @@ func (vc *vcopier) updatePos(ctx context.Context, gtid string) error {
if err != nil {
return err
}
- update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0, vc.vr.stats.CopyRowCount.Get(), vreplicationStoreCompressedGTID)
+ update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0, vc.vr.stats.CopyRowCount.Get(), vc.vr.workflowConfig.StoreCompressedGTID)
_, err = vc.vr.dbClient.Execute(update)
return err
}
@@ -699,7 +701,7 @@ func (vc *vcopier) fastForward(ctx context.Context, copyState map[string]*sqltyp
return err
}
if settings.StartPos.IsZero() {
- update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0, vc.vr.stats.CopyRowCount.Get(), vreplicationStoreCompressedGTID)
+ update := binlogplayer.GenerateUpdatePos(vc.vr.id, pos, time.Now().Unix(), 0, vc.vr.stats.CopyRowCount.Get(), vc.vr.workflowConfig.StoreCompressedGTID)
_, err := vc.vr.dbClient.Execute(update)
return err
}
@@ -1201,9 +1203,3 @@ func vcopierCopyTaskGetNextState(vts vcopierCopyTaskState) vcopierCopyTaskState
}
return vts
}
-
-// getInsertParallelism returns the number of parallel workers to use for inserting batches during the copy phase.
-func getInsertParallelism() int {
- parallelism := int(math.Max(1, float64(vreplicationParallelInsertWorkers)))
- return parallelism
-}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go
index 02e1188cdb7..3a0996e0c39 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_atomic.go
@@ -20,21 +20,19 @@ import (
"context"
"fmt"
"io"
+ "math"
"strconv"
"time"
- "vitess.io/vitess/go/vt/vttablet"
-
"google.golang.org/protobuf/encoding/prototext"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/log"
- "vitess.io/vitess/go/vt/sqlparser"
- "vitess.io/vitess/go/vt/vterrors"
-
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
querypb "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
)
/*
@@ -54,7 +52,7 @@ func newCopyAllState(vc *vcopier) (*copyAllState, error) {
state := ©AllState{
vc: vc,
}
- plan, err := buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
+ plan, err := vc.vr.buildReplicatorPlan(vc.vr.source, vc.vr.colInfoMap, nil, vc.vr.stats, vc.vr.vre.env.CollationEnv(), vc.vr.vre.env.Parser())
if err != nil {
return nil, err
}
@@ -79,13 +77,13 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings
return err
}
- ctx, cancel := context.WithTimeout(ctx, vttablet.CopyPhaseDuration)
+ ctx, cancel := context.WithTimeout(ctx, vc.vr.workflowConfig.CopyPhaseDuration)
defer cancel()
rowsCopiedTicker := time.NewTicker(rowsCopiedUpdateInterval)
defer rowsCopiedTicker.Stop()
- parallelism := getInsertParallelism()
+ parallelism := int(math.Max(1, float64(vc.vr.workflowConfig.ParallelInsertWorkers)))
copyWorkerFactory := vc.newCopyWorkerFactory(parallelism)
var copyWorkQueue *vcopierCopyWorkQueue
@@ -102,6 +100,9 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings
var prevCh <-chan *vcopierCopyTaskResult
var gtid string
+ vstreamOptions := &binlogdatapb.VStreamOptions{
+ ConfigOverrides: vc.vr.workflowConfig.Overrides,
+ }
serr := vc.vr.sourceVStreamer.VStreamTables(ctx, func(resp *binlogdatapb.VStreamTablesResponse) error {
defer vc.vr.stats.PhaseTimings.Record("copy", time.Now())
defer vc.vr.stats.CopyLoopCount.Add(1)
@@ -274,7 +275,7 @@ func (vc *vcopier) copyAll(ctx context.Context, settings binlogplayer.VRSettings
default:
}
return nil
- })
+ }, vstreamOptions)
if serr != nil {
log.Infof("VStreamTables failed: %v", serr)
return serr
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go
index fda9012c1b5..a7e4794ba76 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier_test.go
@@ -25,10 +25,9 @@ import (
"testing"
"time"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
- "vitess.io/vitess/go/vt/vttablet"
-
"vitess.io/vitess/go/vt/log"
"github.com/stretchr/testify/require"
@@ -49,30 +48,30 @@ func commonVcopierTestCases() []vcopierTestCase {
return []vcopierTestCase{
// Default experimental flags.
{
- vreplicationExperimentalFlags: vttablet.VReplicationExperimentalFlags,
+ vreplicationExperimentalFlags: vttablet.DefaultVReplicationConfig.ExperimentalFlags,
},
// Parallel bulk inserts enabled with 4 workers.
{
- vreplicationExperimentalFlags: vttablet.VReplicationExperimentalFlags,
+ vreplicationExperimentalFlags: vttablet.DefaultVReplicationConfig.ExperimentalFlags,
vreplicationParallelInsertWorkers: 4,
},
}
}
func testVcopierTestCases(t *testing.T, test func(*testing.T), cases []vcopierTestCase) {
- oldVreplicationExperimentalFlags := vttablet.VReplicationExperimentalFlags
- oldVreplicationParallelInsertWorkers := vreplicationParallelInsertWorkers
+ oldVreplicationExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ oldVreplicationParallelInsertWorkers := vttablet.DefaultVReplicationConfig.ParallelInsertWorkers
// Extra reset at the end in case we return prematurely.
defer func() {
- vttablet.VReplicationExperimentalFlags = oldVreplicationExperimentalFlags
- vreplicationParallelInsertWorkers = oldVreplicationParallelInsertWorkers
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = oldVreplicationExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ParallelInsertWorkers = oldVreplicationParallelInsertWorkers
}()
for _, tc := range cases {
tc := tc // Avoid export loop bugs.
// Set test flags.
- vttablet.VReplicationExperimentalFlags = tc.vreplicationExperimentalFlags
- vreplicationParallelInsertWorkers = tc.vreplicationParallelInsertWorkers
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = tc.vreplicationExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ParallelInsertWorkers = tc.vreplicationParallelInsertWorkers
// Run test case.
t.Run(
fmt.Sprintf(
@@ -82,8 +81,8 @@ func testVcopierTestCases(t *testing.T, test func(*testing.T), cases []vcopierTe
test,
)
// Reset.
- vttablet.VReplicationExperimentalFlags = oldVreplicationExperimentalFlags
- vreplicationParallelInsertWorkers = oldVreplicationParallelInsertWorkers
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = oldVreplicationExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ParallelInsertWorkers = oldVreplicationParallelInsertWorkers
}
}
@@ -97,10 +96,10 @@ func testPlayerCopyCharPK(t *testing.T) {
reset := vstreamer.AdjustPacketSize(1)
defer reset()
- savedCopyPhaseDuration := vttablet.CopyPhaseDuration
+ savedCopyPhaseDuration := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
// copyPhaseDuration should be low enough to have time to send one row.
- vttablet.CopyPhaseDuration = 500 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = savedCopyPhaseDuration }()
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 500 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = savedCopyPhaseDuration }()
savedWaitRetryTime := waitRetryTime
// waitRetry time should be very low to cause the wait loop to execute multiple times.
@@ -175,11 +174,11 @@ func testPlayerCopyCharPK(t *testing.T) {
"/update _vt.vreplication set message='Picked source tablet.*",
"/insert into _vt.copy_state",
"/update _vt.vreplication set state='Copying'",
- "insert into dst(idc,val) values ('a\\0',1)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"idc" type:BINARY charset:63 flags:20611} rows:{lengths:2 values:"a\\\\x00"}'.*`,
- `update dst set val=3 where idc='a\0' and ('a\0') <= ('a\0')`,
- "insert into dst(idc,val) values ('c\\0',2)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"idc" type:BINARY charset:63 flags:20611} rows:{lengths:2 values:"c\\\\x00"}'.*`,
+ "insert into dst(idc,val) values (_binary'a\\0',1)",
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"idc" type:BINARY charset:63 flags:20611} rows:{lengths:2 values:"a\\\\x00"}'.*`,
+ `update dst set val=3 where idc=_binary'a\0' and (_binary'a\0') <= (_binary'a\0')`,
+ "insert into dst(idc,val) values (_binary'c\\0',2)",
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"idc" type:BINARY charset:63 flags:20611} rows:{lengths:2 values:"c\\\\x00"}'.*`,
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst",
"/update _vt.vreplication set state='Running",
), recvTimeout)
@@ -203,10 +202,10 @@ func testPlayerCopyVarcharPKCaseInsensitive(t *testing.T) {
reset := vstreamer.AdjustPacketSize(1)
defer reset()
- savedCopyPhaseDuration := vttablet.CopyPhaseDuration
+ savedCopyPhaseDuration := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
// copyPhaseDuration should be low enough to have time to send one row.
- vttablet.CopyPhaseDuration = 500 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = savedCopyPhaseDuration }()
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 500 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = savedCopyPhaseDuration }()
savedWaitRetryTime := waitRetryTime
// waitRetry time should be very low to cause the wait loop to execute multiple times.
@@ -283,21 +282,21 @@ func testPlayerCopyVarcharPKCaseInsensitive(t *testing.T) {
"/update _vt.vreplication set state='Copying'",
// Copy mode.
"insert into dst(idc,val) values ('a',1)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"a"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"a"}'.*`,
// Copy-catchup mode.
`/insert into dst\(idc,val\) select 'B', 3 from dual where \( .* 'B' COLLATE .* \) <= \( .* 'a' COLLATE .* \)`,
).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer {
// Back to copy mode.
// Inserts can happen out of order.
// Updates must happen in order.
- //upd1 := expect.
+ // upd1 := expect.
upd1 := expect.Then(qh.Eventually(
"insert into dst(idc,val) values ('B',3)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"B"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"B"}'.*`,
))
upd2 := expect.Then(qh.Eventually(
"insert into dst(idc,val) values ('c',2)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"c"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"idc" type:VARCHAR charset:33 flags:20483} rows:{lengths:1 values:"c"}'.*`,
))
upd1.Then(upd2.Eventually())
return upd2
@@ -325,10 +324,10 @@ func testPlayerCopyVarcharCompositePKCaseSensitiveCollation(t *testing.T) {
reset := vstreamer.AdjustPacketSize(1)
defer reset()
- savedCopyPhaseDuration := vttablet.CopyPhaseDuration
+ savedCopyPhaseDuration := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
// copyPhaseDuration should be low enough to have time to send one row.
- vttablet.CopyPhaseDuration = 500 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = savedCopyPhaseDuration }()
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 500 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = savedCopyPhaseDuration }()
savedWaitRetryTime := waitRetryTime
// waitRetry time should be very low to cause the wait loop to execute multiple times.
@@ -337,7 +336,7 @@ func testPlayerCopyVarcharCompositePKCaseSensitiveCollation(t *testing.T) {
execStatements(t, []string{
"create table src(id int, idc varbinary(20), idc2 varbinary(20), val int, primary key(id,idc,idc2))",
- "insert into src values(1, 'a', 'a', 1), (1, 'c', 'c', 2)",
+ "insert into src values(1, _binary'a', _binary'a', 1), (1, _binary'c', _binary'c', 2)",
fmt.Sprintf("create table %s.dst(id int, idc varbinary(20), idc2 varbinary(20), val int, primary key(id,idc,idc2))", vrepldb),
})
defer execStatements(t, []string{
@@ -355,7 +354,7 @@ func testPlayerCopyVarcharCompositePKCaseSensitiveCollation(t *testing.T) {
}
// Insert a row with PK which is < the lastPK till now because of the utf8mb4 collation
execStatements(t, []string{
- "insert into src values(1, 'B', 'B', 3)",
+ "insert into src values(1, _binary'B', _binary'B', 3)",
})
// Wait for context to expire and then send the row.
// This will cause the copier to abort and go back to catchup mode.
@@ -405,13 +404,13 @@ func testPlayerCopyVarcharCompositePKCaseSensitiveCollation(t *testing.T) {
"/insert into _vt.copy_state",
"/update _vt.vreplication set state='Copying'",
// Copy mode.
- "insert into dst(id,idc,idc2,val) values (1,'a','a',1)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"idc" type:VARBINARY charset:63 flags:20611} fields:{name:"idc2" type:VARBINARY charset:63 flags:20611} rows:{lengths:1 lengths:1 lengths:1 values:"1aa"}'.*`,
+ "insert into dst(id,idc,idc2,val) values (1,_binary'a',_binary'a',1)",
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"idc" type:VARBINARY charset:63 flags:20611} fields:{name:"idc2" type:VARBINARY charset:63 flags:20611} rows:{lengths:1 lengths:1 lengths:1 values:"1aa"}'.*`,
// Copy-catchup mode.
- `insert into dst(id,idc,idc2,val) select 1, 'B', 'B', 3 from dual where (1,'B','B') <= (1,'a','a')`,
+ `insert into dst(id,idc,idc2,val) select 1, _binary'B', _binary'B', 3 from dual where (1,_binary'B',_binary'B') <= (1,_binary'a',_binary'a')`,
// Copy mode.
- "insert into dst(id,idc,idc2,val) values (1,'c','c',2)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"idc" type:VARBINARY charset:63 flags:20611} fields:{name:"idc2" type:VARBINARY charset:63 flags:20611} rows:{lengths:1 lengths:1 lengths:1 values:"1cc"}'.*`,
+ "insert into dst(id,idc,idc2,val) values (1,_binary'c',_binary'c',2)",
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"idc" type:VARBINARY charset:63 flags:20611} fields:{name:"idc2" type:VARBINARY charset:63 flags:20611} rows:{lengths:1 lengths:1 lengths:1 values:"1cc"}'.*`,
// Wrap-up.
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst",
"/update _vt.vreplication set state='Running'",
@@ -486,7 +485,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) {
"/update _vt.vreplication set pos=",
).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer {
// With parallel inserts, new db client connects are created on-the-fly.
- if vreplicationParallelInsertWorkers > 1 {
+ if vttablet.DefaultVReplicationConfig.ParallelInsertWorkers > 1 {
return expect.Then(qh.Eventually("set @@session.foreign_key_checks=0"))
}
return expect
@@ -495,7 +494,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) {
// Inserts may happen out-of-order. Update happen in-order.
"begin",
"insert into dst1(id,id2) values (1,1), (2,2)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
"set @@session.foreign_key_checks=0",
@@ -508,7 +507,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) {
"commit",
)).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer {
// With parallel inserts, new db client connects are created on-the-fly.
- if vreplicationParallelInsertWorkers > 1 {
+ if vttablet.DefaultVReplicationConfig.ParallelInsertWorkers > 1 {
return expect.Then(qh.Eventually("set @@session.foreign_key_checks=0"))
}
return expect
@@ -516,7 +515,7 @@ func testPlayerCopyTablesWithFK(t *testing.T) {
// copy dst2
"begin",
"insert into dst2(id,id2) values (1,21), (2,22)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
"set @@session.foreign_key_checks=0",
@@ -561,13 +560,13 @@ func testPlayerCopyTables(t *testing.T) {
execStatements(t, []string{
"create table ast1(id int, primary key(id))",
- "create table src1(id int, val varbinary(128), d decimal(8,0), j json, primary key(id))",
+ "create table src1(id int, val varchar(128), d decimal(8,0), j json, primary key(id))",
"insert into src1 values(2, 'bbb', 1, '{\"foo\": \"bar\"}'), (1, 'aaa', 0, JSON_ARRAY(123456789012345678901234567890, \"abcd\")), (3, 'ccc', 2, 'null'), (4, 'ddd', 3, '{\"name\": \"matt\", \"size\": null}'), (5, 'eee', 4, null)",
fmt.Sprintf("create table %s.ast1(id int, primary key(id))", vrepldb),
- fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), val2 varbinary(128), d decimal(8,0), j json, primary key(id))", vrepldb),
- "create table yes(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.yes(id int, val varbinary(128), primary key(id))", vrepldb),
- "create table no(id int, val varbinary(128), primary key(id))",
+ fmt.Sprintf("create table %s.dst1(id int, val varchar(128), val2 varchar(128), d decimal(8,0), j json, primary key(id))", vrepldb),
+ "create table yes(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.yes(id int, val varchar(128), primary key(id))", vrepldb),
+ "create table no(id int, val varchar(128), primary key(id))",
})
defer execStatements(t, []string{
"drop table ast1",
@@ -629,7 +628,7 @@ func testPlayerCopyTables(t *testing.T) {
"commit",
"begin",
"insert into dst1(id,val,val2,d,j) values (1,'aaa','aaa',0,JSON_ARRAY(123456789012345678901234567890, _utf8mb4'abcd')), (2,'bbb','bbb',1,JSON_OBJECT(_utf8mb4'foo', _utf8mb4'bar')), (3,'ccc','ccc',2,CAST(_utf8mb4'null' as JSON)), (4,'ddd','ddd',3,JSON_OBJECT(_utf8mb4'name', _utf8mb4'matt', _utf8mb4'size', null)), (5,'eee','eee',4,null)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"5"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"5"}'.*`,
"commit",
// copy of dst1 is done: delete from copy_state.
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst1",
@@ -685,10 +684,18 @@ func testPlayerCopyBigTable(t *testing.T) {
reset := vstreamer.AdjustPacketSize(1)
defer reset()
- savedCopyPhaseDuration := vttablet.CopyPhaseDuration
+ // The test is written to match the behavior w/o
+ // VReplicationExperimentalFlagOptimizeInserts enabled.
+ origExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = 0
+ defer func() {
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = origExperimentalFlags
+ }()
+
+ savedCopyPhaseDuration := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
// copyPhaseDuration should be low enough to have time to send one row.
- vttablet.CopyPhaseDuration = 500 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = savedCopyPhaseDuration }()
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 500 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = savedCopyPhaseDuration }()
savedWaitRetryTime := waitRetryTime
// waitRetry time should be very low to cause the wait loop to execute multiple times.
@@ -696,9 +703,9 @@ func testPlayerCopyBigTable(t *testing.T) {
defer func() { waitRetryTime = savedWaitRetryTime }()
execStatements(t, []string{
- "create table src(id int, val varbinary(128), primary key(id))",
+ "create table src(id int, val varchar(128), primary key(id))",
"insert into src values(1, 'aaa'), (2, 'bbb')",
- fmt.Sprintf("create table %s.dst(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src",
@@ -765,7 +772,7 @@ func testPlayerCopyBigTable(t *testing.T) {
// The first fast-forward has no starting point. So, it just saves the current position.
"/update _vt.vreplication set state='Copying'",
"insert into dst(id,val) values (1,'aaa')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
// The next catchup executes the new row insert, but will be a no-op.
"insert into dst(id,val) select 3, 'ccc' from dual where (3) <= (1)",
// fastForward has nothing to add. Just saves position.
@@ -775,12 +782,12 @@ func testPlayerCopyBigTable(t *testing.T) {
).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer {
ins1 := expect.Then(qh.Eventually("insert into dst(id,val) values (2,'bbb')"))
upd1 := ins1.Then(qh.Eventually(
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
))
// Third row copied without going back to catchup state.
ins3 := expect.Then(qh.Eventually("insert into dst(id,val) values (3,'ccc')"))
upd3 := ins3.Then(qh.Eventually(
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"3"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"3"}'.*`,
))
upd1.Then(upd3.Eventually())
return upd3
@@ -815,10 +822,18 @@ func testPlayerCopyWildcardRule(t *testing.T) {
reset := vstreamer.AdjustPacketSize(1)
defer reset()
- savedCopyPhaseDuration := vttablet.CopyPhaseDuration
+ // The test is written to match the behavior w/o
+ // VReplicationExperimentalFlagOptimizeInserts enabled.
+ origExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = 0
+ defer func() {
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = origExperimentalFlags
+ }()
+
+ savedCopyPhaseDuration := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
// copyPhaseDuration should be low enough to have time to send one row.
- vttablet.CopyPhaseDuration = 500 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = savedCopyPhaseDuration }()
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 500 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = savedCopyPhaseDuration }()
savedWaitRetryTime := waitRetryTime
// waitRetry time should be very low to cause the wait loop to execute multiple times.
@@ -826,9 +841,9 @@ func testPlayerCopyWildcardRule(t *testing.T) {
defer func() { waitRetryTime = savedWaitRetryTime }()
execStatements(t, []string{
- "create table src(id int, val varbinary(128), primary key(id))",
+ "create table src(id int, val varchar(128), primary key(id))",
"insert into src values(1, 'aaa'), (2, 'bbb')",
- fmt.Sprintf("create table %s.src(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.src(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src",
@@ -894,7 +909,7 @@ func testPlayerCopyWildcardRule(t *testing.T) {
"/update _vt.vreplication set state='Copying'",
// The first fast-forward has no starting point. So, it just saves the current position.
"insert into src(id,val) values (1,'aaa')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
// The next catchup executes the new row insert, but will be a no-op.
"insert into src(id,val) select 3, 'ccc' from dual where (3) <= (1)",
// fastForward has nothing to add. Just saves position.
@@ -904,12 +919,12 @@ func testPlayerCopyWildcardRule(t *testing.T) {
).Then(func(expect qh.ExpectationSequencer) qh.ExpectationSequencer {
ins1 := expect.Then(qh.Eventually("insert into src(id,val) values (2,'bbb')"))
upd1 := ins1.Then(qh.Eventually(
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
))
// Third row copied without going back to catchup state.
ins3 := expect.Then(qh.Eventually("insert into src(id,val) values (3,'ccc')"))
upd3 := ins3.Then(qh.Eventually(
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"3"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"3"}'.*`,
))
upd1.Then(upd3.Eventually())
return upd3
@@ -944,19 +959,19 @@ func testPlayerCopyTableContinuation(t *testing.T) {
// src1 is initialized as partially copied.
// lastpk will be initialized at (6,6) later below.
// dst1 only copies id1 and val. This will allow us to test for correctness if id2 changes in the source.
- "create table src1(id1 int, id2 int, val varbinary(128), primary key(id1, id2))",
+ "create table src1(id1 int, id2 int, val varchar(128), primary key(id1, id2))",
"insert into src1 values(2,2,'no change'), (3,3,'update'), (4,4,'delete'), (5,5,'move within'), (6,6,'move out'), (8,8,'no change'), (9,9,'delete'), (10,10,'update'), (11,11,'move in')",
- fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst1(id int, val varchar(128), primary key(id))", vrepldb),
fmt.Sprintf("insert into %s.dst1 values(2,'no change'), (3,'update'), (4,'delete'), (5,'move within'), (6,'move out')", vrepldb),
// copied is initialized as fully copied
- "create table copied(id int, val varbinary(128), primary key(id))",
+ "create table copied(id int, val varchar(128), primary key(id))",
"insert into copied values(1,'aaa')",
- fmt.Sprintf("create table %s.copied(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.copied(id int, val varchar(128), primary key(id))", vrepldb),
fmt.Sprintf("insert into %s.copied values(1,'aaa')", vrepldb),
// not_copied yet to be copied.
- "create table not_copied(id int, val varbinary(128), primary key(id))",
+ "create table not_copied(id int, val varchar(128), primary key(id))",
"insert into not_copied values(1,'aaa')",
- fmt.Sprintf("create table %s.not_copied(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.not_copied(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src1",
@@ -1068,13 +1083,13 @@ func testPlayerCopyTableContinuation(t *testing.T) {
).Then(qh.Immediately(
"insert into dst1(id,val) values (7,'insert out'), (8,'no change'), (10,'updated'), (12,'move out')",
)).Then(qh.Eventually(
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id1" type:INT32 charset:63 flags:53251} fields:{name:"id2" type:INT32 charset:63 flags:53251} rows:{lengths:2 lengths:1 values:"126"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id1" type:INT32 charset:63 flags:53251} fields:{name:"id2" type:INT32 charset:63 flags:53251} rows:{lengths:2 lengths:1 values:"126"}'.*`,
)).Then(qh.Immediately(
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst1",
"insert into not_copied(id,val) values (1,'bbb')",
)).Then(qh.Eventually(
// Copy again. There should be no events for catchup.
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"1"}'.*`,
)).Then(qh.Immediately(
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*not_copied",
"/update _vt.vreplication set state='Running'",
@@ -1119,9 +1134,9 @@ func testPlayerCopyWildcardTableContinuation(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table src(id int, val varbinary(128), primary key(id))",
+ "create table src(id int, val varchar(128), primary key(id))",
"insert into src values(2,'copied'), (3,'uncopied')",
- fmt.Sprintf("create table %s.dst(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst(id int, val varchar(128), primary key(id))", vrepldb),
fmt.Sprintf("insert into %s.dst values(2,'copied')", vrepldb),
})
defer execStatements(t, []string{
@@ -1173,7 +1188,7 @@ func testPlayerCopyWildcardTableContinuation(t *testing.T) {
expectDeleteQueries(t)
}()
- optimizeInsertsEnabled := vttablet.VReplicationExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagOptimizeInserts != 0
+ optimizeInsertsEnabled := vttablet.DefaultVReplicationConfig.ExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagOptimizeInserts != 0
expectNontxQueries(t, qh.Expect(
"/insert into _vt.vreplication",
@@ -1203,21 +1218,25 @@ func testPlayerCopyWildcardTableContinuation(t *testing.T) {
}
}
+func setExperimentalFlags(flags int64) func() {
+ oldVreplicationExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = flags
+ return func() {
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = oldVreplicationExperimentalFlags
+ }
+}
+
// TestPlayerCopyWildcardTableContinuationWithOptimizeInserts tests the copy workflow where tables have been partially copied
// enabling the optimize inserts functionality
func TestPlayerCopyWildcardTableContinuationWithOptimizeInserts(t *testing.T) {
- oldVreplicationExperimentalFlags := vttablet.VReplicationExperimentalFlags
- vttablet.VReplicationExperimentalFlags = vttablet.VReplicationExperimentalFlagOptimizeInserts
- defer func() {
- vttablet.VReplicationExperimentalFlags = oldVreplicationExperimentalFlags
- }()
+ defer setExperimentalFlags(vttablet.VReplicationExperimentalFlagOptimizeInserts)()
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table src(id int, val varbinary(128), primary key(id))",
+ "create table src(id int, val varchar(128), primary key(id))",
"insert into src values(2,'copied'), (3,'uncopied')",
- fmt.Sprintf("create table %s.dst(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst(id int, val varchar(128), primary key(id))", vrepldb),
fmt.Sprintf("insert into %s.dst values(2,'copied')", vrepldb),
})
defer execStatements(t, []string{
@@ -1341,9 +1360,9 @@ func testPlayerCopyTablesStopAfterCopy(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table src1(id int, val varbinary(128), primary key(id))",
+ "create table src1(id int, val varchar(128), primary key(id))",
"insert into src1 values(2, 'bbb'), (1, 'aaa')",
- fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src1",
@@ -1390,7 +1409,7 @@ func testPlayerCopyTablesStopAfterCopy(t *testing.T) {
).Then(qh.Eventually(
"begin",
"insert into dst1(id,val) values (1,'aaa'), (2,'bbb')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
// copy of dst1 is done: delete from copy_state.
@@ -1420,13 +1439,13 @@ func testPlayerCopyTablesGIPK(t *testing.T) {
execStatements(t, []string{
"SET @@session.sql_generate_invisible_primary_key=ON;",
- "create table src1(val varbinary(128))",
+ "create table src1(val varchar(128))",
"insert into src1 values('aaa'), ('bbb')",
- "create table src2(val varbinary(128))",
+ "create table src2(val varchar(128))",
"insert into src2 values('aaa'), ('bbb')",
- fmt.Sprintf("create table %s.dst1(val varbinary(128))", vrepldb),
+ fmt.Sprintf("create table %s.dst1(val varchar(128))", vrepldb),
"SET @@session.sql_generate_invisible_primary_key=OFF;",
- fmt.Sprintf("create table %s.dst2(my_row_id int, val varbinary(128), primary key(my_row_id))", vrepldb),
+ fmt.Sprintf("create table %s.dst2(my_row_id int, val varchar(128), primary key(my_row_id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src1",
@@ -1478,7 +1497,7 @@ func testPlayerCopyTablesGIPK(t *testing.T) {
).Then(qh.Eventually(
"begin",
"insert into dst1(my_row_id,val) values (1,'aaa'), (2,'bbb')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"my_row_id" type:UINT64 charset:63 flags:49699} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"my_row_id" type:UINT64 charset:63 flags:49699} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
// copy of dst1 is done: delete from copy_state.
@@ -1489,7 +1508,7 @@ func testPlayerCopyTablesGIPK(t *testing.T) {
"commit",
"begin",
"insert into dst2(my_row_id,val) values (1,'aaa'), (2,'bbb')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"my_row_id" type:UINT64 charset:63 flags:49699} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"my_row_id" type:UINT64 charset:63 flags:49699} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
// copy of dst2 is done: delete from copy_state.
@@ -1516,23 +1535,25 @@ func testPlayerCopyTableCancel(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table src1(id int, val varbinary(128), primary key(id))",
+ "create table src1(id int, val varchar(128), primary key(id))",
"insert into src1 values(2, 'bbb'), (1, 'aaa')",
- fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb),
+ fmt.Sprintf("create table %s.dst1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table src1",
fmt.Sprintf("drop table %s.dst1", vrepldb),
})
- saveTimeout := vttablet.CopyPhaseDuration
- vttablet.CopyPhaseDuration = 1 * time.Millisecond
- defer func() { vttablet.CopyPhaseDuration = saveTimeout }()
+ saveTimeout := vttablet.DefaultVReplicationConfig.CopyPhaseDuration
+ vttablet.DefaultVReplicationConfig.CopyPhaseDuration = 1 * time.Millisecond
+ defer func() { vttablet.DefaultVReplicationConfig.CopyPhaseDuration = saveTimeout }()
// Set a hook to reset the copy timeout after first call.
vstreamRowsHook = func(ctx context.Context) {
<-ctx.Done()
- vttablet.CopyPhaseDuration = saveTimeout
+ for _, ct := range playerEngine.controllers {
+ ct.WorkflowConfig.CopyPhaseDuration = saveTimeout
+ }
vstreamRowsHook = nil
}
@@ -1578,7 +1599,7 @@ func testPlayerCopyTableCancel(t *testing.T) {
).Then(qh.Eventually(
"begin",
"insert into dst1(id,val) values (1,'aaa'), (2,'bbb')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
// copy of dst1 is done: delete from copy_state.
@@ -1601,12 +1622,12 @@ func testPlayerCopyTablesWithGeneratedColumn(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table src1(id int, val varbinary(128), val2 varbinary(128) as (concat(id, val)), val3 varbinary(128) as (concat(val, id)), id2 int, primary key(id))",
+ "create table src1(id int, val varchar(128), val2 varchar(128) as (concat(id, val)), val3 varchar(128) as (concat(val, id)), id2 int, primary key(id))",
"insert into src1(id, val, id2) values(2, 'bbb', 20), (1, 'aaa', 10)",
- fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), val2 varbinary(128) as (concat(id, val)), val3 varbinary(128), id2 int, primary key(id))", vrepldb),
- "create table src2(id int, val varbinary(128), val2 varbinary(128) as (concat(id, val)), val3 varbinary(128) as (concat(val, id)), id2 int, primary key(id))",
+ fmt.Sprintf("create table %s.dst1(id int, val varchar(128), val2 varchar(128) as (concat(id, val)), val3 varchar(128), id2 int, primary key(id))", vrepldb),
+ "create table src2(id int, val varchar(128), val2 varchar(128) as (concat(id, val)), val3 varchar(128) as (concat(val, id)), id2 int, primary key(id))",
"insert into src2(id, val, id2) values(2, 'bbb', 20), (1, 'aaa', 10)",
- fmt.Sprintf("create table %s.dst2(val3 varbinary(128), val varbinary(128), id2 int)", vrepldb),
+ fmt.Sprintf("create table %s.dst2(val3 varchar(128), val varchar(128), id2 int)", vrepldb),
})
defer execStatements(t, []string{
"drop table src1",
@@ -1651,11 +1672,11 @@ func testPlayerCopyTablesWithGeneratedColumn(t *testing.T) {
"/update _vt.vreplication set state",
// The first fast-forward has no starting point. So, it just saves the current position.
"insert into dst1(id,val,val3,id2) values (1,'aaa','aaa1',10), (2,'bbb','bbb2',20)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
// copy of dst1 is done: delete from copy_state.
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst1",
"insert into dst2(val3,val,id2) values ('aaa1','aaa',10), ('bbb2','bbb',20)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
// copy of dst2 is done: delete from copy_state.
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst2",
"/update _vt.vreplication set state",
@@ -1732,7 +1753,7 @@ func testCopyTablesWithInvalidDates(t *testing.T) {
).Then(qh.Eventually(
"begin",
"insert into dst1(id,dt) values (1,'2020-01-12'), (2,'0000-00-00')",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} rows:{lengths:1 values:"2"}'.*`,
"commit",
)).Then(qh.Immediately(
// copy of dst1 is done: delete from copy_state.
@@ -1822,7 +1843,7 @@ func testCopyInvisibleColumns(t *testing.T) {
"/update _vt.vreplication set state='Copying'",
// The first fast-forward has no starting point. So, it just saves the current position.
"insert into dst1(id,id2,inv1,inv2) values (1,10,100,1000), (2,20,200,2000)",
- `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \('fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"inv1" type:INT32 charset:63 flags:53251} rows:{lengths:1 lengths:3 values:"2200"}'.*`,
+ `/insert into _vt.copy_state \(lastpk, vrepl_id, table_name\) values \(_binary'fields:{name:"id" type:INT32 charset:63 flags:53251} fields:{name:"inv1" type:INT32 charset:63 flags:53251} rows:{lengths:1 lengths:3 values:"2200"}'.*`,
// copy of dst1 is done: delete from copy_state.
"/delete cs, pca from _vt.copy_state as cs left join _vt.post_copy_action as pca on cs.vrepl_id=pca.vrepl_id and cs.table_name=pca.table_name.*dst1",
"/update _vt.vreplication set state='Running'",
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vdbclient.go b/go/vt/vttablet/tabletmanager/vreplication/vdbclient.go
index 39a8229efc6..63538be881d 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vdbclient.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vdbclient.go
@@ -30,23 +30,27 @@ import (
"vitess.io/vitess/go/vt/vterrors"
)
+const beginStmtLen = int64(len("begin;"))
+
// vdbClient is a wrapper on binlogplayer.DBClient.
// It allows us to retry a failed transactions on lock errors.
type vdbClient struct {
binlogplayer.DBClient
- stats *binlogplayer.Stats
- InTransaction bool
- startTime time.Time
- queries []string
- queriesPos int64
- batchSize int64
- maxBatchSize int64
+ stats *binlogplayer.Stats
+ InTransaction bool
+ startTime time.Time
+ queries []string
+ queriesPos int64
+ batchSize int64
+ maxBatchSize int64
+ relayLogMaxItems int
}
-func newVDBClient(dbclient binlogplayer.DBClient, stats *binlogplayer.Stats) *vdbClient {
+func newVDBClient(dbclient binlogplayer.DBClient, stats *binlogplayer.Stats, relayLogMaxItems int) *vdbClient {
return &vdbClient{
- DBClient: dbclient,
- stats: stats,
+ DBClient: dbclient,
+ stats: stats,
+ relayLogMaxItems: relayLogMaxItems,
}
}
@@ -54,16 +58,19 @@ func (vc *vdbClient) Begin() error {
if vc.InTransaction {
return nil
}
- if err := vc.DBClient.Begin(); err != nil {
- return err
+ if vc.maxBatchSize > 0 {
+ // We are batching the contents of the transaction, which
+ // starts with the BEGIN and ends with the COMMIT, so we
+ // do not send a BEGIN down the wire ahead of time.
+ vc.queriesPos = int64(len(vc.queries))
+ vc.batchSize = beginStmtLen
+ } else {
+ // We're not batching so we start the transaction here
+ // by sending the BEGIN down the wire.
+ if err := vc.DBClient.Begin(); err != nil {
+ return err
+ }
}
-
- // If we're batching, we only batch the contents of the
- // transaction, which starts with the begin and ends with
- // the commit.
- vc.queriesPos = int64(len(vc.queries))
- vc.batchSize = 6 // begin and semicolon
-
vc.queries = append(vc.queries, "begin")
vc.InTransaction = true
vc.startTime = time.Now()
@@ -163,13 +170,13 @@ func (vc *vdbClient) ExecuteTrxQueryBatch() ([]*sqltypes.Result, error) {
// Execute is ExecuteFetch without the maxrows.
func (vc *vdbClient) Execute(query string) (*sqltypes.Result, error) {
// Number of rows should never exceed relayLogMaxItems.
- return vc.ExecuteFetch(query, relayLogMaxItems)
+ return vc.ExecuteFetch(query, vc.relayLogMaxItems)
}
func (vc *vdbClient) ExecuteWithRetry(ctx context.Context, query string) (*sqltypes.Result, error) {
qr, err := vc.Execute(query)
for err != nil {
- if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERLockDeadlock || sqlErr.Number() == sqlerror.ERLockWaitTimeout {
+ if sqlErr, ok := err.(*sqlerror.SQLError); ok && (sqlErr.Number() == sqlerror.ERLockDeadlock || sqlErr.Number() == sqlerror.ERLockWaitTimeout) {
log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dbLockRetryDelay)
if err := vc.Rollback(); err != nil {
return nil, err
@@ -199,7 +206,7 @@ func (vc *vdbClient) Retry() (*sqltypes.Result, error) {
continue
}
// Number of rows should never exceed relayLogMaxItems.
- result, err := vc.DBClient.ExecuteFetch(q, relayLogMaxItems)
+ result, err := vc.DBClient.ExecuteFetch(q, vc.relayLogMaxItems)
if err != nil {
return nil, err
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go
index 2b8b8130f89..98e36119622 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go
@@ -31,7 +31,7 @@ import (
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -133,20 +133,21 @@ func newVPlayer(vr *vreplicator, settings binlogplayer.VRSettings, copyState map
return vr.dbClient.Commit()
}
batchMode := false
- if vttablet.VReplicationExperimentalFlags&vttablet.VReplicationExperimentalFlagVPlayerBatching != 0 {
+ // We only do batching in the running/replicating phase.
+ if len(copyState) == 0 && vr.workflowConfig.ExperimentalFlags&vttablet.VReplicationExperimentalFlagVPlayerBatching != 0 {
batchMode = true
}
if batchMode {
// relayLogMaxSize is effectively the limit used when not batching.
- maxAllowedPacket := int64(relayLogMaxSize)
+ maxAllowedPacket := int64(vr.workflowConfig.RelayLogMaxSize)
// We explicitly do NOT want to batch this, we want to send it down the wire
// immediately so we use ExecuteFetch directly.
res, err := vr.dbClient.ExecuteFetch("select @@session.max_allowed_packet as max_allowed_packet", 1)
if err != nil {
- log.Errorf("Error getting max_allowed_packet, will use the relay_log_max_size value of %d bytes: %v", relayLogMaxSize, err)
+ log.Errorf("Error getting max_allowed_packet, will use the relay_log_max_size value of %d bytes: %v", vr.workflowConfig.RelayLogMaxSize, err)
} else {
if maxAllowedPacket, err = res.Rows[0][0].ToInt64(); err != nil {
- log.Errorf("Error getting max_allowed_packet, will use the relay_log_max_size value of %d bytes: %v", relayLogMaxSize, err)
+ log.Errorf("Error getting max_allowed_packet, will use the relay_log_max_size value of %d bytes: %v", vr.workflowConfig.RelayLogMaxSize, err)
}
}
// Leave 64 bytes of room for the commit to be sure that we have a more than
@@ -193,7 +194,7 @@ func (vp *vplayer) play(ctx context.Context) error {
return nil
}
- plan, err := buildReplicatorPlan(vp.vr.source, vp.vr.colInfoMap, vp.copyState, vp.vr.stats, vp.vr.vre.env.CollationEnv(), vp.vr.vre.env.Parser())
+ plan, err := vp.vr.buildReplicatorPlan(vp.vr.source, vp.vr.colInfoMap, vp.copyState, vp.vr.stats, vp.vr.vre.env.CollationEnv(), vp.vr.vre.env.Parser())
if err != nil {
vp.vr.stats.ErrorCounts.Add([]string{"Plan"}, 1)
return err
@@ -269,13 +270,17 @@ func (vp *vplayer) fetchAndApply(ctx context.Context) (err error) {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
- relay := newRelayLog(ctx, relayLogMaxItems, relayLogMaxSize)
+ relay := newRelayLog(ctx, vp.vr.workflowConfig.RelayLogMaxItems, vp.vr.workflowConfig.RelayLogMaxSize)
streamErr := make(chan error, 1)
go func() {
- streamErr <- vp.vr.sourceVStreamer.VStream(ctx, replication.EncodePosition(vp.startPos), nil, vp.replicatorPlan.VStreamFilter, func(events []*binlogdatapb.VEvent) error {
- return relay.Send(events)
- })
+ vstreamOptions := &binlogdatapb.VStreamOptions{
+ ConfigOverrides: vp.vr.workflowConfig.Overrides,
+ }
+ streamErr <- vp.vr.sourceVStreamer.VStream(ctx, replication.EncodePosition(vp.startPos), nil,
+ vp.replicatorPlan.VStreamFilter, func(events []*binlogdatapb.VEvent) error {
+ return relay.Send(events)
+ }, vstreamOptions)
}()
applyErr := make(chan error, 1)
@@ -382,7 +387,7 @@ func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.Row
// updatePos should get called at a minimum of vreplicationMinimumHeartbeatUpdateInterval.
func (vp *vplayer) updatePos(ctx context.Context, ts int64) (posReached bool, err error) {
- update := binlogplayer.GenerateUpdatePos(vp.vr.id, vp.pos, time.Now().Unix(), ts, vp.vr.stats.CopyRowCount.Get(), vreplicationStoreCompressedGTID)
+ update := binlogplayer.GenerateUpdatePos(vp.vr.id, vp.pos, time.Now().Unix(), ts, vp.vr.stats.CopyRowCount.Get(), vp.vr.workflowConfig.StoreCompressedGTID)
if _, err := vp.query(ctx, update); err != nil {
return false, fmt.Errorf("error %v updating position", err)
}
@@ -403,7 +408,7 @@ func (vp *vplayer) updatePos(ctx context.Context, ts int64) (posReached bool, er
}
func (vp *vplayer) mustUpdateHeartbeat() bool {
- return vp.numAccumulatedHeartbeats >= vreplicationHeartbeatUpdateInterval ||
+ return vp.numAccumulatedHeartbeats >= vp.vr.workflowConfig.HeartbeatUpdateInterval ||
vp.numAccumulatedHeartbeats >= vreplicationMinimumHeartbeatUpdateInterval
}
@@ -414,7 +419,7 @@ func (vp *vplayer) recordHeartbeat() error {
return nil
}
if err := vp.vr.updateHeartbeatTime(tm); err != nil {
- return vterrors.Wrapf(errVPlayerStalled, fmt.Sprintf("%s: %v", failedToRecordHeartbeatMsg, err))
+ return vterrors.Wrapf(errVPlayerStalled, "%s: %v", failedToRecordHeartbeatMsg, err)
}
// Only reset the pending heartbeat count if the update was successful.
// Otherwise the vplayer may not actually be making progress and nobody
@@ -476,12 +481,18 @@ func (vp *vplayer) recordHeartbeat() error {
func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
defer vp.vr.dbClient.Rollback()
+ estimateLag := func() {
+ behind := time.Now().UnixNano() - vp.lastTimestampNs - vp.timeOffsetNs
+ vp.vr.stats.ReplicationLagSeconds.Store(behind / 1e9)
+ vp.vr.stats.VReplicationLags.Add(strconv.Itoa(int(vp.vr.id)), time.Duration(behind/1e9)*time.Second)
+ }
+
// If we're not running, set ReplicationLagSeconds to be very high.
// TODO(sougou): if we also stored the time of the last event, we
// can estimate this value more accurately.
defer vp.vr.stats.ReplicationLagSeconds.Store(math.MaxInt64)
defer vp.vr.stats.VReplicationLags.Add(strconv.Itoa(int(vp.vr.id)), math.MaxInt64)
- var sbm int64 = -1
+ var lagSecs int64
for {
if ctx.Err() != nil {
return ctx.Err()
@@ -489,6 +500,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
// Check throttler.
if checkResult, ok := vp.vr.vre.throttlerClient.ThrottleCheckOKOrWaitAppName(ctx, throttlerapp.Name(vp.throttlerAppName)); !ok {
_ = vp.vr.updateTimeThrottled(throttlerapp.VPlayerName, checkResult.Summary())
+ estimateLag()
continue
}
@@ -496,13 +508,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
if err != nil {
return err
}
- // No events were received. This likely means that there's a network partition.
- // So, we should assume we're falling behind.
- if len(items) == 0 {
- behind := time.Now().UnixNano() - vp.lastTimestampNs - vp.timeOffsetNs
- vp.vr.stats.ReplicationLagSeconds.Store(behind / 1e9)
- vp.vr.stats.VReplicationLags.Add(strconv.Itoa(int(vp.vr.id)), time.Duration(behind/1e9)*time.Second)
- }
+
// Empty transactions are saved at most once every idleTimeout.
// This covers two situations:
// 1. Fetch was idle for idleTimeout.
@@ -520,12 +526,20 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
}
}
+ lagSecs = -1
for i, events := range items {
for j, event := range events {
if event.Timestamp != 0 {
- vp.lastTimestampNs = event.Timestamp * 1e9
- vp.timeOffsetNs = time.Now().UnixNano() - event.CurrentTime
- sbm = event.CurrentTime/1e9 - event.Timestamp
+ // If the event is a heartbeat sent while throttled then do not update
+ // the lag based on it.
+ // If the batch consists only of throttled heartbeat events then we cannot
+ // determine the actual lag, as the vstreamer is fully throttled, and we
+ // will estimate it after processing the batch.
+ if !(event.Type == binlogdatapb.VEventType_HEARTBEAT && event.Throttled) {
+ vp.lastTimestampNs = event.Timestamp * 1e9
+ vp.timeOffsetNs = time.Now().UnixNano() - event.CurrentTime
+ lagSecs = event.CurrentTime/1e9 - event.Timestamp
+ }
}
mustSave := false
switch event.Type {
@@ -549,7 +563,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
if err := vp.applyEvent(ctx, event, mustSave); err != nil {
if err != io.EOF {
vp.vr.stats.ErrorCounts.Add([]string{"Apply"}, 1)
- var table, tableLogMsg string
+ var table, tableLogMsg, gtidLogMsg string
switch {
case event.GetFieldEvent() != nil:
table = event.GetFieldEvent().TableName
@@ -559,18 +573,24 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error {
if table != "" {
tableLogMsg = fmt.Sprintf(" for table %s", table)
}
- log.Errorf("Error applying event%s: %s", tableLogMsg, err.Error())
+ pos := getNextPosition(items, i, j+1)
+ if pos != "" {
+ gtidLogMsg = fmt.Sprintf(" while processing position %s", pos)
+ }
+ log.Errorf("Error applying event%s%s: %s", tableLogMsg, gtidLogMsg, err.Error())
+ err = vterrors.Wrapf(err, "error applying event%s%s", tableLogMsg, gtidLogMsg)
}
return err
}
}
}
- if sbm >= 0 {
- vp.vr.stats.ReplicationLagSeconds.Store(sbm)
- vp.vr.stats.VReplicationLags.Add(strconv.Itoa(int(vp.vr.id)), time.Duration(sbm)*time.Second)
+ if lagSecs >= 0 {
+ vp.vr.stats.ReplicationLagSeconds.Store(lagSecs)
+ vp.vr.stats.VReplicationLags.Add(strconv.Itoa(int(vp.vr.id)), time.Duration(lagSecs)*time.Second)
+ } else { // We couldn't determine the lag, so we need to estimate it
+ estimateLag()
}
-
}
}
@@ -592,6 +612,34 @@ func hasAnotherCommit(items [][]*binlogdatapb.VEvent, i, j int) bool {
return false
}
+// getNextPosition returns the GTID set/position we would be at if the current
+// transaction was committed. This is useful for error handling as we can then
+// determine which GTID we're failing to process from the source and examine the
+// binlog events for that GTID directly on the source to debug the issue.
+// This is needed as it's not as simple as the user incrementing the current
+// position in the stream by 1 as we may be skipping N intermediate GTIDs in the
+// stream due to filtering. For GTIDs that we filter out we still replicate the
+// GTID event itself, just without any internal events and a COMMIT event (see
+// the unsavedEvent handling).
+func getNextPosition(items [][]*binlogdatapb.VEvent, i, j int) string {
+ for i < len(items) {
+ for j < len(items[i]) {
+ switch items[i][j].Type {
+ case binlogdatapb.VEventType_GTID:
+ pos, err := binlogplayer.DecodePosition(items[i][j].Gtid)
+ if err != nil {
+ return ""
+ }
+ return pos.String()
+ }
+ j++
+ }
+ j = 0
+ i++
+ }
+ return ""
+}
+
func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, mustSave bool) error {
stats := NewVrLogStats(event.Type.String())
switch event.Type {
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
index b1925c3c64f..50d93e60e5a 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go
@@ -36,7 +36,7 @@ import (
"vitess.io/vitess/go/vt/binlog/binlogplayer"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logutil"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -53,11 +53,11 @@ func TestPlayerGeneratedInvisiblePrimaryKey(t *testing.T) {
execStatements(t, []string{
"SET @@session.sql_generate_invisible_primary_key=ON;",
- "create table t1(val varbinary(128))",
- fmt.Sprintf("create table %s.t1(val varbinary(128))", vrepldb),
- "create table t2(val varbinary(128))",
+ "create table t1(val varchar(128))",
+ fmt.Sprintf("create table %s.t1(val varchar(128))", vrepldb),
+ "create table t2(val varchar(128))",
"SET @@session.sql_generate_invisible_primary_key=OFF;",
- fmt.Sprintf("create table %s.t2(my_row_id int, val varbinary(128), primary key(my_row_id))", vrepldb),
+ fmt.Sprintf("create table %s.t2(my_row_id int, val varchar(128), primary key(my_row_id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -193,14 +193,18 @@ func TestPlayerInvisibleColumns(t *testing.T) {
}
func TestHeartbeatFrequencyFlag(t *testing.T) {
- origVReplicationHeartbeatUpdateInterval := vreplicationHeartbeatUpdateInterval
+ origVReplicationHeartbeatUpdateInterval := vttablet.DefaultVReplicationConfig.HeartbeatUpdateInterval
defer func() {
- vreplicationHeartbeatUpdateInterval = origVReplicationHeartbeatUpdateInterval
+ vttablet.DefaultVReplicationConfig.HeartbeatUpdateInterval = origVReplicationHeartbeatUpdateInterval
}()
stats := binlogplayer.NewStats()
defer stats.Stop()
- vp := &vplayer{vr: &vreplicator{dbClient: newVDBClient(realDBClientFactory(), stats), stats: stats}}
+ vp := &vplayer{vr: &vreplicator{
+ dbClient: newVDBClient(realDBClientFactory(), stats, vttablet.DefaultVReplicationConfig.RelayLogMaxItems),
+ stats: stats,
+ workflowConfig: vttablet.DefaultVReplicationConfig,
+ }}
type testcount struct {
count int
@@ -218,7 +222,7 @@ func TestHeartbeatFrequencyFlag(t *testing.T) {
}
for _, tcase := range testcases {
t.Run(tcase.name, func(t *testing.T) {
- vreplicationHeartbeatUpdateInterval = tcase.interval
+ vttablet.DefaultVReplicationConfig.HeartbeatUpdateInterval = tcase.interval
for _, tcount := range tcase.counts {
vp.numAccumulatedHeartbeats = tcount.count
require.Equal(t, tcount.mustUpdate, vp.mustUpdateHeartbeat())
@@ -231,8 +235,8 @@ func TestVReplicationTimeUpdated(t *testing.T) {
ctx := context.Background()
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -334,21 +338,21 @@ func TestCharPK(t *testing.T) {
output string
table string
data [][]string
- }{{ //binary(2)
+ }{{ // binary(2)
input: "insert into t1 values(1, 'a')",
- output: "insert into t1(id,val) values (1,'a\\0')",
+ output: "insert into t1(id,val) values (1,_binary'a\\0')",
table: "t1",
data: [][]string{
{"1", "a\000"},
},
}, {
input: "update t1 set id = 2 where val = 'a\000'",
- output: "update t1 set id=2 where val='a\\0'",
+ output: "update t1 set id=2 where val=_binary'a\\0'",
table: "t1",
data: [][]string{
{"2", "a\000"},
},
- }, { //char(2)
+ }, { // char(2)
input: "insert into t2 values(1, 'a')",
output: "insert into t2(id,val) values (1,'a')",
table: "t2",
@@ -362,21 +366,21 @@ func TestCharPK(t *testing.T) {
data: [][]string{
{"2", "a"},
},
- }, { //varbinary(2)
+ }, { // varbinary(2)
input: "insert into t3 values(1, 'a')",
- output: "insert into t3(id,val) values (1,'a')",
+ output: "insert into t3(id,val) values (1,_binary'a')",
table: "t3",
data: [][]string{
{"1", "a"},
},
}, {
input: "update t3 set id = 2 where val = 'a'",
- output: "update t3 set id=2 where val='a'",
+ output: "update t3 set id=2 where val=_binary'a'",
table: "t3",
data: [][]string{
{"2", "a"},
},
- }, { //varchar(2)
+ }, { // varchar(2)
input: "insert into t4 values(1, 'a')",
output: "insert into t4(id,val) values (1,'a')",
table: "t4",
@@ -575,9 +579,20 @@ func TestPlayerForeignKeyCheck(t *testing.T) {
cancel()
}
-func TestPlayerStatementModeWithFilter(t *testing.T) {
+// TestPlayerStatementModeWithFilterAndErrorHandling confirms that we get the
+// expected error when using a filter with statement mode. It also tests the
+// general vplayer applyEvent error and log message handling.
+func TestPlayerStatementModeWithFilterAndErrorHandling(t *testing.T) {
defer deleteTablet(addTablet(100))
+ // We want to check for the expected log message.
+ ole := log.Errorf
+ logger := logutil.NewMemoryLogger()
+ log.Errorf = logger.Errorf
+ defer func() {
+ log.Errorf = ole
+ }()
+
execStatements(t, []string{
"create table src1(id int, val varbinary(128), primary key(id))",
})
@@ -600,21 +615,28 @@ func TestPlayerStatementModeWithFilter(t *testing.T) {
cancel, _ := startVReplication(t, bls, "")
defer cancel()
+ const gtid = "37f16b4c-5a74-11ef-87de-56bfd605e62e:100"
input := []string{
"set @@session.binlog_format='STATEMENT'",
+ fmt.Sprintf("set @@session.gtid_next='%s'", gtid),
"insert into src1 values(1, 'aaa')",
+ "set @@session.gtid_next='AUTOMATIC'",
"set @@session.binlog_format='ROW'",
}
+ expectedMsg := fmt.Sprintf("[Ee]rror applying event while processing position .*%s.* filter rules are not supported for SBR.*", gtid)
+
// It does not work when filter is enabled
output := qh.Expect(
- "begin",
"rollback",
- "/update _vt.vreplication set message='filter rules are not supported for SBR",
+ fmt.Sprintf("/update _vt.vreplication set message='%s", expectedMsg),
)
execStatements(t, input)
expectDBClientQueries(t, output)
+
+ logs := logger.String()
+ require.Regexp(t, expectedMsg, logs)
}
func TestPlayerStatementMode(t *testing.T) {
@@ -725,7 +747,7 @@ func TestPlayerFilters(t *testing.T) {
Filter: "select id1, val from src5 where val = 'abc'",
}, {
Match: "dst_charset",
- Filter: "select id1, concat(substr(_utf8mb4 val collate utf8mb4_bin,1,1),'abcxyz') val, concat(substr(_utf8mb4 val collate utf8mb4_bin,1,1),'abcxyz') val2 from src_charset",
+ Filter: "select id1, concat(substr(CONVERT(val USING utf8mb4) COLLATE utf8mb4_bin,1,1),'abcxyz') val, concat(substr(CONVERT(val USING utf8mb4) COLLATE utf8mb4_bin,1,1),'abcxyz') val2 from src_charset",
}},
}
bls := &binlogdatapb.BinlogSource{
@@ -749,7 +771,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into src1 values(1, 'aaa')",
output: qh.Expect(
"begin",
- "insert into dst1(id,val) values (1,'aaa')",
+ "insert into dst1(id,val) values (1,_binary'aaa')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -759,7 +781,7 @@ func TestPlayerFilters(t *testing.T) {
},
logs: []LogExpectation{
{"FIELD", "/src1.*id.*INT32.*val.*VARBINARY.*"},
- {"ROWCHANGE", "insert into dst1(id,val) values (1,'aaa')"},
+ {"ROWCHANGE", "insert into dst1(id,val) values (1,_binary'aaa')"},
{"ROW", "/src1.*3.*1aaa.*"},
},
}, {
@@ -767,7 +789,7 @@ func TestPlayerFilters(t *testing.T) {
input: "update src1 set val='bbb'",
output: qh.Expect(
"begin",
- "update dst1 set val='bbb' where id=1",
+ "update dst1 set val=_binary'bbb' where id=1",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -776,7 +798,7 @@ func TestPlayerFilters(t *testing.T) {
{"1", "bbb"},
},
logs: []LogExpectation{
- {"ROWCHANGE", "update dst1 set val='bbb' where id=1"},
+ {"ROWCHANGE", "update dst1 set val=_binary'bbb' where id=1"},
{"ROW", "/src1.*3.*1aaa.*"},
},
}, {
@@ -846,7 +868,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into src3 values(1, 'aaa')",
output: qh.Expect(
"begin",
- "insert ignore into dst3(id,val) values (1,'aaa')",
+ "insert ignore into dst3(id,val) values (1,_binary'aaa')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -859,7 +881,7 @@ func TestPlayerFilters(t *testing.T) {
input: "update src3 set val='bbb'",
output: qh.Expect(
"begin",
- "insert ignore into dst3(id,val) values (1,'bbb')",
+ "insert ignore into dst3(id,val) values (1,_binary'bbb')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -884,7 +906,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into yes values(1, 'aaa')",
output: qh.Expect(
"begin",
- "insert into yes(id,val) values (1,'aaa')",
+ "insert into yes(id,val) values (1,_binary'aaa')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -897,7 +919,7 @@ func TestPlayerFilters(t *testing.T) {
input: "update yes set val='bbb'",
output: qh.Expect(
"begin",
- "update yes set val='bbb' where id=1",
+ "update yes set val=_binary'bbb' where id=1",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -914,7 +936,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into nopk values(1, 'aaa')",
output: qh.Expect(
"begin",
- "insert into nopk(id,val) values (1,'aaa')",
+ "insert into nopk(id,val) values (1,_binary'aaa')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -927,8 +949,8 @@ func TestPlayerFilters(t *testing.T) {
input: "update nopk set val='bbb' where id=1",
output: qh.Expect(
"begin",
- "delete from nopk where id=1 and val='aaa'",
- "insert into nopk(id,val) values (1,'bbb')",
+ "delete from nopk where id=1 and val=_binary'aaa'",
+ "insert into nopk(id,val) values (1,_binary'bbb')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -941,7 +963,7 @@ func TestPlayerFilters(t *testing.T) {
input: "delete from nopk where id=1",
output: qh.Expect(
"begin",
- "delete from nopk where id=1 and val='bbb'",
+ "delete from nopk where id=1 and val=_binary'bbb'",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -952,8 +974,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into src4 values (1,100,'aaa'),(2,200,'bbb'),(3,100,'ccc')",
output: qh.Expect(
"begin",
- "insert into dst4(id1,val) values (1,'aaa')",
- "insert into dst4(id1,val) values (3,'ccc')",
+ "insert into dst4(id1,val) values (1,_binary'aaa'), (3,_binary'ccc')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -964,8 +985,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into src5 values (1,100,'abc'),(2,200,'xyz'),(3,100,'xyz'),(4,300,'abc'),(5,200,'xyz')",
output: qh.Expect(
"begin",
- "insert into dst5(id1,val) values (1,'abc')",
- "insert into dst5(id1,val) values (4,'abc')",
+ "insert into dst5(id1,val) values (1,_binary'abc'), (4,_binary'abc')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -976,7 +996,7 @@ func TestPlayerFilters(t *testing.T) {
input: "insert into src_charset values (1,'木元')",
output: qh.Expect(
"begin",
- "insert into dst_charset(id1,val,val2) values (1,concat(substr(_utf8mb4 '木元' collate utf8mb4_bin, 1, 1), 'abcxyz'),concat(substr(_utf8mb4 '木元' collate utf8mb4_bin, 1, 1), 'abcxyz'))",
+ "insert into dst_charset(id1,val,val2) values (1,concat(substr(convert(_binary'木元' using utf8mb4) collate utf8mb4_bin, 1, 1), 'abcxyz'),concat(substr(convert(_binary'木元' using utf8mb4) collate utf8mb4_bin, 1, 1), 'abcxyz'))",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -1003,12 +1023,12 @@ func TestPlayerKeywordNames(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table `begin`(`primary` int, `column` varbinary(128), primary key(`primary`))",
- fmt.Sprintf("create table %s.`begin`(`primary` int, `column` varbinary(128), primary key(`primary`))", vrepldb),
- "create table `rollback`(`primary` int, `column` varbinary(128), primary key(`primary`))",
- fmt.Sprintf("create table %s.`rollback`(`primary` int, `column` varbinary(128), primary key(`primary`))", vrepldb),
- "create table `commit`(`primary` int, `column` varbinary(128), primary key(`primary`))",
- fmt.Sprintf("create table %s.`commit`(`primary` int, `column` varbinary(128), primary key(`primary`))", vrepldb),
+ "create table `begin`(`primary` int, `column` varchar(128), primary key(`primary`))",
+ fmt.Sprintf("create table %s.`begin`(`primary` int, `column` varchar(128), primary key(`primary`))", vrepldb),
+ "create table `rollback`(`primary` int, `column` varchar(128), primary key(`primary`))",
+ fmt.Sprintf("create table %s.`rollback`(`primary` int, `column` varchar(128), primary key(`primary`))", vrepldb),
+ "create table `commit`(`primary` int, `column` varchar(128), primary key(`primary`))",
+ fmt.Sprintf("create table %s.`commit`(`primary` int, `column` varchar(128), primary key(`primary`))", vrepldb),
})
defer execStatements(t, []string{
"drop table `begin`",
@@ -1234,7 +1254,7 @@ func TestPlayerKeyspaceID(t *testing.T) {
input: "insert into src1 values(1, 'aaa')",
output: qh.Expect(
"begin",
- "insert into dst1(id,val) values (1,'\x16k@\xb4J\xbaK\xd6')",
+ "insert into dst1(id,val) values (1,_binary'\x16k@\xb4J\xbaK\xd6')",
"/update _vt.vreplication set pos=",
"commit",
),
@@ -1472,9 +1492,7 @@ func TestPlayerRowMove(t *testing.T) {
})
expectDBClientQueries(t, qh.Expect(
"begin",
- "insert into dst(val1,sval2,rcount) values (1,ifnull(1, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1",
- "insert into dst(val1,sval2,rcount) values (2,ifnull(2, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1",
- "insert into dst(val1,sval2,rcount) values (2,ifnull(3, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1",
+ "insert into dst(val1,sval2,rcount) values (1,ifnull(1, 0),1), (2,ifnull(2, 0),1), (2,ifnull(3, 0),1) on duplicate key update sval2=sval2+ifnull(values(sval2), 0), rcount=rcount+1",
"/update _vt.vreplication set pos=",
"commit",
))
@@ -1482,7 +1500,7 @@ func TestPlayerRowMove(t *testing.T) {
{"1", "1", "1"},
{"2", "5", "2"},
})
- validateQueryCountStat(t, "replicate", 3)
+ validateQueryCountStat(t, "replicate", 1)
execStatements(t, []string{
"update src set val1=1, val2=4 where id=3",
@@ -1498,7 +1516,7 @@ func TestPlayerRowMove(t *testing.T) {
{"1", "5", "2"},
{"2", "2", "1"},
})
- validateQueryCountStat(t, "replicate", 5)
+ validateQueryCountStat(t, "replicate", 3)
}
func TestPlayerTypes(t *testing.T) {
@@ -1579,7 +1597,7 @@ func TestPlayerTypes(t *testing.T) {
},
}, {
input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b,f,l,q,s,v,z')",
- output: "insert into vitess_strings(vb,c,vc,b,tb,bl,ttx,tx,en,s) values ('a','b','c','d\\0\\0\\0\\0','e','f','g','h','a','a,b,f,l,q,s,v,z')",
+ output: "insert into vitess_strings(vb,c,vc,b,tb,bl,ttx,tx,en,s) values (_binary'a','b','c',_binary'd\\0\\0\\0\\0',_binary'e',_binary'f','g','h','a','a,b,f,l,q,s,v,z')",
table: "vitess_strings",
data: [][]string{
{"a", "b", "c", "d\000\000\000\000", "e", "f", "g", "h", "a", "a,b,f,l,q,s,v,z"},
@@ -1600,7 +1618,7 @@ func TestPlayerTypes(t *testing.T) {
},
}, {
input: "insert into binary_pk values('a', 'aaa')",
- output: "insert into binary_pk(b,val) values ('a\\0\\0\\0','aaa')",
+ output: "insert into binary_pk(b,val) values (_binary'a\\0\\0\\0',_binary'aaa')",
table: "binary_pk",
data: [][]string{
{"a\000\000\000", "aaa"},
@@ -1615,7 +1633,7 @@ func TestPlayerTypes(t *testing.T) {
}, {
// Binary pk is a special case: https://github.com/vitessio/vitess/issues/3984
input: "update binary_pk set val='bbb' where b='a\\0\\0\\0'",
- output: "update binary_pk set val='bbb' where b='a\\0\\0\\0'",
+ output: "update binary_pk set val=_binary'bbb' where b=_binary'a\\0\\0\\0'",
table: "binary_pk",
data: [][]string{
{"a\000\000\000", "bbb"},
@@ -1709,7 +1727,7 @@ func TestPlayerDDL(t *testing.T) {
OnDdl: binlogdatapb.OnDDLAction_STOP,
}
cancel, id := startVReplication(t, bls, "")
- pos0 := primaryPosition(t) //For debugging only
+ pos0 := primaryPosition(t) // For debugging only
execStatements(t, []string{"alter table t1 add column val varchar(128)"})
pos1 := primaryPosition(t)
// The stop position must be the GTID of the first DDL
@@ -1723,7 +1741,7 @@ func TestPlayerDDL(t *testing.T) {
execStatements(t, []string{"alter table t1 drop column val"})
pos2 := primaryPosition(t)
log.Errorf("Expected log:: TestPlayerDDL Positions are: before first alter %v, after first alter %v, before second alter %v, after second alter %v",
- pos0, pos1, pos2b, pos2) //For debugging only: to check what are the positions when test works and if/when it fails
+ pos0, pos1, pos2b, pos2) // For debugging only: to check what are the positions when test works and if/when it fails
// Restart vreplication
if _, err := playerEngine.Exec(fmt.Sprintf(`update _vt.vreplication set state = 'Running', message='' where id=%d`, id)); err != nil {
t.Fatal(err)
@@ -1758,8 +1776,8 @@ func TestPlayerDDL(t *testing.T) {
execStatements(t, []string{"alter table t1 add column val2 varchar(128)"})
expectDBClientQueries(t, qh.Expect(
"alter table t1 add column val2 varchar(128)",
- "/update _vt.vreplication set message='Duplicate",
- "/update _vt.vreplication set state='Error', message='Duplicate",
+ "/update _vt.vreplication set message='error applying event: Duplicate",
+ "/update _vt.vreplication set state='Error', message='terminal error: error applying event: Duplicate",
))
cancel()
@@ -1846,14 +1864,14 @@ func TestGTIDCompress(t *testing.T) {
func TestPlayerStopPos(t *testing.T) {
defer deleteTablet(addTablet(100))
- vreplicationStoreCompressedGTID = true
+ vttablet.DefaultVReplicationConfig.StoreCompressedGTID = true
defer func() {
- vreplicationStoreCompressedGTID = false
+ vttablet.DefaultVReplicationConfig.StoreCompressedGTID = false
}()
execStatements(t, []string{
- "create table yes(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.yes(id int, val varbinary(128), primary key(id))", vrepldb),
- "create table no(id int, val varbinary(128), primary key(id))",
+ "create table yes(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.yes(id int, val varchar(128), primary key(id))", vrepldb),
+ "create table no(id int, val varchar(128), primary key(id))",
})
defer execStatements(t, []string{
"drop table yes",
@@ -2061,8 +2079,8 @@ func TestPlayerIdleUpdate(t *testing.T) {
idleTimeout = 100 * time.Millisecond
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2114,8 +2132,8 @@ func TestPlayerSplitTransaction(t *testing.T) {
defer setFlag("vstream_packet_size", "10000")
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2156,9 +2174,17 @@ func TestPlayerSplitTransaction(t *testing.T) {
func TestPlayerLockErrors(t *testing.T) {
defer deleteTablet(addTablet(100))
+ // The immediate retry behavior does not apply when doing
+ // VPlayer Batching.
+ origExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = 0
+ defer func() {
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = origExperimentalFlags
+ }()
+
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2235,9 +2261,17 @@ func TestPlayerLockErrors(t *testing.T) {
func TestPlayerCancelOnLock(t *testing.T) {
defer deleteTablet(addTablet(100))
+ // The immediate retry behavior does not apply when doing
+ // VPlayer Batching.
+ origExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = 0
+ defer func() {
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = origExperimentalFlags
+ }()
+
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2313,8 +2347,8 @@ func TestPlayerTransactions(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2408,18 +2442,18 @@ func TestPlayerRelayLogMaxSize(t *testing.T) {
func() {
switch i {
case 0:
- savedSize := relayLogMaxSize
- defer func() { relayLogMaxSize = savedSize }()
- relayLogMaxSize = 10
+ savedSize := vttablet.DefaultVReplicationConfig.RelayLogMaxSize
+ defer func() { vttablet.DefaultVReplicationConfig.RelayLogMaxSize = savedSize }()
+ vttablet.DefaultVReplicationConfig.RelayLogMaxSize = 10
case 1:
- savedLen := relayLogMaxItems
- defer func() { relayLogMaxItems = savedLen }()
- relayLogMaxItems = 2
+ savedLen := vttablet.DefaultVReplicationConfig.RelayLogMaxItems
+ defer func() { vttablet.DefaultVReplicationConfig.RelayLogMaxItems = savedLen }()
+ vttablet.DefaultVReplicationConfig.RelayLogMaxItems = 2
}
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2507,13 +2541,13 @@ func TestPlayerRelayLogMaxSize(t *testing.T) {
func TestRestartOnVStreamEnd(t *testing.T) {
defer deleteTablet(addTablet(100))
- savedDelay := retryDelay
- defer func() { retryDelay = savedDelay }()
- retryDelay = 1 * time.Millisecond
+ savedDelay := vttablet.DefaultVReplicationConfig.RetryDelay
+ defer func() { vttablet.DefaultVReplicationConfig.RetryDelay = savedDelay }()
+ vttablet.DefaultVReplicationConfig.RetryDelay = 1 * time.Millisecond
execStatements(t, []string{
- "create table t1(id int, val varbinary(128), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(128), primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(128), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(128), primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2755,7 +2789,7 @@ func TestVReplicationLogs(t *testing.T) {
err := dbClient.Connect()
require.NoError(t, err)
defer dbClient.Close()
- vdbc := newVDBClient(dbClient, binlogplayer.NewStats())
+ vdbc := newVDBClient(dbClient, binlogplayer.NewStats(), vttablet.DefaultVReplicationConfig.RelayLogMaxItems)
query := "select vrepl_id, state, message, count from _vt.vreplication_log order by id desc limit 1"
expected := []string{
@@ -2778,10 +2812,10 @@ func TestGeneratedColumns(t *testing.T) {
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table t1(id int, val varbinary(6), val2 varbinary(6) as (concat(id, val)), val3 varbinary(6) as (concat(val, id)), id2 int, primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val varbinary(6), val2 varbinary(6) as (concat(id, val)), val3 varbinary(6), id2 int, primary key(id))", vrepldb),
- "create table t2(id int, val varbinary(128), val2 varbinary(128) as (concat(id, val)) stored, val3 varbinary(128) as (concat(val, id)), id2 int, primary key(id))",
- fmt.Sprintf("create table %s.t2(id int, val3 varbinary(128), val varbinary(128), id2 int, primary key(id))", vrepldb),
+ "create table t1(id int, val varchar(6), val2 varchar(6) as (concat(id, val)), val3 varchar(6) as (concat(val, id)), id2 int, primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val varchar(6), val2 varchar(6) as (concat(id, val)), val3 varchar(6), id2 int, primary key(id))", vrepldb),
+ "create table t2(id int, val varchar(128), val2 varchar(128) as (concat(id, val)) stored, val3 varchar(128) as (concat(val, id)), id2 int, primary key(id))",
+ fmt.Sprintf("create table %s.t2(id int, val3 varchar(128), val varchar(128), id2 int, primary key(id))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -2931,18 +2965,18 @@ func TestPlayerNoBlob(t *testing.T) {
if !runNoBlobTest {
t.Skip()
}
- oldVreplicationExperimentalFlags := vttablet.VReplicationExperimentalFlags
- vttablet.VReplicationExperimentalFlags = vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage
+ oldVreplicationExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage
defer func() {
- vttablet.VReplicationExperimentalFlags = oldVreplicationExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = oldVreplicationExperimentalFlags
}()
defer deleteTablet(addTablet(100))
execStatements(t, []string{
- "create table t1(id int, val1 varchar(20), blb1 blob, id2 int, blb2 longblob, val2 varbinary(10), primary key(id))",
- fmt.Sprintf("create table %s.t1(id int, val1 varchar(20), blb1 blob, id2 int, blb2 longblob, val2 varbinary(10), primary key(id))", vrepldb),
- "create table t2(id int, val1 varchar(20), txt1 text, id2 int, val2 varbinary(10), unique key(id, val1))",
- fmt.Sprintf("create table %s.t2(id int, val1 varchar(20), txt1 text, id2 int, val2 varbinary(10), primary key(id, val1))", vrepldb),
+ "create table t1(id int, val1 varchar(20), blb1 text, id2 int, blb2 longtext, val2 varchar(10), primary key(id))",
+ fmt.Sprintf("create table %s.t1(id int, val1 varchar(20), blb1 text, id2 int, blb2 longtext, val2 varchar(10), primary key(id))", vrepldb),
+ "create table t2(id int, val1 varchar(20), txt1 text, id2 int, val2 varchar(10), unique key(id, val1))",
+ fmt.Sprintf("create table %s.t2(id int, val1 varchar(20), txt1 text, id2 int, val2 varchar(10), primary key(id, val1))", vrepldb),
})
defer execStatements(t, []string{
"drop table t1",
@@ -3071,10 +3105,10 @@ func TestPlayerNoBlob(t *testing.T) {
func TestPlayerBatchMode(t *testing.T) {
// To test trx batch splitting at 1024-64 bytes.
maxAllowedPacket := 1024
- oldVreplicationExperimentalFlags := vttablet.VReplicationExperimentalFlags
- vttablet.VReplicationExperimentalFlags = vttablet.VReplicationExperimentalFlagVPlayerBatching
+ oldVreplicationExperimentalFlags := vttablet.DefaultVReplicationConfig.ExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = vttablet.VReplicationExperimentalFlagVPlayerBatching
defer func() {
- vttablet.VReplicationExperimentalFlags = oldVreplicationExperimentalFlags
+ vttablet.DefaultVReplicationConfig.ExperimentalFlags = oldVreplicationExperimentalFlags
}()
defer deleteTablet(addTablet(100))
@@ -3329,16 +3363,16 @@ func TestPlayerStalls(t *testing.T) {
logger := logutil.NewMemoryLogger()
log.Errorf = logger.Errorf
- ovmhu := vreplicationMinimumHeartbeatUpdateInterval
- ogvpt := vplayerProgressDeadline
- orlmi := relayLogMaxItems
- ord := retryDelay
+ oldMinimumHeartbeatUpdateInterval := vreplicationMinimumHeartbeatUpdateInterval
+ oldProgressDeadline := vplayerProgressDeadline
+ oldRelayLogMaxItems := vttablet.DefaultVReplicationConfig.RelayLogMaxItems
+ oldRetryDelay := vttablet.DefaultVReplicationConfig.RetryDelay
defer func() {
log.Errorf = ole
- vreplicationMinimumHeartbeatUpdateInterval = ovmhu
- vplayerProgressDeadline = ogvpt
- relayLogMaxItems = orlmi
- retryDelay = ord
+ vreplicationMinimumHeartbeatUpdateInterval = oldMinimumHeartbeatUpdateInterval
+ vplayerProgressDeadline = oldProgressDeadline
+ vttablet.DefaultVReplicationConfig.RelayLogMaxItems = oldRelayLogMaxItems
+ vttablet.DefaultVReplicationConfig.RetryDelay = oldRetryDelay
}()
// Shorten the deadline for the test.
@@ -3346,11 +3380,11 @@ func TestPlayerStalls(t *testing.T) {
// Shorten the time for a required heartbeat recording for the test.
vreplicationMinimumHeartbeatUpdateInterval = 5
// So each relay log batch will be a single statement transaction.
- relayLogMaxItems = 1
+ vttablet.DefaultVReplicationConfig.RelayLogMaxItems = 1
// Don't retry the workflow if it goes into the error state.
- retryDelay = 10 * time.Minute
- maxTimeToRetryError = 1 * time.Second
+ vttablet.DefaultVReplicationConfig.RetryDelay = 10 * time.Minute
+ vttablet.DefaultVReplicationConfig.MaxTimeToRetryError = 1 * time.Second
// A channel to communicate across goroutines.
done := make(chan struct{})
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go
index 2a4d598c960..42701288a44 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go
@@ -37,6 +37,7 @@ import (
"vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -113,6 +114,7 @@ type vreplicator struct {
WorkflowName string
throttleUpdatesRateLimiter *timer.RateLimiter
+ workflowConfig *vttablet.VReplicationConfig
}
// newVReplicator creates a new vreplicator. The valid fields from the source are:
@@ -137,19 +139,25 @@ type vreplicator struct {
// alias like "a+b as targetcol" must be used.
// More advanced constructs can be used. Please see the table plan builder
// documentation for more info.
-func newVReplicator(id int32, source *binlogdatapb.BinlogSource, sourceVStreamer VStreamerClient, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon, vre *Engine) *vreplicator {
- if vreplicationHeartbeatUpdateInterval > vreplicationMinimumHeartbeatUpdateInterval {
+func newVReplicator(id int32, source *binlogdatapb.BinlogSource, sourceVStreamer VStreamerClient, stats *binlogplayer.Stats,
+ dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon, vre *Engine, workflowConfig *vttablet.VReplicationConfig) *vreplicator {
+ if workflowConfig == nil {
+ workflowConfig = vttablet.DefaultVReplicationConfig
+ }
+ if workflowConfig.HeartbeatUpdateInterval > vreplicationMinimumHeartbeatUpdateInterval {
log.Warningf("The supplied value for vreplication_heartbeat_update_interval:%d seconds is larger than the maximum allowed:%d seconds, vreplication will fallback to %d",
- vreplicationHeartbeatUpdateInterval, vreplicationMinimumHeartbeatUpdateInterval, vreplicationMinimumHeartbeatUpdateInterval)
+ workflowConfig.HeartbeatUpdateInterval, vreplicationMinimumHeartbeatUpdateInterval, vreplicationMinimumHeartbeatUpdateInterval)
}
+ vttablet.InitVReplicationConfigDefaults()
vr := &vreplicator{
vre: vre,
id: id,
source: source,
sourceVStreamer: sourceVStreamer,
stats: stats,
- dbClient: newVDBClient(dbClient, stats),
+ dbClient: newVDBClient(dbClient, stats, workflowConfig.RelayLogMaxItems),
mysqld: mysqld,
+ workflowConfig: workflowConfig,
}
vr.setExistingRowsCopied()
return vr
@@ -178,11 +186,18 @@ func newVReplicator(id int32, source *binlogdatapb.BinlogSource, sourceVStreamer
// code.
func (vr *vreplicator) Replicate(ctx context.Context) error {
err := vr.replicate(ctx)
- if err != nil {
- if err := vr.setMessage(err.Error()); err != nil {
- binlogplayer.LogError("Failed to set error state", err)
+ if err == nil {
+ return nil
+ }
+ if vr.dbClient.IsClosed() {
+ // Connection was possible terminated by the server. We should renew it.
+ if cerr := vr.dbClient.Connect(); cerr != nil {
+ return vterrors.Wrapf(err, "failed to reconnect to the database: %v", cerr)
}
}
+ if err := vr.setMessage(err.Error()); err != nil {
+ binlogplayer.LogError("Failed to set error state", err)
+ }
return err
}
@@ -239,7 +254,7 @@ func (vr *vreplicator) replicate(ctx context.Context) error {
if err := vr.getSettingFKCheck(); err != nil {
return err
}
- //defensive guard, should be a no-op since it should happen after copy is done
+ // defensive guard, should be a no-op since it should happen after copy is done
defer vr.resetFKCheckAfterCopy(vr.dbClient)
if err := vr.getSettingFKRestrict(); err != nil {
return err
@@ -493,8 +508,14 @@ func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, me
}
vr.stats.State.Store(state.String())
query := fmt.Sprintf("update _vt.vreplication set state='%v', message=%v where id=%v", state, encodeString(binlogplayer.MessageTruncate(message)), vr.id)
- if _, err := vr.dbClient.ExecuteFetch(query, 1); err != nil {
- return fmt.Errorf("could not set state: %v: %v", query, err)
+ // If we're batching a transaction, then include the state update
+ // in the current transaction batch.
+ if vr.dbClient.InTransaction && vr.dbClient.maxBatchSize > 0 {
+ vr.dbClient.AddQueryToTrxBatch(query)
+ } else { // Otherwise, send it down the wire
+ if _, err := vr.dbClient.ExecuteFetch(query, 1); err != nil {
+ return fmt.Errorf("could not set state: %v: %v", query, err)
+ }
}
if state == vr.state {
return nil
@@ -1105,7 +1126,7 @@ func (vr *vreplicator) newClientConnection(ctx context.Context) (*vdbClient, err
if err := dbc.Connect(); err != nil {
return nil, vterrors.Wrap(err, "can't connect to database")
}
- dbClient := newVDBClient(dbc, vr.stats)
+ dbClient := newVDBClient(dbc, vr.stats, vr.workflowConfig.RelayLogMaxItems)
if _, err := vr.setSQLMode(ctx, dbClient); err != nil {
return nil, vterrors.Wrap(err, "failed to set sql_mode")
}
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go
index 20a5450741d..b4e3ba4e366 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator_test.go
@@ -36,6 +36,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/mysqlctl"
"vitess.io/vitess/go/vt/schemadiff"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
@@ -232,7 +233,7 @@ func TestDeferSecondaryKeys(t *testing.T) {
_, err = dbClient.ExecuteFetch(fmt.Sprintf("delete from _vt.vreplication where id = %d", id), 1)
require.NoError(t, err)
}()
- vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine)
+ vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine, vttablet.DefaultVReplicationConfig)
getActionsSQLf := "select action from _vt.post_copy_action where table_name='%s'"
getCurrentDDL := func(tableName string) string {
req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{tableName}}
@@ -386,7 +387,7 @@ func TestDeferSecondaryKeys(t *testing.T) {
if err != nil {
return err
}
- myvr := newVReplicator(myid, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine)
+ myvr := newVReplicator(myid, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine, vttablet.DefaultVReplicationConfig)
myvr.WorkflowType = int32(binlogdatapb.VReplicationWorkflowType_Reshard)
// Insert second post copy action record to simulate a shard merge where you
// have N controllers/replicators running for the same table on the tablet.
@@ -630,7 +631,7 @@ func TestCancelledDeferSecondaryKeys(t *testing.T) {
_, err = dbClient.ExecuteFetch(fmt.Sprintf("delete from _vt.vreplication where id = %d", id), 1)
require.NoError(t, err)
}()
- vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine)
+ vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine, vttablet.DefaultVReplicationConfig)
vr.WorkflowType = int32(binlogdatapb.VReplicationWorkflowType_MoveTables)
getCurrentDDL := func(tableName string) string {
req := &tabletmanagerdatapb.GetSchemaRequest{Tables: []string{tableName}}
@@ -692,7 +693,7 @@ func TestCancelledDeferSecondaryKeys(t *testing.T) {
assert.NoError(t, err)
assert.Equal(t, 1, len(res.Rows))
// TODO: figure out why the KILL never shows up...
- //require.Equal(t, "1", res.Rows[0][0].ToString())
+ // require.Equal(t, "1", res.Rows[0][0].ToString())
// Confirm that the post copy action record still exists
// so it will later be retried.
@@ -749,7 +750,7 @@ func TestResumingFromPreviousWorkflowKeepingRowsCopied(t *testing.T) {
_, err = dbClient.ExecuteFetch(fmt.Sprintf("delete from _vt.vreplication where id = %d", id), 1)
require.NoError(t, err)
}()
- vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine)
+ vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine, vttablet.DefaultVReplicationConfig)
assert.Equal(t, rowsCopied, vr.stats.CopyRowCount.Get())
}
@@ -850,8 +851,8 @@ func TestThrottlerAppNames(t *testing.T) {
_, err = dbClient.ExecuteFetch(fmt.Sprintf("delete from _vt.vreplication where id = %d", id), 1)
require.NoError(t, err)
}()
- vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine)
- settings, _, err := vr.loadSettings(ctx, newVDBClient(dbClient, stats))
+ vr := newVReplicator(id, bls, vsclient, stats, dbClient, env.Mysqld, playerEngine, vttablet.DefaultVReplicationConfig)
+ settings, _, err := vr.loadSettings(ctx, newVDBClient(dbClient, stats, vttablet.DefaultVReplicationConfig.RelayLogMaxItems))
require.NoError(t, err)
throttlerAppName := vr.throttlerAppName()
diff --git a/go/vt/vttablet/tabletmanager/vreplication/vrlog_test.go b/go/vt/vttablet/tabletmanager/vreplication/vrlog_test.go
index 3bd0dcf3217..48fe7fa983e 100644
--- a/go/vt/vttablet/tabletmanager/vreplication/vrlog_test.go
+++ b/go/vt/vttablet/tabletmanager/vreplication/vrlog_test.go
@@ -23,6 +23,8 @@ import (
"strings"
"testing"
"time"
+
+ "github.com/stretchr/testify/require"
)
func TestVrLog(t *testing.T) {
@@ -49,9 +51,7 @@ func TestVrLog(t *testing.T) {
}
want := fmt.Sprintf("%s Event %s", eventType, detail)
- if !strings.Contains(s, want) {
- t.Fatalf(fmt.Sprintf("want %s, got %s", want, s))
- }
+ require.Contains(t, s, want)
if strings.HasSuffix(s, "\\n") {
t.Fatalf("does not end in a newline: %s", s)
}
@@ -65,9 +65,7 @@ func TestVrLog(t *testing.T) {
if err != nil {
t.Fatalf("Duration is not an integer: %s", err)
}
- if lastColValue == 0 {
- t.Fatalf("Duration should not be zero")
- }
+ require.NotZero(t, lastColValue, "duration")
stats = &VrLogStats{}
stats.Send("detail123")
diff --git a/go/vt/vttablet/tabletserver/binlog_watcher.go b/go/vt/vttablet/tabletserver/binlog_watcher.go
index cff7697c18a..80ac1194c7e 100644
--- a/go/vt/vttablet/tabletserver/binlog_watcher.go
+++ b/go/vt/vttablet/tabletserver/binlog_watcher.go
@@ -31,7 +31,9 @@ import (
// VStreamer defines the functions of VStreamer
// that the BinlogWatcher needs.
type VStreamer interface {
- Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error
+ Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error,
+ options *binlogdatapb.VStreamOptions) error
}
// BinlogWatcher is a tabletserver service that watches the
@@ -93,7 +95,7 @@ func (blw *BinlogWatcher) process(ctx context.Context) {
// VStreamer will reload the schema when it encounters a DDL.
err := blw.vs.Stream(ctx, "current", nil, filter, throttlerapp.BinlogWatcherName, func(events []*binlogdatapb.VEvent) error {
return nil
- })
+ }, nil)
log.Infof("ReplicationWatcher VStream ended: %v, retrying in 5 seconds", err)
select {
case <-ctx.Done():
diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn.go b/go/vt/vttablet/tabletserver/connpool/dbconn.go
index af8c5fbc78e..fcadc3d4075 100644
--- a/go/vt/vttablet/tabletserver/connpool/dbconn.go
+++ b/go/vt/vttablet/tabletserver/connpool/dbconn.go
@@ -166,7 +166,7 @@ func (dbc *Conn) execOnce(ctx context.Context, query string, maxrows int, wantfi
// Check if the context is already past its deadline before
// trying to execute the query.
if err := ctx.Err(); err != nil {
- return nil, fmt.Errorf("%v before execution started", err)
+ return nil, vterrors.Errorf(vtrpcpb.Code_CANCELED, "%s before execution started", dbc.getErrorMessageFromContextError(ctx))
}
now := time.Now()
@@ -200,8 +200,8 @@ func (dbc *Conn) execOnce(ctx context.Context, query string, maxrows int, wantfi
}
}
-// terminate kills the query or connection based on the transaction status
-func (dbc *Conn) terminate(ctx context.Context, insideTxn bool, now time.Time) {
+// getErrorMessageFromContextError gets the error message from context error.
+func (dbc *Conn) getErrorMessageFromContextError(ctx context.Context) string {
var errMsg string
switch {
case errors.Is(ctx.Err(), context.DeadlineExceeded):
@@ -211,6 +211,12 @@ func (dbc *Conn) terminate(ctx context.Context, insideTxn bool, now time.Time) {
default:
errMsg = ctx.Err().Error()
}
+ return errMsg
+}
+
+// terminate kills the query or connection based on the transaction status
+func (dbc *Conn) terminate(ctx context.Context, insideTxn bool, now time.Time) {
+ errMsg := dbc.getErrorMessageFromContextError(ctx)
if insideTxn {
// we can't safely kill a query in a transaction, we need to kill the connection
_ = dbc.Kill(errMsg, time.Since(now))
@@ -229,7 +235,7 @@ func (dbc *Conn) FetchNext(ctx context.Context, maxrows int, wantfields bool) (*
// Check if the context is already past its deadline before
// trying to fetch the next result.
if err := ctx.Err(); err != nil {
- return nil, fmt.Errorf("%v before reading next result set", err)
+ return nil, vterrors.Errorf(vtrpcpb.Code_CANCELED, "%s before reading next result set", dbc.getErrorMessageFromContextError(ctx))
}
res, _, _, err := dbc.conn.ReadQueryResult(maxrows, wantfields)
if err != nil {
@@ -557,6 +563,11 @@ func (dbc *Conn) BaseShowTablesWithSizes() string {
return dbc.conn.BaseShowTablesWithSizes()
}
+// BaseShowInnodbTableSizes returns a query that shows innodb-internal FULLTEXT index tables and their sizes
+func (dbc *Conn) BaseShowInnodbTableSizes() string {
+ return dbc.conn.BaseShowInnodbTableSizes()
+}
+
func (dbc *Conn) ConnCheck(ctx context.Context) error {
if err := dbc.conn.ConnCheck(); err != nil {
return dbc.Reconnect(ctx)
@@ -598,3 +609,7 @@ func (dbc *Conn) applySameSetting(ctx context.Context) error {
_, err := dbc.execOnce(ctx, dbc.setting.ApplyQuery(), 1, false, false)
return err
}
+
+func (dbc *Conn) IsUnixSocket() bool {
+ return dbc.conn.IsClientUnixSocket()
+}
diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go
index 6f3c77de528..1d9104c4354 100644
--- a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go
+++ b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go
@@ -20,7 +20,6 @@ import (
"context"
"errors"
"fmt"
- "strings"
"sync/atomic"
"testing"
"time"
@@ -76,18 +75,12 @@ func TestDBConnExec(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
// Exec succeed, not asking for fields.
result, err := dbConn.Exec(ctx, sql, 1, false)
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
expectedResult.Fields = nil
- if !expectedResult.Equal(result) {
- t.Errorf("Exec: %v, want %v", expectedResult, result)
- }
+ require.True(t, expectedResult.Equal(result))
compareTimingCounts(t, "PoolTest.Exec", 1, startCounts, mysqlTimings.Counts())
@@ -100,10 +93,8 @@ func TestDBConnExec(t *testing.T) {
Query: "",
})
_, err = dbConn.Exec(ctx, sql, 1, false)
- want := "connection fail"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "connection fail")
// The client side error triggers a retry in exec.
compareTimingCounts(t, "PoolTest.Exec", 2, startCounts, mysqlTimings.Counts())
@@ -114,10 +105,8 @@ func TestDBConnExec(t *testing.T) {
// This time the initial query fails as does the reconnect attempt.
db.EnableConnFail()
_, err = dbConn.Exec(ctx, sql, 1, false)
- want = "packet read failed"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "packet read failed")
db.DisableConnFail()
compareTimingCounts(t, "PoolTest.Exec", 1, startCounts, mysqlTimings.Counts())
@@ -150,14 +139,10 @@ func TestDBConnExecLost(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
// Exec succeed, not asking for fields.
result, err := dbConn.Exec(ctx, sql, 1, false)
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
expectedResult.Fields = nil
if !expectedResult.Equal(result) {
t.Errorf("Exec: %v, want %v", expectedResult, result)
@@ -173,10 +158,8 @@ func TestDBConnExecLost(t *testing.T) {
Query: "",
})
_, err = dbConn.Exec(ctx, sql, 1, false)
- want := "Lost connection to MySQL server during query"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "Lost connection to MySQL server during query")
// Should *not* see a retry, so only increment by 1
compareTimingCounts(t, "PoolTest.Exec", 1, startCounts, mysqlTimings.Counts())
@@ -212,15 +195,11 @@ func TestDBConnDeadline(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
_, err = dbConn.Exec(ctx, sql, 1, false)
- want := "context deadline exceeded before execution started"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "(errno 3024) (sqlstate HY000): Query execution was interrupted, maximum statement execution time exceeded before execution started")
compareTimingCounts(t, "PoolTest.Exec", 0, startCounts, mysqlTimings.Counts())
@@ -230,9 +209,7 @@ func TestDBConnDeadline(t *testing.T) {
defer cancel()
result, err := dbConn.Exec(ctx, sql, 1, false)
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
expectedResult.Fields = nil
if !expectedResult.Equal(result) {
t.Errorf("Exec: %v, want %v", expectedResult, result)
@@ -244,9 +221,7 @@ func TestDBConnDeadline(t *testing.T) {
// Test with just the Background context (with no deadline)
result, err = dbConn.Exec(context.Background(), sql, 1, false)
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
expectedResult.Fields = nil
if !expectedResult.Equal(result) {
t.Errorf("Exec: %v, want %v", expectedResult, result)
@@ -266,18 +241,14 @@ func TestDBConnKill(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
query := fmt.Sprintf("kill %d", dbConn.ID())
db.AddQuery(query, &sqltypes.Result{})
// Kill failed because we are not able to connect to the database
db.EnableConnFail()
err = dbConn.Kill("test kill", 0)
- want := "errno 2013"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "errno 2013")
db.DisableConnFail()
// Kill succeed
@@ -294,10 +265,8 @@ func TestDBConnKill(t *testing.T) {
// Kill failed because "kill query_id" failed
db.AddRejectedQuery(newKillQuery, errors.New("rejected"))
err = dbConn.Kill("test kill", 0)
- want = "rejected"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "rejected")
}
func TestDBKillWithContext(t *testing.T) {
@@ -479,18 +448,17 @@ func TestDBNoPoolConnKill(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
query := fmt.Sprintf("kill %d", dbConn.ID())
db.AddQuery(query, &sqltypes.Result{})
// Kill failed because we are not able to connect to the database
db.EnableConnFail()
err = dbConn.Kill("test kill", 0)
- want := "errno 2013"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ var sqlErr *sqlerror.SQLError
+ isSqlErr := errors.As(sqlerror.NewSQLErrorFromError(err), &sqlErr)
+ require.True(t, isSqlErr)
+ require.EqualValues(t, sqlerror.CRServerLost, sqlErr.Number())
db.DisableConnFail()
// Kill succeed
@@ -507,10 +475,8 @@ func TestDBNoPoolConnKill(t *testing.T) {
// Kill failed because "kill query_id" failed
db.AddRejectedQuery(newKillQuery, errors.New("rejected"))
err = dbConn.Kill("test kill", 0)
- want = "rejected"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Exec: %v, want %s", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "rejected")
}
func TestDBConnStream(t *testing.T) {
@@ -536,9 +502,7 @@ func TestDBConnStream(t *testing.T) {
if dbConn != nil {
defer dbConn.Close()
}
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
+ require.NoError(t, err)
var result sqltypes.Result
err = dbConn.Stream(
ctx, sql, func(r *sqltypes.Result) error {
@@ -552,12 +516,8 @@ func TestDBConnStream(t *testing.T) {
return nil
}, alloc,
10, querypb.ExecuteOptions_ALL)
- if err != nil {
- t.Fatalf("should not get an error, err: %v", err)
- }
- if !expectedResult.Equal(&result) {
- t.Errorf("Exec: %v, want %v", expectedResult, &result)
- }
+ require.NoError(t, err)
+ require.True(t, expectedResult.Equal(&result))
// Stream fail
db.Close()
dbConn.Close()
@@ -569,10 +529,8 @@ func TestDBConnStream(t *testing.T) {
},
10, querypb.ExecuteOptions_ALL)
db.DisableConnFail()
- want := "no such file or directory (errno 2002)"
- if err == nil || !strings.Contains(err.Error(), want) {
- t.Errorf("Error: '%v', must contain '%s'", err, want)
- }
+ require.Error(t, err)
+ require.ErrorContains(t, err, "no such file or directory (errno 2002)")
}
// TestDBConnKillCall tests that direct Kill method calls work as expected.
diff --git a/go/vt/vttablet/tabletserver/controller.go b/go/vt/vttablet/tabletserver/controller.go
index 0336d9a73cc..cef0dd2baee 100644
--- a/go/vt/vttablet/tabletserver/controller.go
+++ b/go/vt/vttablet/tabletserver/controller.go
@@ -22,6 +22,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/mysqlctl"
+ "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vttablet/queryservice"
"vitess.io/vitess/go/vt/vttablet/tabletserver/rules"
@@ -93,6 +94,31 @@ type Controller interface {
// CheckThrottler
CheckThrottler(ctx context.Context, appName string, flags *throttle.CheckFlags) *throttle.CheckResult
GetThrottlerStatus(ctx context.Context) *throttle.ThrottlerStatus
+
+ // RedoPreparedTransactions recreates the transactions with stored prepared transaction log.
+ RedoPreparedTransactions()
+
+ // SetTwoPCAllowed sets whether TwoPC is allowed or not. It also takes the reason of why it is being set.
+ // The reason should be an enum value defined in the tabletserver.
+ SetTwoPCAllowed(int, bool)
+
+ // UnresolvedTransactions returns all unresolved transactions list
+ UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error)
+
+ // ReadTransaction returns all unresolved transactions list
+ ReadTransaction(ctx context.Context, target *querypb.Target, dtid string) (*querypb.TransactionMetadata, error)
+
+ // GetTransactionInfo returns data about a single transaction
+ GetTransactionInfo(ctx context.Context, target *querypb.Target, dtid string) (*tabletmanagerdata.GetTransactionInfoResponse, error)
+
+ // ConcludeTransaction deletes the distributed transaction metadata
+ ConcludeTransaction(ctx context.Context, target *querypb.Target, dtid string) error
+
+ // RollbackPrepared rolls back the prepared transaction and removes the transaction log.
+ RollbackPrepared(ctx context.Context, target *querypb.Target, dtid string, originalID int64) error
+
+ // WaitForPreparedTwoPCTransactions waits for all prepared transactions to be resolved.
+ WaitForPreparedTwoPCTransactions(ctx context.Context) error
}
// Ensure TabletServer satisfies Controller interface.
diff --git a/go/vt/vttablet/tabletserver/debug_2pc.go b/go/vt/vttablet/tabletserver/debug_2pc.go
index a0de20104db..5db72be0fba 100644
--- a/go/vt/vttablet/tabletserver/debug_2pc.go
+++ b/go/vt/vttablet/tabletserver/debug_2pc.go
@@ -19,12 +19,16 @@ limitations under the License.
package tabletserver
import (
+ "context"
"os"
"path"
"strconv"
"time"
+ "vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/log"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/vterrors"
)
const DebugTwoPc = true
@@ -46,3 +50,24 @@ func commitPreparedDelayForTest(tsv *TabletServer) {
time.Sleep(time.Duration(delVal) * time.Second)
}
}
+
+// checkTestFailure is used to simulate failures in 2PC flow for testing when DebugTwoPc is true.
+func checkTestFailure(ctx context.Context, shard string) error {
+ if shard != "80-" {
+ return nil
+ }
+ callerID := callerid.EffectiveCallerIDFromContext(ctx)
+ if callerID == nil {
+ return nil
+ }
+ switch callerID.Principal {
+ case "CP_80-_R":
+ // retryable error.
+ return vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "commit prepared: retryable error")
+ case "CP_80-_NR":
+ // non retryable error.
+ return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "commit prepared: non retryable error")
+ default:
+ return nil
+ }
+}
diff --git a/go/vt/vttablet/tabletserver/debugenv.go b/go/vt/vttablet/tabletserver/debugenv.go
index 54cf09db7d6..6f1ea854ea9 100644
--- a/go/vt/vttablet/tabletserver/debugenv.go
+++ b/go/vt/vttablet/tabletserver/debugenv.go
@@ -23,9 +23,10 @@ import (
"html"
"net/http"
"strconv"
- "text/template"
"time"
+ "github.com/google/safehtml/template"
+
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/vt/log"
)
@@ -70,90 +71,131 @@ func debugEnvHandler(tsv *TabletServer, w http.ResponseWriter, r *http.Request)
return
}
+ switch r.Method {
+ case http.MethodPost:
+ handlePost(tsv, w, r)
+ case http.MethodGet:
+ handleGet(tsv, w, r)
+ default:
+ http.Error(w, "Method not allowed", http.StatusMethodNotAllowed)
+ }
+}
+
+func handlePost(tsv *TabletServer, w http.ResponseWriter, r *http.Request) {
+ varname := r.FormValue("varname")
+ value := r.FormValue("value")
+
var msg string
- if r.Method == "POST" {
- varname := r.FormValue("varname")
- value := r.FormValue("value")
- setIntVal := func(f func(int)) {
- ival, err := strconv.Atoi(value)
- if err != nil {
- msg = fmt.Sprintf("Failed setting value for %v: %v", varname, err)
- return
- }
- f(ival)
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ if varname == "" || value == "" {
+ http.Error(w, "Missing varname or value", http.StatusBadRequest)
+ return
+ }
+
+ setIntVal := func(f func(int)) error {
+ ival, err := strconv.Atoi(value)
+ if err != nil {
+ return fmt.Errorf("invalid int value for %v: %v", varname, err)
}
- setIntValCtx := func(f func(context.Context, int) error) {
- ival, err := strconv.Atoi(value)
- if err == nil {
- err = f(r.Context(), ival)
- if err == nil {
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
- return
- }
- }
- msg = fmt.Sprintf("Failed setting value for %v: %v", varname, err)
+ f(ival)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ return nil
+ }
+
+ setIntValCtx := func(f func(context.Context, int) error) error {
+ ival, err := strconv.Atoi(value)
+ if err == nil {
+ err = f(r.Context(), ival)
}
- setInt64Val := func(f func(int64)) {
- ival, err := strconv.ParseInt(value, 10, 64)
- if err != nil {
- msg = fmt.Sprintf("Failed setting value for %v: %v", varname, err)
- return
- }
- f(ival)
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ if err != nil {
+ return fmt.Errorf("failed setting value for %v: %v", varname, err)
}
- setDurationVal := func(f func(time.Duration)) {
- durationVal, err := time.ParseDuration(value)
- if err != nil {
- msg = fmt.Sprintf("Failed setting value for %v: %v", varname, err)
- return
- }
- f(durationVal)
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ return nil
+ }
+
+ setInt64Val := func(f func(int64)) error {
+ ival, err := strconv.ParseInt(value, 10, 64)
+ if err != nil {
+ return fmt.Errorf("invalid int64 value for %v: %v", varname, err)
}
- setFloat64Val := func(f func(float64)) {
- fval, err := strconv.ParseFloat(value, 64)
- if err != nil {
- msg = fmt.Sprintf("Failed setting value for %v: %v", varname, err)
- return
- }
- f(fval)
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ f(ival)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ return nil
+ }
+
+ setDurationVal := func(f func(time.Duration)) error {
+ durationVal, err := time.ParseDuration(value)
+ if err != nil {
+ return fmt.Errorf("invalid duration value for %v: %v", varname, err)
}
- switch varname {
- case "PoolSize":
- setIntValCtx(tsv.SetPoolSize)
- case "StreamPoolSize":
- setIntValCtx(tsv.SetStreamPoolSize)
- case "TxPoolSize":
- setIntValCtx(tsv.SetTxPoolSize)
- case "MaxResultSize":
- setIntVal(tsv.SetMaxResultSize)
- case "WarnResultSize":
- setIntVal(tsv.SetWarnResultSize)
- case "RowStreamerMaxInnoDBTrxHistLen":
- setInt64Val(func(val int64) { tsv.Config().RowStreamer.MaxInnoDBTrxHistLen = val })
- case "RowStreamerMaxMySQLReplLagSecs":
- setInt64Val(func(val int64) { tsv.Config().RowStreamer.MaxMySQLReplLagSecs = val })
- case "UnhealthyThreshold":
- setDurationVal(func(d time.Duration) { tsv.Config().Healthcheck.UnhealthyThreshold = d })
- setDurationVal(tsv.hs.SetUnhealthyThreshold)
- setDurationVal(tsv.sm.SetUnhealthyThreshold)
- case "ThrottleMetricThreshold":
- setFloat64Val(tsv.SetThrottleMetricThreshold)
- case "Consolidator":
- tsv.SetConsolidatorMode(value)
- msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ f(durationVal)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ return nil
+ }
+
+ setFloat64Val := func(f func(float64)) error {
+ fval, err := strconv.ParseFloat(value, 64)
+ if err != nil {
+ return fmt.Errorf("invalid float64 value for %v: %v", varname, err)
}
+ f(fval)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ return nil
+ }
+
+ var err error
+ switch varname {
+ case "ReadPoolSize":
+ err = setIntValCtx(tsv.SetPoolSize)
+ case "StreamPoolSize":
+ err = setIntValCtx(tsv.SetStreamPoolSize)
+ case "TransactionPoolSize":
+ err = setIntValCtx(tsv.SetTxPoolSize)
+ case "MaxResultSize":
+ err = setIntVal(tsv.SetMaxResultSize)
+ case "WarnResultSize":
+ err = setIntVal(tsv.SetWarnResultSize)
+ case "RowStreamerMaxInnoDBTrxHistLen":
+ err = setInt64Val(func(val int64) { tsv.Config().RowStreamer.MaxInnoDBTrxHistLen = val })
+ case "RowStreamerMaxMySQLReplLagSecs":
+ err = setInt64Val(func(val int64) { tsv.Config().RowStreamer.MaxMySQLReplLagSecs = val })
+ case "UnhealthyThreshold":
+ err = setDurationVal(func(d time.Duration) { tsv.Config().Healthcheck.UnhealthyThreshold = d })
+ case "ThrottleMetricThreshold":
+ err = setFloat64Val(tsv.SetThrottleMetricThreshold)
+ case "Consolidator":
+ tsv.SetConsolidatorMode(value)
+ msg = fmt.Sprintf("Setting %v to: %v", varname, value)
+ }
+
+ if err != nil {
+ http.Error(w, err.Error(), http.StatusBadRequest)
+ return
}
+ vars := getVars(tsv)
+ sendResponse(r, w, vars, msg)
+}
+
+func handleGet(tsv *TabletServer, w http.ResponseWriter, r *http.Request) {
+ vars := getVars(tsv)
+ sendResponse(r, w, vars, "")
+}
+
+func sendResponse(r *http.Request, w http.ResponseWriter, vars []envValue, msg string) {
+ format := r.FormValue("format")
+ if format == "json" {
+ respondWithJSON(w, vars, msg)
+ return
+ }
+ respondWithHTML(w, vars, msg)
+}
+
+func getVars(tsv *TabletServer) []envValue {
var vars []envValue
- vars = addVar(vars, "PoolSize", tsv.PoolSize)
+ vars = addVar(vars, "ReadPoolSize", tsv.PoolSize)
vars = addVar(vars, "StreamPoolSize", tsv.StreamPoolSize)
- vars = addVar(vars, "TxPoolSize", tsv.TxPoolSize)
- vars = addVar(vars, "QueryCacheCapacity", tsv.QueryPlanCacheCap) // QueryCacheCapacity is deprecated in v21, it is replaced by QueryEnginePlanCacheCapacity
- vars = addVar(vars, "QueryEnginePlanCacheCapacity", tsv.QueryPlanCacheCap)
+ vars = addVar(vars, "TransactionPoolSize", tsv.TxPoolSize)
vars = addVar(vars, "MaxResultSize", tsv.MaxResultSize)
vars = addVar(vars, "WarnResultSize", tsv.WarnResultSize)
vars = addVar(vars, "RowStreamerMaxInnoDBTrxHistLen", func() int64 { return tsv.Config().RowStreamer.MaxInnoDBTrxHistLen })
@@ -165,18 +207,22 @@ func debugEnvHandler(tsv *TabletServer, w http.ResponseWriter, r *http.Request)
Value: tsv.ConsolidatorMode(),
})
- format := r.FormValue("format")
- if format == "json" {
- mvars := make(map[string]string)
- for _, v := range vars {
- mvars[v.Name] = v.Value
- }
- w.Header().Set("Content-Type", "application/json")
- _ = json.NewEncoder(w).Encode(mvars)
- return
+ return vars
+}
+
+func respondWithJSON(w http.ResponseWriter, vars []envValue, msg string) {
+ mvars := make(map[string]string)
+ for _, v := range vars {
+ mvars[v.Name] = v.Value
}
+ if msg != "" {
+ mvars["ResponseMessage"] = msg
+ }
+ w.Header().Set("Content-Type", "application/json")
+ _ = json.NewEncoder(w).Encode(mvars)
+}
- // gridTable is reused from twopcz.go.
+func respondWithHTML(w http.ResponseWriter, vars []envValue, msg string) {
w.Write(gridTable)
w.Write([]byte("Internal Variables \n"))
if msg != "" {
diff --git a/go/vt/vttablet/tabletserver/dt_executor.go b/go/vt/vttablet/tabletserver/dt_executor.go
index 9ddca3247a3..1789632c2d0 100644
--- a/go/vt/vttablet/tabletserver/dt_executor.go
+++ b/go/vt/vttablet/tabletserver/dt_executor.go
@@ -23,25 +23,32 @@ import (
"vitess.io/vitess/go/trace"
"vitess.io/vitess/go/vt/log"
querypb "vitess.io/vitess/go/vt/proto/query"
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/rules"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
)
// DTExecutor is used for executing a distributed transactional request.
type DTExecutor struct {
- ctx context.Context
- logStats *tabletenv.LogStats
- te *TxEngine
+ ctx context.Context
+ logStats *tabletenv.LogStats
+ te *TxEngine
+ qe *QueryEngine
+ shardFunc func() string
}
// NewDTExecutor creates a new distributed transaction executor.
-func NewDTExecutor(ctx context.Context, te *TxEngine, logStats *tabletenv.LogStats) *DTExecutor {
+func NewDTExecutor(ctx context.Context, logStats *tabletenv.LogStats, te *TxEngine, qe *QueryEngine, shardFunc func() string) *DTExecutor {
return &DTExecutor{
- ctx: ctx,
- te: te,
- logStats: logStats,
+ ctx: ctx,
+ logStats: logStats,
+ te: te,
+ qe: qe,
+ shardFunc: shardFunc,
}
}
@@ -53,6 +60,9 @@ func (dte *DTExecutor) Prepare(transactionID int64, dtid string) error {
if !dte.te.twopcEnabled {
return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
}
+ if !dte.te.IsTwoPCAllowed() {
+ return vterrors.VT10002("two-pc is enabled, but semi-sync is not")
+ }
defer dte.te.env.Stats().QueryTimings.Record("PREPARE", time.Now())
dte.logStats.TransactionID = transactionID
@@ -62,25 +72,69 @@ func (dte *DTExecutor) Prepare(transactionID int64, dtid string) error {
}
// If no queries were executed, we just rollback.
- if len(conn.TxProperties().Queries) == 0 {
+ queries := conn.TxProperties().GetQueries()
+ if len(queries) == 0 {
dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
return nil
}
+ // We can only prepare on a Unix socket connection.
+ // Unix socket are reliable and we can be sure that the connection is not lost with the server after prepare.
+ if !conn.IsUnixSocket() {
+ dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
+ return vterrors.VT10002("cannot prepare the transaction on a network connection")
+ }
+
// If the connection is tainted, we cannot prepare it. As there could be temporary tables involved.
if conn.IsTainted() {
dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
return vterrors.VT10002("cannot prepare the transaction on a reserved connection")
}
+ // Fail Prepare if any query rule disallows it.
+ // This could be due to ongoing cutover happening in vreplication workflow
+ // regarding OnlineDDL or MoveTables.
+ for _, query := range queries {
+ qr := dte.qe.queryRuleSources.FilterByPlan(query.Sql, 0, query.Tables...)
+ if qr != nil {
+ act, _, _, _ := qr.GetAction("", "", nil, sqlparser.MarginComments{})
+ if act != rules.QRContinue {
+ dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
+ return vterrors.VT10002("cannot prepare the transaction due to query rule")
+ }
+ }
+ }
+
err = dte.te.preparedPool.Put(conn, dtid)
if err != nil {
dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
return vterrors.Errorf(vtrpcpb.Code_RESOURCE_EXHAUSTED, "prepare failed for transaction %d: %v", transactionID, err)
}
+ // Recheck the rules. As some prepare transaction could have passed the first check.
+ // If they are put in the prepared pool, then vreplication workflow waits.
+ // This check helps reject the prepare that came later.
+ for _, query := range queries {
+ qr := dte.qe.queryRuleSources.FilterByPlan(query.Sql, 0, query.Tables...)
+ if qr != nil {
+ act, _, _, _ := qr.GetAction("", "", nil, sqlparser.MarginComments{})
+ if act != rules.QRContinue {
+ dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
+ dte.te.preparedPool.FetchForRollback(dtid)
+ return vterrors.VT10002("cannot prepare the transaction due to query rule")
+ }
+ }
+ }
+
+ // If OnlineDDL killed the connection. We should avoid the prepare for it.
+ if conn.IsClosed() {
+ dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
+ dte.te.preparedPool.FetchForRollback(dtid)
+ return vterrors.VT10002("cannot prepare the transaction on a closed connection")
+ }
+
return dte.inTransaction(func(localConn *StatefulConnection) error {
- return dte.te.twoPC.SaveRedo(dte.ctx, localConn, dtid, conn.TxProperties().Queries)
+ return dte.te.twoPC.SaveRedo(dte.ctx, localConn, dtid, queries)
})
}
@@ -107,11 +161,22 @@ func (dte *DTExecutor) CommitPrepared(dtid string) (err error) {
ctx := trace.CopySpan(context.Background(), dte.ctx)
defer func() {
if err != nil {
- dte.markFailed(ctx, dtid)
log.Warningf("failed to commit the prepared transaction '%s' with error: %v", dtid, err)
+ fail := dte.te.checkErrorAndMarkFailed(ctx, dtid, err, "TwopcCommit")
+ if fail {
+ dte.te.env.Stats().CommitPreparedFail.Add("NonRetryable", 1)
+ } else {
+ dte.te.env.Stats().CommitPreparedFail.Add("Retryable", 1)
+ }
}
dte.te.txPool.RollbackAndRelease(ctx, conn)
}()
+ if DebugTwoPc {
+ if err := checkTestFailure(dte.ctx, dte.shardFunc()); err != nil {
+ log.Errorf("failing test on commit prepared: %v", err)
+ return err
+ }
+ }
if err = dte.te.twoPC.DeleteRedo(ctx, conn, dtid); err != nil {
return err
}
@@ -122,33 +187,6 @@ func (dte *DTExecutor) CommitPrepared(dtid string) (err error) {
return nil
}
-// markFailed does the necessary work to mark a CommitPrepared
-// as failed. It marks the dtid as failed in the prepared pool,
-// increments the InternalErros counter, and also changes the
-// state of the transaction in the redo log as failed. If the
-// state change does not succeed, it just logs the event.
-// The function uses the passed in context that has no timeout
-// instead of DTExecutor's context.
-func (dte *DTExecutor) markFailed(ctx context.Context, dtid string) {
- dte.te.env.Stats().InternalErrors.Add("TwopcCommit", 1)
- dte.te.preparedPool.SetFailed(dtid)
- conn, _, _, err := dte.te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
- if err != nil {
- log.Errorf("markFailed: Begin failed for dtid %s: %v", dtid, err)
- return
- }
- defer dte.te.txPool.RollbackAndRelease(ctx, conn)
-
- if err = dte.te.twoPC.UpdateRedo(ctx, conn, dtid, RedoStateFailed); err != nil {
- log.Errorf("markFailed: UpdateRedo failed for dtid %s: %v", dtid, err)
- return
- }
-
- if _, err = dte.te.txPool.Commit(ctx, conn); err != nil {
- log.Errorf("markFailed: Commit failed for dtid %s: %v", dtid, err)
- }
-}
-
// RollbackPrepared rolls back a prepared transaction. This function handles
// the case of an incomplete prepare.
//
@@ -198,34 +236,36 @@ func (dte *DTExecutor) CreateTransaction(dtid string, participants []*querypb.Ta
// StartCommit atomically commits the transaction along with the
// decision to commit the associated 2pc transaction.
-func (dte *DTExecutor) StartCommit(transactionID int64, dtid string) error {
+func (dte *DTExecutor) StartCommit(transactionID int64, dtid string) (querypb.StartCommitState, error) {
if !dte.te.twopcEnabled {
- return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
+ return querypb.StartCommitState_Fail, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
}
defer dte.te.env.Stats().QueryTimings.Record("START_COMMIT", time.Now())
dte.logStats.TransactionID = transactionID
conn, err := dte.te.txPool.GetAndLock(transactionID, "for 2pc commit")
if err != nil {
- return err
+ return querypb.StartCommitState_Fail, err
}
defer dte.te.txPool.RollbackAndRelease(dte.ctx, conn)
// If the connection is tainted, we cannot take a commit decision on it.
if conn.IsTainted() {
dte.inTransaction(func(conn *StatefulConnection) error {
- return dte.te.twoPC.Transition(dte.ctx, conn, dtid, querypb.TransactionState_ROLLBACK)
+ return dte.te.twoPC.Transition(dte.ctx, conn, dtid, DTStateRollback)
})
// return the error, defer call above will roll back the transaction.
- return vterrors.VT10002("cannot commit the transaction on a reserved connection")
+ return querypb.StartCommitState_Fail, vterrors.VT10002("cannot commit the transaction on a reserved connection")
}
- err = dte.te.twoPC.Transition(dte.ctx, conn, dtid, querypb.TransactionState_COMMIT)
+ err = dte.te.twoPC.Transition(dte.ctx, conn, dtid, DTStateCommit)
if err != nil {
- return err
+ return querypb.StartCommitState_Fail, err
}
- _, err = dte.te.txPool.Commit(dte.ctx, conn)
- return err
+ if _, err = dte.te.txPool.Commit(dte.ctx, conn); err != nil {
+ return querypb.StartCommitState_Unknown, err
+ }
+ return querypb.StartCommitState_Success, nil
}
// SetRollback transitions the 2pc transaction to the Rollback state.
@@ -247,7 +287,7 @@ func (dte *DTExecutor) SetRollback(dtid string, transactionID int64) error {
}
return dte.inTransaction(func(conn *StatefulConnection) error {
- return dte.te.twoPC.Transition(dte.ctx, conn, dtid, querypb.TransactionState_ROLLBACK)
+ return dte.te.twoPC.Transition(dte.ctx, conn, dtid, DTStateRollback)
})
}
@@ -272,6 +312,14 @@ func (dte *DTExecutor) ReadTransaction(dtid string) (*querypb.TransactionMetadat
return dte.te.twoPC.ReadTransaction(dte.ctx, dtid)
}
+// GetTransactionInfo returns the data of the specified dtid.
+func (dte *DTExecutor) GetTransactionInfo(dtid string) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ if !dte.te.twopcEnabled {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
+ }
+ return dte.te.twoPC.GetTransactionInfo(dte.ctx, dtid)
+}
+
// ReadTwopcInflight returns info about all in-flight 2pc transactions.
func (dte *DTExecutor) ReadTwopcInflight() (distributed []*tx.DistributedTx, prepared, failed []*tx.PreparedTx, err error) {
if !dte.te.twopcEnabled {
@@ -289,7 +337,7 @@ func (dte *DTExecutor) ReadTwopcInflight() (distributed []*tx.DistributedTx, pre
}
func (dte *DTExecutor) inTransaction(f func(*StatefulConnection) error) error {
- conn, _, _, err := dte.te.txPool.Begin(dte.ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := dte.te.txPool.Begin(dte.ctx, &querypb.ExecuteOptions{}, false, 0, nil)
if err != nil {
return err
}
@@ -308,6 +356,14 @@ func (dte *DTExecutor) inTransaction(f func(*StatefulConnection) error) error {
}
// UnresolvedTransactions returns the list of unresolved distributed transactions.
-func (dte *DTExecutor) UnresolvedTransactions() ([]*querypb.TransactionMetadata, error) {
- return dte.te.twoPC.UnresolvedTransactions(dte.ctx, time.Now().Add(-dte.te.abandonAge))
+func (dte *DTExecutor) UnresolvedTransactions(requestedAge time.Duration) ([]*querypb.TransactionMetadata, error) {
+ if !dte.te.twopcEnabled {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
+ }
+ // override default time if provided in the request.
+ age := dte.te.abandonAge
+ if requestedAge > 0 {
+ age = requestedAge
+ }
+ return dte.te.twoPC.UnresolvedTransactions(dte.ctx, time.Now().Add(-age))
}
diff --git a/go/vt/vttablet/tabletserver/dt_executor_test.go b/go/vt/vttablet/tabletserver/dt_executor_test.go
index fb45ab454fc..b21667392d6 100644
--- a/go/vt/vttablet/tabletserver/dt_executor_test.go
+++ b/go/vt/vttablet/tabletserver/dt_executor_test.go
@@ -21,30 +21,34 @@ import (
"errors"
"fmt"
"reflect"
+ "regexp"
+ "strconv"
"strings"
"testing"
"time"
- "vitess.io/vitess/go/event/syslogger"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
-
+ "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"google.golang.org/protobuf/proto"
+ "vitess.io/vitess/go/event/syslogger"
"vitess.io/vitess/go/mysql/fakesqldb"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
-
querypb "vitess.io/vitess/go/vt/proto/query"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/rules"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
)
func TestTxExecutorEmptyPrepare(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
// start a transaction.
txid := newTransaction(tsv, nil)
@@ -65,9 +69,8 @@ func TestTxExecutorEmptyPrepare(t *testing.T) {
func TestExecutorPrepareFailure(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
// start a transaction
txid := newTxForPrep(ctx, tsv)
@@ -86,9 +89,8 @@ func TestExecutorPrepareFailure(t *testing.T) {
func TestTxExecutorPrepare(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid, "aa")
require.NoError(t, err)
@@ -106,9 +108,8 @@ func TestTxExecutorPrepare(t *testing.T) {
func TestDTExecutorPrepareResevedConn(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
// Reserve a connection
@@ -121,9 +122,8 @@ func TestDTExecutorPrepareResevedConn(t *testing.T) {
func TestTxExecutorPrepareNotInTx(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, _, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
err := txe.Prepare(0, "aa")
require.EqualError(t, err, "transaction 0: not found (potential transaction timeout)")
}
@@ -131,9 +131,8 @@ func TestTxExecutorPrepareNotInTx(t *testing.T) {
func TestTxExecutorPreparePoolFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid1 := newTxForPrep(ctx, tsv)
txid2 := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid1, "aa")
@@ -147,9 +146,8 @@ func TestTxExecutorPreparePoolFail(t *testing.T) {
func TestTxExecutorPrepareRedoBeginFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
db.AddRejectedQuery("begin", errors.New("begin fail"))
err := txe.Prepare(txid, "aa")
@@ -161,9 +159,8 @@ func TestTxExecutorPrepareRedoBeginFail(t *testing.T) {
func TestTxExecutorPrepareRedoFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid, "bb")
defer txe.RollbackPrepared("bb", 0)
@@ -174,9 +171,8 @@ func TestTxExecutorPrepareRedoFail(t *testing.T) {
func TestTxExecutorPrepareRedoCommitFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
db.AddRejectedQuery("commit", errors.New("commit fail"))
err := txe.Prepare(txid, "aa")
@@ -185,12 +181,63 @@ func TestTxExecutorPrepareRedoCommitFail(t *testing.T) {
require.Contains(t, err.Error(), "commit fail")
}
+func TestExecutorPrepareRuleFailure(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+
+ alterRule := rules.NewQueryRule("disable update", "disable update", rules.QRBuffer)
+ alterRule.AddTableCond("test_table")
+
+ r := rules.New()
+ r.Add(alterRule)
+ txe.qe.queryRuleSources.RegisterSource("bufferQuery")
+ err := txe.qe.queryRuleSources.SetRules("bufferQuery", r)
+ require.NoError(t, err)
+
+ // start a transaction
+ txid := newTxForPrep(ctx, tsv)
+
+ // taint the connection.
+ sc, err := tsv.te.txPool.GetAndLock(txid, "adding query property")
+ require.NoError(t, err)
+ sc.txProps.Queries = append(sc.txProps.Queries, tx.Query{
+ Sql: "update test_table set col = 5",
+ Tables: []string{"test_table"},
+ })
+ sc.Unlock()
+
+ // try 2pc commit of Metadata Manager.
+ err = txe.Prepare(txid, "aa")
+ require.EqualError(t, err, "VT10002: atomic distributed transaction not allowed: cannot prepare the transaction due to query rule")
+}
+
+func TestExecutorPrepareConnFailure(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+
+ // start a transaction
+ txid := newTxForPrep(ctx, tsv)
+
+ // taint the connection.
+ sc, err := tsv.te.txPool.GetAndLock(txid, "adding query property")
+ require.NoError(t, err)
+ sc.Unlock()
+ sc.dbConn.Close()
+
+ // try 2pc commit of Metadata Manager.
+ err = txe.Prepare(txid, "aa")
+ require.EqualError(t, err, "VT10002: atomic distributed transaction not allowed: cannot prepare the transaction on a closed connection")
+}
+
func TestTxExecutorCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid, "aa")
require.NoError(t, err)
@@ -204,9 +251,8 @@ func TestTxExecutorCommit(t *testing.T) {
func TestTxExecutorCommitRedoFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
tl := syslogger.NewTestLogger()
defer tl.Close()
@@ -220,8 +266,8 @@ func TestTxExecutorCommitRedoFail(t *testing.T) {
require.NoError(t, err)
// fail commit prepare as the delete redo query is in rejected query.
- db.AddRejectedQuery("delete from _vt.redo_state where dtid = 'bb'", errors.New("delete redo log fail"))
- db.AddQuery("update _vt.redo_state set state = 0 where dtid = 'bb'", sqltypes.MakeTestResult(nil))
+ db.AddRejectedQuery("delete from _vt.redo_state where dtid = _binary'bb'", errors.New("delete redo log fail"))
+ db.AddQuery("update _vt.redo_state set state = 0 where dtid = _binary'bb'", sqltypes.MakeTestResult(nil))
err = txe.CommitPrepared("bb")
require.ErrorContains(t, err, "delete redo log fail")
@@ -237,9 +283,8 @@ func TestTxExecutorCommitRedoFail(t *testing.T) {
func TestTxExecutorCommitRedoCommitFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid, "aa")
require.NoError(t, err)
@@ -253,9 +298,8 @@ func TestTxExecutorCommitRedoCommitFail(t *testing.T) {
func TestTxExecutorRollbackBeginFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
err := txe.Prepare(txid, "aa")
require.NoError(t, err)
@@ -268,9 +312,8 @@ func TestTxExecutorRollbackBeginFail(t *testing.T) {
func TestTxExecutorRollbackRedoFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
txid := newTxForPrep(ctx, tsv)
// Allow all additions to redo logs to succeed
db.AddQueryPattern("insert into _vt\\.redo_state.*", &sqltypes.Result{})
@@ -284,12 +327,11 @@ func TestTxExecutorRollbackRedoFail(t *testing.T) {
func TestExecutorCreateTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, _, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
- db.AddQueryPattern(fmt.Sprintf("insert into _vt\\.dt_state\\(dtid, state, time_created\\) values \\('aa', %d,.*", int(querypb.TransactionState_PREPARE)), &sqltypes.Result{})
- db.AddQueryPattern("insert into _vt\\.dt_participant\\(dtid, id, keyspace, shard\\) values \\('aa', 1,.*", &sqltypes.Result{})
+ db.AddQueryPattern(fmt.Sprintf("insert into _vt\\.dt_state\\(dtid, state, time_created\\) values \\(_binary'aa', %d,.*", int(querypb.TransactionState_PREPARE)), &sqltypes.Result{})
+ db.AddQueryPattern("insert into _vt\\.dt_participant\\(dtid, id, keyspace, shard\\) values \\(_binary'aa', 1,.*", &sqltypes.Result{})
err := txe.CreateTransaction("aa", []*querypb.Target{{
Keyspace: "t1",
Shard: "0",
@@ -300,29 +342,29 @@ func TestExecutorCreateTransaction(t *testing.T) {
func TestExecutorStartCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
- commitTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = 'aa' and state = %d", int(querypb.TransactionState_COMMIT), int(querypb.TransactionState_PREPARE))
+ commitTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = _binary'aa' and state = %d", int(querypb.TransactionState_COMMIT), int(querypb.TransactionState_PREPARE))
db.AddQuery(commitTransition, &sqltypes.Result{RowsAffected: 1})
txid := newTxForPrep(ctx, tsv)
- err := txe.StartCommit(txid, "aa")
+ state, err := txe.StartCommit(txid, "aa")
require.NoError(t, err)
+ assert.Equal(t, querypb.StartCommitState_Success, state)
db.AddQuery(commitTransition, &sqltypes.Result{})
txid = newTxForPrep(ctx, tsv)
- err = txe.StartCommit(txid, "aa")
+ state, err = txe.StartCommit(txid, "aa")
require.Error(t, err)
require.Contains(t, err.Error(), "could not transition to COMMIT: aa")
+ assert.Equal(t, querypb.StartCommitState_Fail, state)
}
func TestExecutorStartCommitFailure(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
// start a transaction
txid := newTxForPrep(ctx, tsv)
@@ -334,22 +376,22 @@ func TestExecutorStartCommitFailure(t *testing.T) {
sc.Unlock()
// add rollback state update expectation
- rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = 'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
+ rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = _binary'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
db.AddQuery(rollbackTransition, sqltypes.MakeTestResult(nil))
// try 2pc commit of Metadata Manager.
- err = txe.StartCommit(txid, "aa")
+ state, err := txe.StartCommit(txid, "aa")
require.EqualError(t, err, "VT10002: atomic distributed transaction not allowed: cannot commit the transaction on a reserved connection")
+ assert.Equal(t, querypb.StartCommitState_Fail, state)
}
func TestExecutorSetRollback(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
- rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = 'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
+ rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = _binary'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
db.AddQuery(rollbackTransition, &sqltypes.Result{RowsAffected: 1})
txid := newTxForPrep(ctx, tsv)
err := txe.SetRollback("aa", txid)
@@ -362,15 +404,70 @@ func TestExecutorSetRollback(t *testing.T) {
require.Contains(t, err.Error(), "could not transition to ROLLBACK: aa")
}
+// TestExecutorUnresolvedTransactions tests with what timestamp value the query is executed to fetch unresolved transactions.
+func TestExecutorUnresolvedTransactions(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ txe, _, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+
+ pattern := `(?i)select\s+t\.dtid,\s+t\.state,\s+t\.time_created,\s+p\.keyspace,\s+p\.shard\s+from\s+_vt\.dt_state\s+t\s+join\s+_vt\.dt_participant\s+p\s+on\s+t\.dtid\s+=\s+p\.dtid\s+where\s+time_created\s+<\s+(\d+)\s+order\s+by\s+t\.state\s+desc,\s+t\.dtid`
+ re := regexp.MustCompile(pattern)
+
+ var executedQuery string
+ db.AddQueryPatternWithCallback(pattern, &sqltypes.Result{}, func(query string) {
+ executedQuery = query
+ })
+
+ tcases := []struct {
+ abandonAge time.Duration
+ expected time.Time
+ }{
+ {abandonAge: 0, expected: time.Now().Add(-txe.te.abandonAge)},
+ {abandonAge: 100 * time.Second, expected: time.Now().Add(-100 * time.Second)},
+ }
+
+ for _, tcase := range tcases {
+ t.Run(fmt.Sprintf("abandonAge=%v", tcase.abandonAge), func(t *testing.T) {
+ _, err := txe.UnresolvedTransactions(tcase.abandonAge)
+ require.NoError(t, err)
+ require.NotEmpty(t, executedQuery)
+
+ // extract the time value
+ matches := re.FindStringSubmatch(executedQuery)
+ require.Len(t, matches, 2)
+ timeCreated := convertNanoStringToTime(t, matches[1])
+
+ // diff should be in microseconds, so we allow 10ms difference
+ require.WithinDuration(t, timeCreated, tcase.expected, 10*time.Millisecond)
+ })
+ }
+
+}
+
+func convertNanoStringToTime(t *testing.T, unixNanoStr string) time.Time {
+ t.Helper()
+
+ // Convert the string to an integer (int64)
+ unixNano, err := strconv.ParseInt(unixNanoStr, 10, 64)
+ require.NoError(t, err)
+
+ // Convert nanoseconds to seconds and nanoseconds
+ seconds := unixNano / int64(time.Second)
+ nanos := unixNano % int64(time.Second)
+
+ // Create a time.Time object
+ return time.Unix(seconds, nanos)
+}
+
func TestExecutorConcludeTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, _, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
- db.AddQuery("delete from _vt.dt_state where dtid = 'aa'", &sqltypes.Result{})
- db.AddQuery("delete from _vt.dt_participant where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.dt_state where dtid = _binary'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.dt_participant where dtid = _binary'aa'", &sqltypes.Result{})
err := txe.ConcludeTransaction("aa")
require.NoError(t, err)
}
@@ -378,11 +475,10 @@ func TestExecutorConcludeTransaction(t *testing.T) {
func TestExecutorReadTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, _, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", &sqltypes.Result{})
got, err := txe.ReadTransaction("aa")
require.NoError(t, err)
want := &querypb.TransactionMetadata{}
@@ -402,8 +498,8 @@ func TestExecutorReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
- db.AddQuery("select keyspace, shard from _vt.dt_participant where dtid = 'aa'", &sqltypes.Result{
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
+ db.AddQuery("select keyspace, shard from _vt.dt_participant where dtid = _binary'aa'", &sqltypes.Result{
Fields: []*querypb.Field{
{Type: sqltypes.VarChar},
{Type: sqltypes.VarChar},
@@ -448,7 +544,7 @@ func TestExecutorReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
want.State = querypb.TransactionState_COMMIT
got, err = txe.ReadTransaction("aa")
require.NoError(t, err)
@@ -468,7 +564,7 @@ func TestExecutorReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
want.State = querypb.TransactionState_ROLLBACK
got, err = txe.ReadTransaction("aa")
require.NoError(t, err)
@@ -480,9 +576,8 @@ func TestExecutorReadTransaction(t *testing.T) {
func TestExecutorReadAllTransactions(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- txe, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ txe, _, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
db.AddQuery(txe.te.twoPC.readAllTransactions, &sqltypes.Result{
Fields: []*querypb.Field{
@@ -578,7 +673,10 @@ func TestNoTwopc(t *testing.T) {
fun: func() error { return txe.CreateTransaction("aa", nil) },
}, {
desc: "StartCommit",
- fun: func() error { return txe.StartCommit(1, "aa") },
+ fun: func() error {
+ _, err := txe.StartCommit(1, "aa")
+ return err
+ },
}, {
desc: "SetRollback",
fun: func() error { return txe.SetRollback("aa", 1) },
@@ -597,29 +695,47 @@ func TestNoTwopc(t *testing.T) {
_, _, _, err := txe.ReadTwopcInflight()
return err
},
+ }, {
+ desc: "UnresolvedTransactions",
+ fun: func() error {
+ _, err := txe.UnresolvedTransactions(0 /* requestedAge */)
+ return err
+ },
}}
want := "2pc is not enabled"
for _, tc := range testcases {
- err := tc.fun()
- require.EqualError(t, err, want)
+ t.Run(tc.desc, func(t *testing.T) {
+ err := tc.fun()
+ require.EqualError(t, err, want)
+ })
}
}
-func newTestTxExecutor(t *testing.T, ctx context.Context) (txe *DTExecutor, tsv *TabletServer, db *fakesqldb.DB) {
+func newTestTxExecutor(t *testing.T, ctx context.Context) (txe *DTExecutor, tsv *TabletServer, db *fakesqldb.DB, closer func()) {
db = setUpQueryExecutorTest(t)
logStats := tabletenv.NewLogStats(ctx, "TestTxExecutor")
tsv = newTestTabletServer(ctx, smallTxPool, db)
- db.AddQueryPattern("insert into _vt\\.redo_state\\(dtid, state, time_created\\) values \\('aa', 1,.*", &sqltypes.Result{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest")
+ se := schema.NewEngine(env)
+ qe := NewQueryEngine(env, se)
+ db.AddQueryPattern("insert into _vt\\.redo_state\\(dtid, state, time_created\\) values \\(_binary'aa', 1,.*", &sqltypes.Result{})
db.AddQueryPattern("insert into _vt\\.redo_statement.*", &sqltypes.Result{})
- db.AddQuery("delete from _vt.redo_state where dtid = 'aa'", &sqltypes.Result{})
- db.AddQuery("delete from _vt.redo_statement where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.redo_state where dtid = _binary'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.redo_statement where dtid = _binary'aa'", &sqltypes.Result{})
db.AddQuery("update test_table set `name` = 2 where pk = 1 limit 10001", &sqltypes.Result{})
+ db.AddRejectedQuery("bogus", sqlerror.NewSQLError(sqlerror.ERUnknownError, sqlerror.SSUnknownSQLState, "bogus query"))
return &DTExecutor{
- ctx: ctx,
- logStats: logStats,
- te: tsv.te,
- }, tsv, db
+ ctx: ctx,
+ logStats: logStats,
+ te: tsv.te,
+ qe: qe,
+ }, tsv, db, func() {
+ db.Close()
+ tsv.StopService()
+ }
}
// newShortAgeExecutor is same as newTestTxExecutor, but shorter transaction abandon age.
@@ -627,10 +743,10 @@ func newShortAgeExecutor(t *testing.T, ctx context.Context) (txe *DTExecutor, ts
db = setUpQueryExecutorTest(t)
logStats := tabletenv.NewLogStats(ctx, "TestTxExecutor")
tsv = newTestTabletServer(ctx, smallTxPool|shortTwopcAge, db)
- db.AddQueryPattern("insert into _vt\\.redo_state\\(dtid, state, time_created\\) values \\('aa', 1,.*", &sqltypes.Result{})
+ db.AddQueryPattern("insert into _vt\\.redo_state\\(dtid, state, time_created\\) values \\(_binary'aa', 1,.*", &sqltypes.Result{})
db.AddQueryPattern("insert into _vt\\.redo_statement.*", &sqltypes.Result{})
- db.AddQuery("delete from _vt.redo_state where dtid = 'aa'", &sqltypes.Result{})
- db.AddQuery("delete from _vt.redo_statement where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.redo_state where dtid = _binary'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.redo_statement where dtid = _binary'aa'", &sqltypes.Result{})
db.AddQuery("update test_table set `name` = 2 where pk = 1 limit 10001", &sqltypes.Result{})
return &DTExecutor{
ctx: ctx,
diff --git a/go/vt/vttablet/tabletserver/health_streamer.go b/go/vt/vttablet/tabletserver/health_streamer.go
index cfc5ea5e974..eaeba6315e3 100644
--- a/go/vt/vttablet/tabletserver/health_streamer.go
+++ b/go/vt/vttablet/tabletserver/health_streamer.go
@@ -66,11 +66,18 @@ type healthStreamer struct {
degradedThreshold time.Duration
unhealthyThreshold atomic.Int64
- mu sync.Mutex
- ctx context.Context
- cancel context.CancelFunc
- clients map[chan *querypb.StreamHealthResponse]struct{}
- state *querypb.StreamHealthResponse
+ // cancelMu is a mutex used to protect the cancel variable
+ // and for ensuring we don't call setup functions in parallel.
+ cancelMu sync.Mutex
+ ctx context.Context
+ cancel context.CancelFunc
+
+ // fieldsMu is used to protect access to the fields below.
+ // We require two separate mutexes, so that we don't have to acquire the same mutex
+ // in Close and reload that can lead to a deadlock described in https://github.com/vitessio/vitess/issues/17229#issuecomment-2476136610.
+ fieldsMu sync.Mutex
+ clients map[chan *querypb.StreamHealthResponse]struct{}
+ state *querypb.StreamHealthResponse
// isServingPrimary stores if this tablet is currently the serving primary or not.
isServingPrimary bool
@@ -110,8 +117,8 @@ func (hs *healthStreamer) InitDBConfig(target *querypb.Target) {
}
func (hs *healthStreamer) Open() {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.cancelMu.Lock()
+ defer hs.cancelMu.Unlock()
if hs.cancel != nil {
return
@@ -120,8 +127,8 @@ func (hs *healthStreamer) Open() {
}
func (hs *healthStreamer) Close() {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.cancelMu.Lock()
+ defer hs.cancelMu.Unlock()
if hs.cancel != nil {
hs.se.UnregisterNotifier("healthStreamer")
@@ -158,13 +165,16 @@ func (hs *healthStreamer) Stream(ctx context.Context, callback func(*querypb.Str
}
func (hs *healthStreamer) register() (chan *querypb.StreamHealthResponse, context.Context) {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.cancelMu.Lock()
+ defer hs.cancelMu.Unlock()
if hs.cancel == nil {
return nil, nil
}
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
+
ch := make(chan *querypb.StreamHealthResponse, streamHealthBufferSize)
hs.clients[ch] = struct{}{}
@@ -174,15 +184,15 @@ func (hs *healthStreamer) register() (chan *querypb.StreamHealthResponse, contex
}
func (hs *healthStreamer) unregister(ch chan *querypb.StreamHealthResponse) {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
delete(hs.clients, ch)
}
func (hs *healthStreamer) ChangeState(tabletType topodatapb.TabletType, ptsTimestamp time.Time, lag time.Duration, err error, serving bool) {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
hs.state.Target.TabletType = tabletType
if tabletType == topodatapb.TabletType_PRIMARY {
@@ -236,8 +246,8 @@ func (hs *healthStreamer) broadCastToClients(shr *querypb.StreamHealthResponse)
}
func (hs *healthStreamer) AppendDetails(details []*kv) []*kv {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
if hs.state.Target.TabletType == topodatapb.TabletType_PRIMARY {
return details
}
@@ -282,9 +292,11 @@ func (hs *healthStreamer) SetUnhealthyThreshold(v time.Duration) {
// MakePrimary tells the healthstreamer that the current tablet is now the primary,
// so it can read and write to the MySQL instance for schema-tracking.
func (hs *healthStreamer) MakePrimary(serving bool) {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
hs.isServingPrimary = serving
+ // We let go of the lock here because we don't want to hold the lock when calling RegisterNotifier.
+ // If we keep holding the lock, there is a potential deadlock that can happen.
+ hs.fieldsMu.Unlock()
// We register for notifications from the schema Engine only when schema tracking is enabled,
// and we are going to a serving primary state.
if serving && hs.signalWhenSchemaChange {
@@ -298,15 +310,15 @@ func (hs *healthStreamer) MakePrimary(serving bool) {
// MakeNonPrimary tells the healthstreamer that the current tablet is now not a primary.
func (hs *healthStreamer) MakeNonPrimary() {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
hs.isServingPrimary = false
}
// reload reloads the schema from the underlying mysql for the tables that we get the alert on.
func (hs *healthStreamer) reload(created, altered, dropped []*schema.Table, udfsChanged bool) error {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
// Schema Reload to happen only on primary when it is serving.
// We can be in a state when the primary is not serving after we have run DemotePrimary. In that case,
// we don't want to run any queries in MySQL, so we shouldn't reload anything in the healthStreamer.
@@ -349,8 +361,8 @@ func (hs *healthStreamer) reload(created, altered, dropped []*schema.Table, udfs
// sendUnresolvedTransactionSignal sends broadcast message about unresolved transactions.
func (hs *healthStreamer) sendUnresolvedTransactionSignal() {
- hs.mu.Lock()
- defer hs.mu.Unlock()
+ hs.fieldsMu.Lock()
+ defer hs.fieldsMu.Unlock()
// send signal only when primary is serving.
if !hs.isServingPrimary {
return
diff --git a/go/vt/vttablet/tabletserver/health_streamer_test.go b/go/vt/vttablet/tabletserver/health_streamer_test.go
index 95517880339..9561518eed6 100644
--- a/go/vt/vttablet/tabletserver/health_streamer_test.go
+++ b/go/vt/vttablet/tabletserver/health_streamer_test.go
@@ -39,6 +39,8 @@ import (
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
+const baseInnoDBTableSizesPattern = `(?s).*SELECT.*its\.space = it\.space.*SUM\(its\.file_size\).*`
+
func TestHealthStreamerClosed(t *testing.T) {
cfg := newConfig(nil)
env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "ReplTrackerTest")
@@ -276,6 +278,13 @@ func TestReloadSchema(t *testing.T) {
// Update the query pattern for the query that schema.Engine uses to get the tables so that it runs a reload again.
// If we don't change the t.create_time to a value greater than before, then the schema engine doesn't reload the database.
+ db.AddQueryPattern(baseInnoDBTableSizesPattern,
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "it.name | normal_tables_sum_file_size | normal_tables_sum_allocated_size",
+ "varchar|int64|int64",
+ ),
+ ))
db.AddQueryPattern("SELECT .* information_schema.innodb_tablespaces .*",
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
@@ -285,7 +294,6 @@ func TestReloadSchema(t *testing.T) {
"product|BASE TABLE|1684735967||114688|114688",
"users|BASE TABLE|1684735967||114688|114688",
))
-
db.AddQuery(mysql.BaseShowTables,
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
@@ -351,6 +359,14 @@ func TestReloadView(t *testing.T) {
db.AddQuery("commit", &sqltypes.Result{})
db.AddQuery("rollback", &sqltypes.Result{})
// Add the query pattern for the query that schema.Engine uses to get the tables.
+ // InnoDBTableSizes query
+ db.AddQueryPattern(baseInnoDBTableSizesPattern,
+ sqltypes.MakeTestResult(
+ sqltypes.MakeTestFields(
+ "it.name | normal_tables_sum_file_size | normal_tables_sum_allocated_size",
+ "varchar|int64|int64",
+ ),
+ ))
db.AddQueryPattern("SELECT .* information_schema.innodb_tablespaces .*",
sqltypes.MakeTestResult(
sqltypes.MakeTestFields(
@@ -417,7 +433,7 @@ func TestReloadView(t *testing.T) {
expGetViewDefinitionsQuery string
viewDefinitionsOutput *sqltypes.Result
- expClearQuery string
+ expClearQuery []string
expInsertQuery []string
expViewsChanged []string
}{
@@ -433,7 +449,10 @@ func TestReloadView(t *testing.T) {
expViewsChanged: []string{"view_a", "view_b"},
expGetViewDefinitionsQuery: "select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('view_a', 'view_b')",
expCreateStmtQuery: []string{"show create table view_a", "show create table view_b"},
- expClearQuery: "delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('view_a', 'view_b')",
+ expClearQuery: []string{
+ "delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('view_a', 'view_b')",
+ "delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('view_b', 'view_a')",
+ },
expInsertQuery: []string{
"insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'view_a', 'create_view_a', 'def_a')",
"insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'view_b', 'create_view_b', 'def_b')",
@@ -450,7 +469,7 @@ func TestReloadView(t *testing.T) {
expViewsChanged: []string{"view_b"},
expGetViewDefinitionsQuery: "select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('view_b')",
expCreateStmtQuery: []string{"show create table view_b"},
- expClearQuery: "delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('view_b')",
+ expClearQuery: []string{"delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('view_b')"},
expInsertQuery: []string{
"insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'view_b', 'create_view_mod_b', 'def_mod_b')",
},
@@ -467,7 +486,14 @@ func TestReloadView(t *testing.T) {
expViewsChanged: []string{"view_a", "view_b", "view_c"},
expGetViewDefinitionsQuery: "select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('view_b', 'view_c', 'view_a')",
expCreateStmtQuery: []string{"show create table view_a", "show create table view_c"},
- expClearQuery: "delete from _vt.views where table_schema = database() and table_name in ('view_b', 'view_c', 'view_a')",
+ expClearQuery: []string{
+ "delete from _vt.views where table_schema = database() and table_name in ('view_a', 'view_b', 'view_c')",
+ "delete from _vt.views where table_schema = database() and table_name in ('view_a', 'view_c', 'view_b')",
+ "delete from _vt.views where table_schema = database() and table_name in ('view_b', 'view_a', 'view_c')",
+ "delete from _vt.views where table_schema = database() and table_name in ('view_b', 'view_c', 'view_a')",
+ "delete from _vt.views where table_schema = database() and table_name in ('view_c', 'view_a', 'view_b')",
+ "delete from _vt.views where table_schema = database() and table_name in ('view_c', 'view_b', 'view_a')",
+ },
expInsertQuery: []string{
"insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'view_a', 'create_view_mod_a', 'def_mod_a')",
"insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'view_c', 'create_view_c', 'def_c')",
@@ -486,8 +512,9 @@ func TestReloadView(t *testing.T) {
for idx := range tcases[0].expInsertQuery {
db.AddQuery(tcases[0].expInsertQuery[idx], &sqltypes.Result{})
}
- db.AddQuery(tcases[0].expClearQuery, &sqltypes.Result{})
-
+ for _, query := range tcases[0].expClearQuery {
+ db.AddQuery(query, &sqltypes.Result{})
+ }
var tcCount atomic.Int32
ch := make(chan struct{})
@@ -519,7 +546,9 @@ func TestReloadView(t *testing.T) {
for i := range tcases[idx].expInsertQuery {
db.AddQuery(tcases[idx].expInsertQuery[i], &sqltypes.Result{})
}
- db.AddQuery(tcases[idx].expClearQuery, &sqltypes.Result{})
+ for _, query := range tcases[idx].expClearQuery {
+ db.AddQuery(query, &sqltypes.Result{})
+ }
db.AddQueryPattern("SELECT .* information_schema.innodb_tablespaces .*", tcases[idx].showTablesWithSizesOutput)
db.AddQueryPattern(".*SELECT table_name, view_definition.*views.*", tcases[idx].detectViewChangeOutput)
case <-time.After(10 * time.Second):
@@ -543,3 +572,44 @@ func testStream(hs *healthStreamer) (<-chan *querypb.StreamHealthResponse, conte
func testBlpFunc() (int64, int32) {
return 1, 2
}
+
+// TestDeadlockBwCloseAndReload tests the deadlock observed between Close and Reload
+// functions. More details can be found in the issue https://github.com/vitessio/vitess/issues/17229#issuecomment-2476136610.
+func TestDeadlockBwCloseAndReload(t *testing.T) {
+ cfg := newConfig(nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TestNotServingPrimary")
+ alias := &topodatapb.TabletAlias{
+ Cell: "cell",
+ Uid: 1,
+ }
+ se := schema.NewEngineForTests()
+ // Create a new health streamer and set it to a serving primary state
+ hs := newHealthStreamer(env, alias, se)
+ hs.signalWhenSchemaChange = true
+ hs.Open()
+ hs.MakePrimary(true)
+ defer hs.Close()
+
+ wg := sync.WaitGroup{}
+ wg.Add(2)
+ // Try running Close & MakePrimary and reload in parallel multiple times.
+ // This reproduces the deadlock quite readily.
+ go func() {
+ defer wg.Done()
+ for i := 0; i < 100; i++ {
+ hs.Close()
+ hs.Open()
+ hs.MakePrimary(true)
+ }
+ }()
+
+ go func() {
+ defer wg.Done()
+ for i := 0; i < 100; i++ {
+ se.BroadcastForTesting(nil, nil, nil, true)
+ }
+ }()
+
+ // Wait for wait group to finish.
+ wg.Wait()
+}
diff --git a/go/vt/vttablet/tabletserver/messager/engine.go b/go/vt/vttablet/tabletserver/messager/engine.go
index 612619f7ccc..9d8b09e819a 100644
--- a/go/vt/vttablet/tabletserver/messager/engine.go
+++ b/go/vt/vttablet/tabletserver/messager/engine.go
@@ -45,15 +45,23 @@ type TabletService interface {
// VStreamer defines the functions of VStreamer
// that the messager needs.
type VStreamer interface {
- Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error
+ Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter,
+ throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error
StreamResults(ctx context.Context, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error
}
// Engine is the engine for handling messages.
type Engine struct {
- mu sync.Mutex
- isOpen bool
- managers map[string]*messageManager
+ // mu is a mutex used to protect the isOpen variable
+ // and for ensuring we don't call setup functions in parallel.
+ mu sync.Mutex
+ isOpen bool
+
+ // managersMu is a mutex used to protect the managers field.
+ // We require two separate mutexes, so that we don't have to acquire the same mutex
+ // in Close and schemaChanged which can lead to a deadlock described in https://github.com/vitessio/vitess/issues/17229.
+ managersMu sync.Mutex
+ managers map[string]*messageManager
tsv TabletService
se *schema.Engine
@@ -75,15 +83,12 @@ func NewEngine(tsv TabletService, se *schema.Engine, vs VStreamer) *Engine {
// Open starts the Engine service.
func (me *Engine) Open() {
me.mu.Lock()
+ defer me.mu.Unlock()
if me.isOpen {
- me.mu.Unlock()
return
}
me.isOpen = true
- me.mu.Unlock()
log.Info("Messager: opening")
- // Unlock before invoking RegisterNotifier because it
- // obtains the same lock.
me.se.RegisterNotifier("messages", me.schemaChanged, true)
}
@@ -101,6 +106,8 @@ func (me *Engine) Close() {
log.Infof("messager Engine - unregistering notifiers")
me.se.UnregisterNotifier("messages")
log.Infof("messager Engine - closing all managers")
+ me.managersMu.Lock()
+ defer me.managersMu.Unlock()
for _, mm := range me.managers {
mm.Close()
}
@@ -109,8 +116,8 @@ func (me *Engine) Close() {
}
func (me *Engine) GetGenerator(name string) (QueryGenerator, error) {
- me.mu.Lock()
- defer me.mu.Unlock()
+ me.managersMu.Lock()
+ defer me.managersMu.Unlock()
mm := me.managers[name]
if mm == nil {
return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "message table %s not found in schema", name)
@@ -131,6 +138,8 @@ func (me *Engine) Subscribe(ctx context.Context, name string, send func(*sqltype
if !me.isOpen {
return nil, vterrors.Errorf(vtrpcpb.Code_UNAVAILABLE, "messager engine is closed, probably because this is not a primary any more")
}
+ me.managersMu.Lock()
+ defer me.managersMu.Unlock()
mm := me.managers[name]
if mm == nil {
return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "message table %s not found", name)
@@ -139,8 +148,8 @@ func (me *Engine) Subscribe(ctx context.Context, name string, send func(*sqltype
}
func (me *Engine) schemaChanged(tables map[string]*schema.Table, created, altered, dropped []*schema.Table, _ bool) {
- me.mu.Lock()
- defer me.mu.Unlock()
+ me.managersMu.Lock()
+ defer me.managersMu.Unlock()
for _, table := range append(dropped, altered...) {
name := table.Name.String()
mm := me.managers[name]
diff --git a/go/vt/vttablet/tabletserver/messager/engine_test.go b/go/vt/vttablet/tabletserver/messager/engine_test.go
index 30e849ac73b..124e6a9a380 100644
--- a/go/vt/vttablet/tabletserver/messager/engine_test.go
+++ b/go/vt/vttablet/tabletserver/messager/engine_test.go
@@ -19,6 +19,7 @@ package messager
import (
"context"
"reflect"
+ "sync"
"testing"
"vitess.io/vitess/go/sqltypes"
@@ -156,7 +157,7 @@ func newTestEngine() *Engine {
tsv := &fakeTabletServer{
Env: tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "MessagerTest"),
}
- se := schema.NewEngine(tsv)
+ se := schema.NewEngineForTests()
te := NewEngine(tsv, se, newFakeVStreamer())
te.Open()
return te
@@ -169,3 +170,33 @@ func newEngineReceiver() (f func(qr *sqltypes.Result) error, ch chan *sqltypes.R
return nil
}, ch
}
+
+// TestDeadlockBwCloseAndSchemaChange tests the deadlock observed between Close and schemaChanged
+// functions. More details can be found in the issue https://github.com/vitessio/vitess/issues/17229.
+func TestDeadlockBwCloseAndSchemaChange(t *testing.T) {
+ engine := newTestEngine()
+ defer engine.Close()
+ se := engine.se
+
+ wg := sync.WaitGroup{}
+ wg.Add(2)
+ // Try running Close and schemaChanged in parallel multiple times.
+ // This reproduces the deadlock quite readily.
+ go func() {
+ defer wg.Done()
+ for i := 0; i < 100; i++ {
+ engine.Close()
+ engine.Open()
+ }
+ }()
+
+ go func() {
+ defer wg.Done()
+ for i := 0; i < 100; i++ {
+ se.BroadcastForTesting(nil, nil, nil, true)
+ }
+ }()
+
+ // Wait for wait group to finish.
+ wg.Wait()
+}
diff --git a/go/vt/vttablet/tabletserver/messager/message_manager.go b/go/vt/vttablet/tabletserver/messager/message_manager.go
index 2f4f8605870..7a217fb63b7 100644
--- a/go/vt/vttablet/tabletserver/messager/message_manager.go
+++ b/go/vt/vttablet/tabletserver/messager/message_manager.go
@@ -236,6 +236,9 @@ type messageManager struct {
ackQuery *sqlparser.ParsedQuery
postponeQuery *sqlparser.ParsedQuery
purgeQuery *sqlparser.ParsedQuery
+
+ // idType is the type of the id column in the message table.
+ idType sqltypes.Type
}
// newMessageManager creates a new message manager.
@@ -259,6 +262,7 @@ func newMessageManager(tsv TabletService, vs VStreamer, table *schema.Table, pos
purgeTicks: timer.NewTimer(table.MessageInfo.PollInterval),
postponeSema: postponeSema,
messagesPending: true,
+ idType: table.MessageInfo.IDType,
}
mm.cond.L = &mm.mu
@@ -742,7 +746,7 @@ func (mm *messageManager) runOneVStream(ctx context.Context) error {
}
}
return nil
- })
+ }, nil)
return err
}
@@ -856,7 +860,7 @@ func (mm *messageManager) GenerateAckQuery(ids []string) (string, map[string]*qu
}
for _, id := range ids {
idbvs.Values = append(idbvs.Values, &querypb.Value{
- Type: querypb.Type_VARBINARY,
+ Type: mm.idType,
Value: []byte(id),
})
}
@@ -874,7 +878,7 @@ func (mm *messageManager) GeneratePostponeQuery(ids []string) (string, map[strin
}
for _, id := range ids {
idbvs.Values = append(idbvs.Values, &querypb.Value{
- Type: querypb.Type_VARBINARY,
+ Type: mm.idType,
Value: []byte(id),
})
}
diff --git a/go/vt/vttablet/tabletserver/messager/message_manager_test.go b/go/vt/vttablet/tabletserver/messager/message_manager_test.go
index fdf39556e5c..5e1c21f773f 100644
--- a/go/vt/vttablet/tabletserver/messager/message_manager_test.go
+++ b/go/vt/vttablet/tabletserver/messager/message_manager_test.go
@@ -74,6 +74,7 @@ func newMMTable() *schema.Table {
BatchSize: 1,
CacheSize: 10,
PollInterval: 1 * time.Second,
+ IDType: sqltypes.VarBinary,
},
}
}
@@ -91,6 +92,7 @@ func newMMTableWithBackoff() *schema.Table {
BatchSize: 1,
CacheSize: 10,
PollInterval: 1 * time.Second,
+ IDType: sqltypes.VarBinary,
},
}
}
@@ -889,7 +891,8 @@ func (fv *fakeVStreamer) setPollerResponse(pr []*binlogdatapb.VStreamResultsResp
fv.pollerResponse = pr
}
-func (fv *fakeVStreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error {
+func (fv *fakeVStreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error {
fv.streamInvocations.Add(1)
for {
fv.mu.Lock()
diff --git a/go/vt/vttablet/tabletserver/planbuilder/builder.go b/go/vt/vttablet/tabletserver/planbuilder/builder.go
index 94f5fc1caa2..6df89f7caf8 100644
--- a/go/vt/vttablet/tabletserver/planbuilder/builder.go
+++ b/go/vt/vttablet/tabletserver/planbuilder/builder.go
@@ -33,7 +33,7 @@ func analyzeSelect(env *vtenv.Environment, sel *sqlparser.Select, tables map[str
PlanID: PlanSelect,
FullQuery: GenerateLimitQuery(sel),
}
- plan.Table, plan.AllTables = lookupTables(sel.From, tables)
+ plan.Table = lookupTables(sel.From, tables)
if sel.Where != nil {
comp, ok := sel.Where.Expr.(*sqlparser.ComparisonExpr)
@@ -72,7 +72,7 @@ func analyzeUpdate(upd *sqlparser.Update, tables map[string]*schema.Table) (plan
plan = &Plan{
PlanID: PlanUpdate,
}
- plan.Table, plan.AllTables = lookupTables(upd.TableExprs, tables)
+ plan.Table = lookupTables(upd.TableExprs, tables)
// Store the WHERE clause as string for the hot row protection (txserializer).
if upd.Where != nil {
@@ -102,7 +102,7 @@ func analyzeDelete(del *sqlparser.Delete, tables map[string]*schema.Table) (plan
plan = &Plan{
PlanID: PlanDelete,
}
- plan.Table, plan.AllTables = lookupTables(del.TableExprs, tables)
+ plan.Table = lookupTables(del.TableExprs, tables)
if del.Where != nil {
buf := sqlparser.NewTrackedBuffer(nil)
@@ -127,11 +127,7 @@ func analyzeInsert(ins *sqlparser.Insert, tables map[string]*schema.Table) (plan
FullQuery: GenerateFullQuery(ins),
}
- tableName, err := ins.Table.TableName()
- if err != nil {
- return nil, err
- }
- plan.Table = tables[sqlparser.GetTableName(tableName).String()]
+ plan.Table = lookupTables(sqlparser.TableExprs{ins.Table}, tables)
return plan, nil
}
@@ -188,16 +184,26 @@ func analyzeSet(set *sqlparser.Set) (plan *Plan) {
}
}
-func lookupTables(tableExprs sqlparser.TableExprs, tables map[string]*schema.Table) (singleTable *schema.Table, allTables []*schema.Table) {
+func lookupTables(tableExprs sqlparser.TableExprs, tables map[string]*schema.Table) (singleTable *schema.Table) {
for _, tableExpr := range tableExprs {
if t := lookupSingleTable(tableExpr, tables); t != nil {
- allTables = append(allTables, t)
+ if singleTable != nil {
+ return nil
+ }
+ singleTable = t
}
}
- if len(allTables) == 1 {
- singleTable = allTables[0]
+ return singleTable
+}
+
+func lookupAllTables(stmt sqlparser.Statement, tables map[string]*schema.Table) (allTables []*schema.Table) {
+ tablesUsed := sqlparser.ExtractAllTables(stmt)
+ for _, tbl := range tablesUsed {
+ if t := tables[tbl]; t != nil {
+ allTables = append(allTables, t)
+ }
}
- return singleTable, allTables
+ return allTables
}
func lookupSingleTable(tableExpr sqlparser.TableExpr, tables map[string]*schema.Table) *schema.Table {
@@ -229,12 +235,14 @@ func analyzeFlush(stmt *sqlparser.Flush, tables map[string]*schema.Table) (*Plan
for _, tbl := range stmt.TableNames {
if schemaTbl, ok := tables[tbl.Name.String()]; ok {
- plan.AllTables = append(plan.AllTables, schemaTbl)
+ if plan.Table != nil {
+ // If there are multiple tables, we empty out the table field.
+ plan.Table = nil
+ break
+ }
+ plan.Table = schemaTbl
}
}
- if len(plan.AllTables) == 1 {
- plan.Table = plan.AllTables[0]
- }
if stmt.WithLock {
plan.NeedsReservedConn = true
diff --git a/go/vt/vttablet/tabletserver/planbuilder/permission.go b/go/vt/vttablet/tabletserver/planbuilder/permission.go
index dbc6cfccdad..1949d6ce739 100644
--- a/go/vt/vttablet/tabletserver/planbuilder/permission.go
+++ b/go/vt/vttablet/tabletserver/planbuilder/permission.go
@@ -45,17 +45,17 @@ func BuildPermissions(stmt sqlparser.Statement) []Permission {
case *sqlparser.Union:
permissions = buildSubqueryPermissions(node, tableacl.READER, permissions)
case *sqlparser.Insert:
- permissions = buildTableExprPermissions(node.Table, tableacl.WRITER, permissions)
+ permissions = buildTableExprPermissions(node.Table, tableacl.WRITER, nil, permissions)
permissions = buildSubqueryPermissions(node, tableacl.READER, permissions)
case *sqlparser.Update:
- permissions = buildTableExprsPermissions(node.TableExprs, tableacl.WRITER, permissions)
+ permissions = buildTableExprsPermissions(node.TableExprs, tableacl.WRITER, nil, permissions)
permissions = buildSubqueryPermissions(node, tableacl.READER, permissions)
case *sqlparser.Delete:
- permissions = buildTableExprsPermissions(node.TableExprs, tableacl.WRITER, permissions)
+ permissions = buildTableExprsPermissions(node.TableExprs, tableacl.WRITER, nil, permissions)
permissions = buildSubqueryPermissions(node, tableacl.READER, permissions)
case sqlparser.DDLStatement:
for _, t := range node.AffectedTables() {
- permissions = buildTableNamePermissions(t, tableacl.ADMIN, permissions)
+ permissions = buildTableNamePermissions(t, tableacl.ADMIN, nil, permissions)
}
case
*sqlparser.AlterMigration,
@@ -66,10 +66,10 @@ func BuildPermissions(stmt sqlparser.Statement) []Permission {
permissions = []Permission{} // TODO(shlomi) what are the correct permissions here? Table is unknown
case *sqlparser.Flush:
for _, t := range node.TableNames {
- permissions = buildTableNamePermissions(t, tableacl.ADMIN, permissions)
+ permissions = buildTableNamePermissions(t, tableacl.ADMIN, nil, permissions)
}
case *sqlparser.Analyze:
- permissions = buildTableNamePermissions(node.Table, tableacl.WRITER, permissions)
+ permissions = buildTableNamePermissions(node.Table, tableacl.WRITER, nil, permissions)
case *sqlparser.OtherAdmin, *sqlparser.CallProc, *sqlparser.Begin, *sqlparser.Commit, *sqlparser.Rollback,
*sqlparser.Load, *sqlparser.Savepoint, *sqlparser.Release, *sqlparser.SRollback, *sqlparser.Set, *sqlparser.Show, sqlparser.Explain,
*sqlparser.UnlockTables:
@@ -81,43 +81,92 @@ func BuildPermissions(stmt sqlparser.Statement) []Permission {
}
func buildSubqueryPermissions(stmt sqlparser.Statement, role tableacl.Role, permissions []Permission) []Permission {
- _ = sqlparser.Walk(func(node sqlparser.SQLNode) (bool, error) {
- if sel, ok := node.(*sqlparser.Select); ok {
- permissions = buildTableExprsPermissions(sel.From, role, permissions)
+ var cteScopes [][]sqlparser.IdentifierCS
+ sqlparser.Rewrite(stmt, func(cursor *sqlparser.Cursor) bool {
+ switch node := cursor.Node().(type) {
+ case *sqlparser.Select:
+ if node.With != nil {
+ cteScopes = append(cteScopes, gatherCTEs(node.With))
+ }
+ var ctes []sqlparser.IdentifierCS
+ for _, cteScope := range cteScopes {
+ ctes = append(ctes, cteScope...)
+ }
+ permissions = buildTableExprsPermissions(node.From, role, ctes, permissions)
+ case *sqlparser.Delete:
+ if node.With != nil {
+ cteScopes = append(cteScopes, gatherCTEs(node.With))
+ }
+ case *sqlparser.Update:
+ if node.With != nil {
+ cteScopes = append(cteScopes, gatherCTEs(node.With))
+ }
+ case *sqlparser.Union:
+ if node.With != nil {
+ cteScopes = append(cteScopes, gatherCTEs(node.With))
+ }
}
- return true, nil
- }, stmt)
+ return true
+ }, func(cursor *sqlparser.Cursor) bool {
+ // When we encounter a With expression coming up, we should remove
+ // the last value from the cte scopes to ensure we none of the outer
+ // elements of the query see this table name.
+ _, isWith := cursor.Node().(*sqlparser.With)
+ if isWith {
+ cteScopes = cteScopes[:len(cteScopes)-1]
+ }
+ return true
+ })
return permissions
}
-func buildTableExprsPermissions(node []sqlparser.TableExpr, role tableacl.Role, permissions []Permission) []Permission {
+// gatherCTEs gathers the CTEs from the WITH clause.
+func gatherCTEs(with *sqlparser.With) []sqlparser.IdentifierCS {
+ var ctes []sqlparser.IdentifierCS
+ for _, cte := range with.CTEs {
+ ctes = append(ctes, cte.ID)
+ }
+ return ctes
+}
+
+func buildTableExprsPermissions(node []sqlparser.TableExpr, role tableacl.Role, ctes []sqlparser.IdentifierCS, permissions []Permission) []Permission {
for _, node := range node {
- permissions = buildTableExprPermissions(node, role, permissions)
+ permissions = buildTableExprPermissions(node, role, ctes, permissions)
}
return permissions
}
-func buildTableExprPermissions(node sqlparser.TableExpr, role tableacl.Role, permissions []Permission) []Permission {
+func buildTableExprPermissions(node sqlparser.TableExpr, role tableacl.Role, ctes []sqlparser.IdentifierCS, permissions []Permission) []Permission {
switch node := node.(type) {
case *sqlparser.AliasedTableExpr:
// An AliasedTableExpr can also be a derived table, but we should skip them here
// because the buildSubQueryPermissions walker will catch them and extract
// the corresponding table names.
if tblName, ok := node.Expr.(sqlparser.TableName); ok {
- permissions = buildTableNamePermissions(tblName, role, permissions)
+ permissions = buildTableNamePermissions(tblName, role, ctes, permissions)
}
case *sqlparser.ParenTableExpr:
- permissions = buildTableExprsPermissions(node.Exprs, role, permissions)
+ permissions = buildTableExprsPermissions(node.Exprs, role, ctes, permissions)
case *sqlparser.JoinTableExpr:
- permissions = buildTableExprPermissions(node.LeftExpr, role, permissions)
- permissions = buildTableExprPermissions(node.RightExpr, role, permissions)
+ permissions = buildTableExprPermissions(node.LeftExpr, role, ctes, permissions)
+ permissions = buildTableExprPermissions(node.RightExpr, role, ctes, permissions)
}
return permissions
}
-func buildTableNamePermissions(node sqlparser.TableName, role tableacl.Role, permissions []Permission) []Permission {
+func buildTableNamePermissions(node sqlparser.TableName, role tableacl.Role, ctes []sqlparser.IdentifierCS, permissions []Permission) []Permission {
+ tableName := node.Name.String()
+ // Check whether this table is a cte or not.
+ // If the table name is qualified, then it cannot be a cte.
+ if node.Qualifier.IsEmpty() {
+ for _, cte := range ctes {
+ if cte.String() == tableName {
+ return permissions
+ }
+ }
+ }
permissions = append(permissions, Permission{
- TableName: node.Name.String(),
+ TableName: tableName,
Role: role,
})
return permissions
diff --git a/go/vt/vttablet/tabletserver/planbuilder/permission_test.go b/go/vt/vttablet/tabletserver/planbuilder/permission_test.go
index 0ece6ed19b2..ab238661664 100644
--- a/go/vt/vttablet/tabletserver/planbuilder/permission_test.go
+++ b/go/vt/vttablet/tabletserver/planbuilder/permission_test.go
@@ -180,6 +180,45 @@ func TestBuildPermissions(t *testing.T) {
TableName: "seq",
Role: tableacl.WRITER,
}},
+ }, {
+ input: "with t as (select count(*) as a from user) select a from t",
+ output: []Permission{{
+ TableName: "user",
+ Role: tableacl.READER,
+ }},
+ }, {
+ input: "with d as (select id, count(*) as a from user) select d.a from music join d on music.user_id = d.id group by 1",
+ output: []Permission{{
+ TableName: "music",
+ Role: tableacl.READER,
+ }, {
+ TableName: "user",
+ Role: tableacl.READER,
+ }},
+ }, {
+ input: "WITH t1 AS ( SELECT id FROM t2 ) SELECT * FROM t1 JOIN ks.t1 AS t3",
+ output: []Permission{{
+ TableName: "t1",
+ Role: tableacl.READER,
+ }, {
+ TableName: "t2",
+ Role: tableacl.READER,
+ }},
+ }, {
+ input: "WITH RECURSIVE t1 (n) AS ( SELECT id from t2 UNION ALL SELECT n + 1 FROM t1 WHERE n < 5 ) SELECT * FROM t1 JOIN t1 AS t3",
+ output: []Permission{{
+ TableName: "t2",
+ Role: tableacl.READER,
+ }},
+ }, {
+ input: "(with t1 as (select count(*) as a from user) select a from t1) union select * from t1",
+ output: []Permission{{
+ TableName: "user",
+ Role: tableacl.READER,
+ }, {
+ TableName: "t1",
+ Role: tableacl.READER,
+ }},
}}
for _, tcase := range tcases {
diff --git a/go/vt/vttablet/tabletserver/planbuilder/plan.go b/go/vt/vttablet/tabletserver/planbuilder/plan.go
index 7b1e57c2f90..5dbabcc8763 100644
--- a/go/vt/vttablet/tabletserver/planbuilder/plan.go
+++ b/go/vt/vttablet/tabletserver/planbuilder/plan.go
@@ -157,7 +157,7 @@ type Plan struct {
PlanID PlanType
// When the query indicates a single table
Table *schema.Table
- // SELECT, UPDATE, DELETE statements may list multiple tables
+ // This indicates all the tables that are accessed in the query.
AllTables []*schema.Table
// Permissions stores the permissions for the tables accessed in the query.
@@ -205,10 +205,7 @@ func (plan *Plan) TableNames() (names []string) {
func Build(env *vtenv.Environment, statement sqlparser.Statement, tables map[string]*schema.Table, dbName string, viewsEnabled bool) (plan *Plan, err error) {
switch stmt := statement.(type) {
case *sqlparser.Union:
- plan, err = &Plan{
- PlanID: PlanSelect,
- FullQuery: GenerateLimitQuery(stmt),
- }, nil
+ plan = &Plan{PlanID: PlanSelect, FullQuery: GenerateLimitQuery(stmt)}
case *sqlparser.Select:
plan, err = analyzeSelect(env, stmt, tables)
case *sqlparser.Insert:
@@ -218,45 +215,51 @@ func Build(env *vtenv.Environment, statement sqlparser.Statement, tables map[str
case *sqlparser.Delete:
plan, err = analyzeDelete(stmt, tables)
case *sqlparser.Set:
- plan, err = analyzeSet(stmt), nil
+ plan = analyzeSet(stmt)
case sqlparser.DDLStatement:
plan, err = analyzeDDL(stmt)
case *sqlparser.AlterMigration:
- plan, err = &Plan{PlanID: PlanAlterMigration, FullStmt: stmt}, nil
+ plan = &Plan{PlanID: PlanAlterMigration, FullStmt: stmt}
case *sqlparser.RevertMigration:
- plan, err = &Plan{PlanID: PlanRevertMigration, FullStmt: stmt}, nil
+ plan = &Plan{PlanID: PlanRevertMigration, FullStmt: stmt}
case *sqlparser.ShowMigrationLogs:
- plan, err = &Plan{PlanID: PlanShowMigrationLogs, FullStmt: stmt}, nil
+ plan = &Plan{PlanID: PlanShowMigrationLogs, FullStmt: stmt}
case *sqlparser.ShowThrottledApps:
- plan, err = &Plan{PlanID: PlanShowThrottledApps, FullStmt: stmt}, nil
+ plan = &Plan{PlanID: PlanShowThrottledApps, FullStmt: stmt}
case *sqlparser.ShowThrottlerStatus:
- plan, err = &Plan{PlanID: PlanShowThrottlerStatus, FullStmt: stmt}, nil
+ plan = &Plan{PlanID: PlanShowThrottlerStatus, FullStmt: stmt}
case *sqlparser.Show:
plan, err = analyzeShow(stmt, dbName)
case *sqlparser.Analyze, sqlparser.Explain:
- plan, err = &Plan{PlanID: PlanOtherRead}, nil
+ // Analyze and Explain are treated as read-only queries.
+ // We send down a string, and get a table result back.
+ plan = &Plan{
+ PlanID: PlanSelect,
+ FullQuery: GenerateFullQuery(stmt),
+ }
case *sqlparser.OtherAdmin:
- plan, err = &Plan{PlanID: PlanOtherAdmin}, nil
+ plan = &Plan{PlanID: PlanOtherAdmin}
case *sqlparser.Savepoint:
- plan, err = &Plan{PlanID: PlanSavepoint}, nil
+ plan = &Plan{PlanID: PlanSavepoint, FullStmt: stmt}
case *sqlparser.Release:
- plan, err = &Plan{PlanID: PlanRelease}, nil
+ plan = &Plan{PlanID: PlanRelease}
case *sqlparser.SRollback:
- plan, err = &Plan{PlanID: PlanSRollback}, nil
+ plan = &Plan{PlanID: PlanSRollback, FullStmt: stmt}
case *sqlparser.Load:
- plan, err = &Plan{PlanID: PlanLoad}, nil
+ plan = &Plan{PlanID: PlanLoad}
case *sqlparser.Flush:
plan, err = analyzeFlush(stmt, tables)
case *sqlparser.UnlockTables:
- plan, err = &Plan{PlanID: PlanUnlockTables}, nil
+ plan = &Plan{PlanID: PlanUnlockTables}
case *sqlparser.CallProc:
- plan, err = &Plan{PlanID: PlanCallProc, FullQuery: GenerateFullQuery(stmt)}, nil
+ plan = &Plan{PlanID: PlanCallProc, FullQuery: GenerateFullQuery(stmt)}
default:
return nil, vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "invalid SQL")
}
if err != nil {
return nil, err
}
+ plan.AllTables = lookupAllTables(statement, tables)
plan.Permissions = BuildPermissions(statement)
return plan, nil
}
@@ -274,14 +277,14 @@ func BuildStreaming(statement sqlparser.Statement, tables map[string]*schema.Tab
if hasLockFunc(stmt) {
plan.NeedsReservedConn = true
}
- plan.Table, plan.AllTables = lookupTables(stmt.From, tables)
+ plan.Table = lookupTables(stmt.From, tables)
case *sqlparser.Show, *sqlparser.Union, *sqlparser.CallProc, sqlparser.Explain:
case *sqlparser.Analyze:
plan.PlanID = PlanOtherRead
default:
return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%s not allowed for streaming", sqlparser.ASTToStatementType(statement))
}
-
+ plan.AllTables = lookupAllTables(statement, tables)
return plan, nil
}
diff --git a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt
index cafbe43231d..f549f506ad3 100644
--- a/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt
+++ b/go/vt/vttablet/tabletserver/planbuilder/testdata/exec_cases.txt
@@ -763,14 +763,15 @@ options:PassthroughDMLs
# analyze
"analyze table a"
{
- "PlanID": "OtherRead",
+ "PlanID": "Select",
"TableName": "",
"Permissions": [
- {
- "TableName": "a",
- "Role": 1
- }
- ]
+ {
+ "TableName": "a",
+ "Role": 1
+ }
+ ],
+ "FullQuery": "analyze table a"
}
# show
@@ -783,15 +784,17 @@ options:PassthroughDMLs
# describe
"describe a"
{
- "PlanID": "OtherRead",
- "TableName": ""
+ "PlanID": "Select",
+ "TableName": "",
+ "FullQuery": "explain a"
}
# explain
"explain a"
{
- "PlanID": "OtherRead",
- "TableName": ""
+ "PlanID": "Select",
+ "TableName": "",
+ "FullQuery": "explain a"
}
# repair
diff --git a/go/vt/vttablet/tabletserver/production.go b/go/vt/vttablet/tabletserver/production.go
index 70cb8b092fa..e0d8cb4fd66 100644
--- a/go/vt/vttablet/tabletserver/production.go
+++ b/go/vt/vttablet/tabletserver/production.go
@@ -18,6 +18,10 @@ limitations under the License.
package tabletserver
+import (
+ "context"
+)
+
// This file defines debug constants that are always false.
// This file is used for building production code.
// We use go build directives to include a file that defines the constant to true
@@ -28,3 +32,7 @@ package tabletserver
const DebugTwoPc = false
func commitPreparedDelayForTest(tsv *TabletServer) {}
+
+func checkTestFailure(context.Context, string) error {
+ return nil
+}
diff --git a/go/vt/vttablet/tabletserver/query_executor.go b/go/vt/vttablet/tabletserver/query_executor.go
index 02b8dd9171a..519b60b79d6 100644
--- a/go/vt/vttablet/tabletserver/query_executor.go
+++ b/go/vt/vttablet/tabletserver/query_executor.go
@@ -29,12 +29,14 @@ import (
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/pools/smartconnpool"
-
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/trace"
"vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/callinfo"
"vitess.io/vitess/go/vt/log"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/tableacl"
@@ -45,10 +47,7 @@ import (
"vitess.io/vitess/go/vt/vttablet/tabletserver/rules"
eschema "vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
)
// QueryExecutor is used for executing a query request.
@@ -168,9 +167,15 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) {
}
defer conn.Unlock()
if qre.setting != nil {
- if err = conn.ApplySetting(qre.ctx, qre.setting); err != nil {
+ applied, err := conn.ApplySetting(qre.ctx, qre.setting)
+ if err != nil {
return nil, vterrors.Wrap(err, "failed to execute system setting on the connection")
}
+ // If we have applied the settings on the connection, then we should record the query detail.
+ // This is required for redoing the transaction in case of a failure.
+ if applied {
+ conn.TxProperties().RecordQueryDetail(qre.setting.ApplyQuery(), nil)
+ }
}
return qre.txConnExec(conn)
}
@@ -192,8 +197,10 @@ func (qre *QueryExecutor) Execute() (reply *sqltypes.Result, err error) {
return qr, nil
case p.PlanOtherRead, p.PlanOtherAdmin, p.PlanFlush, p.PlanSavepoint, p.PlanRelease, p.PlanSRollback:
return qre.execOther()
- case p.PlanInsert, p.PlanUpdate, p.PlanDelete, p.PlanInsertMessage, p.PlanDDL, p.PlanLoad:
+ case p.PlanInsert, p.PlanUpdate, p.PlanDelete, p.PlanInsertMessage, p.PlanLoad:
return qre.execAutocommit(qre.txConnExec)
+ case p.PlanDDL:
+ return qre.execDDL(nil)
case p.PlanUpdateLimit, p.PlanDeleteLimit:
return qre.execAsTransaction(qre.txConnExec)
case p.PlanCallProc:
@@ -234,7 +241,7 @@ func (qre *QueryExecutor) execAutocommit(f func(conn *StatefulConnection) (*sqlt
return nil, errTxThrottled
}
- conn, _, _, err := qre.tsv.te.txPool.Begin(qre.ctx, qre.options, false, 0, nil, qre.setting)
+ conn, _, _, err := qre.tsv.te.txPool.Begin(qre.ctx, qre.options, false, 0, qre.setting)
if err != nil {
return nil, err
@@ -248,7 +255,7 @@ func (qre *QueryExecutor) execAsTransaction(f func(conn *StatefulConnection) (*s
if qre.tsv.txThrottler.Throttle(qre.tsv.getPriorityFromOptions(qre.options), qre.options.GetWorkloadName()) {
return nil, errTxThrottled
}
- conn, beginSQL, _, err := qre.tsv.te.txPool.Begin(qre.ctx, qre.options, false, 0, nil, qre.setting)
+ conn, beginSQL, _, err := qre.tsv.te.txPool.Begin(qre.ctx, qre.options, false, 0, qre.setting)
if err != nil {
return nil, err
}
@@ -277,8 +284,10 @@ func (qre *QueryExecutor) execAsTransaction(f func(conn *StatefulConnection) (*s
func (qre *QueryExecutor) txConnExec(conn *StatefulConnection) (*sqltypes.Result, error) {
switch qre.plan.PlanID {
- case p.PlanInsert, p.PlanUpdate, p.PlanDelete, p.PlanSet:
+ case p.PlanInsert, p.PlanUpdate, p.PlanDelete:
return qre.txFetch(conn, true)
+ case p.PlanSet:
+ return qre.txFetch(conn, false)
case p.PlanInsertMessage:
qre.bindVars["#time_now"] = sqltypes.Int64BindVariable(time.Now().UnixNano())
return qre.txFetch(conn, true)
@@ -286,8 +295,12 @@ func (qre *QueryExecutor) txConnExec(conn *StatefulConnection) (*sqltypes.Result
return qre.execDMLLimit(conn)
case p.PlanOtherRead, p.PlanOtherAdmin, p.PlanFlush, p.PlanUnlockTables:
return qre.execStatefulConn(conn, qre.query, true)
- case p.PlanSavepoint, p.PlanRelease, p.PlanSRollback:
- return qre.execStatefulConn(conn, qre.query, true)
+ case p.PlanSavepoint:
+ return qre.execSavepointQuery(conn, qre.query, qre.plan.FullStmt)
+ case p.PlanSRollback:
+ return qre.execRollbackToSavepoint(conn, qre.query, qre.plan.FullStmt)
+ case p.PlanRelease:
+ return qre.execTxQuery(conn, qre.query, false)
case p.PlanSelect, p.PlanSelectImpossible, p.PlanShow, p.PlanSelectLockFunc:
maxrows := qre.getSelectLimit()
qre.bindVars["#maxLimit"] = sqltypes.Int64BindVariable(maxrows + 1)
@@ -377,7 +390,7 @@ func (qre *QueryExecutor) Stream(callback StreamCallback) error {
}
defer txConn.Unlock()
if qre.setting != nil {
- if err = txConn.ApplySetting(qre.ctx, qre.setting); err != nil {
+ if _, err = txConn.ApplySetting(qre.ctx, qre.setting); err != nil {
return vterrors.Wrap(err, "failed to execute system setting on the connection")
}
}
@@ -538,7 +551,7 @@ func (qre *QueryExecutor) checkAccess(authorized *tableacl.ACLResult, tableName
return nil
}
-func (qre *QueryExecutor) execDDL(conn *StatefulConnection) (*sqltypes.Result, error) {
+func (qre *QueryExecutor) execDDL(conn *StatefulConnection) (result *sqltypes.Result, err error) {
// Let's see if this is a normal DDL statement or an Online DDL statement.
// An Online DDL statement is identified by /*vt+ .. */ comment with expected directives, like uuid etc.
if onlineDDL, err := schema.OnlineDDLFromCommentedStatement(qre.plan.FullStmt); err == nil {
@@ -549,6 +562,21 @@ func (qre *QueryExecutor) execDDL(conn *StatefulConnection) (*sqltypes.Result, e
}
}
+ if conn == nil {
+ conn, err = qre.tsv.te.txPool.createConn(qre.ctx, qre.options, qre.setting)
+ if err != nil {
+ return nil, err
+ }
+ defer conn.Release(tx.ConnRelease)
+ }
+
+ // A DDL statement should commit the current transaction in the VTGate.
+ // The change was made in PR: https://github.com/vitessio/vitess/pull/14110 in v18.
+ // DDL statement received by vttablet will be outside of a transaction.
+ if conn.txProps != nil {
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "DDL statement executed inside a transaction")
+ }
+
isTemporaryTable := false
if ddlStmt, ok := qre.plan.FullStmt.(sqlparser.DDLStatement); ok {
isTemporaryTable = ddlStmt.IsTemporary()
@@ -580,19 +608,7 @@ func (qre *QueryExecutor) execDDL(conn *StatefulConnection) (*sqltypes.Result, e
return nil, err
}
}
- result, err := qre.execStatefulConn(conn, sql, true)
- if err != nil {
- return nil, err
- }
- // Only perform this operation when the connection has transaction open.
- // TODO: This actually does not retain the old transaction. We should see how to provide correct behaviour to client.
- if conn.txProps != nil {
- err = qre.BeginAgain(qre.ctx, conn)
- if err != nil {
- return nil, err
- }
- }
- return result, nil
+ return qre.execStatefulConn(conn, sql, true)
}
func (qre *QueryExecutor) execLoad(conn *StatefulConnection) (*sqltypes.Result, error) {
@@ -603,20 +619,6 @@ func (qre *QueryExecutor) execLoad(conn *StatefulConnection) (*sqltypes.Result,
return result, nil
}
-// BeginAgain commits the existing transaction and begins a new one
-func (*QueryExecutor) BeginAgain(ctx context.Context, dc *StatefulConnection) error {
- if dc.IsClosed() || dc.TxProperties().Autocommit {
- return nil
- }
- if _, err := dc.Exec(ctx, "commit", 1, false); err != nil {
- return err
- }
- if _, err := dc.Exec(ctx, "begin", 1, false); err != nil {
- return err
- }
- return nil
-}
-
func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) {
env := evalengine.NewExpressionEnv(qre.ctx, qre.bindVars, evalengine.NewEmptyVCursor(qre.tsv.Environment(), time.Local))
result, err := env.Evaluate(qre.plan.NextCount)
@@ -670,7 +672,6 @@ func (qre *QueryExecutor) execNextval() (*sqltypes.Result, error) {
newLast += cache
}
query = fmt.Sprintf("update %s set next_id = %d where id = 0", sqlparser.String(tableName), newLast)
- conn.TxProperties().RecordQuery(query)
_, err = qre.execStatefulConn(conn, query, false)
if err != nil {
return nil, err
@@ -801,14 +802,53 @@ func (qre *QueryExecutor) txFetch(conn *StatefulConnection, record bool) (*sqlty
if err != nil {
return nil, err
}
+ return qre.execTxQuery(conn, sql, record)
+}
+
+// execTxQuery executes the query provided and record in Tx Property if record is true.
+func (qre *QueryExecutor) execTxQuery(conn *StatefulConnection, sql string, record bool) (*sqltypes.Result, error) {
qr, err := qre.execStatefulConn(conn, sql, true)
if err != nil {
return nil, err
}
// Only record successful queries.
if record {
- conn.TxProperties().RecordQuery(sql)
+ conn.TxProperties().RecordQueryDetail(sql, qre.plan.TableNames())
+ }
+ return qr, nil
+}
+
+// execTxQuery executes the query provided and record in Tx Property if record is true.
+func (qre *QueryExecutor) execSavepointQuery(conn *StatefulConnection, sql string, ast sqlparser.Statement) (*sqltypes.Result, error) {
+ qr, err := qre.execStatefulConn(conn, sql, true)
+ if err != nil {
+ return nil, err
+ }
+
+ // Only record successful queries.
+ sp, ok := ast.(*sqlparser.Savepoint)
+ if !ok {
+ return nil, vterrors.VT13001("expected to get a savepoint statement")
+ }
+ conn.TxProperties().RecordSavePointDetail(sp.Name.String())
+
+ return qr, nil
+}
+
+// execTxQuery executes the query provided and record in Tx Property if record is true.
+func (qre *QueryExecutor) execRollbackToSavepoint(conn *StatefulConnection, sql string, ast sqlparser.Statement) (*sqltypes.Result, error) {
+ qr, err := qre.execStatefulConn(conn, sql, true)
+ if err != nil {
+ return nil, err
+ }
+
+ // Only record successful queries.
+ sp, ok := ast.(*sqlparser.SRollback)
+ if !ok {
+ return nil, vterrors.VT13001("expected to get a rollback statement")
}
+
+ _ = conn.TxProperties().RollbackToSavepoint(sp.Name.String())
return qr, nil
}
@@ -951,6 +991,8 @@ func (qre *QueryExecutor) execAlterMigration() (*sqltypes.Result, error) {
return qre.tsv.onlineDDLExecutor.ForceCutOverMigration(qre.ctx, alterMigration.UUID)
case sqlparser.ForceCutOverAllMigrationType:
return qre.tsv.onlineDDLExecutor.ForceCutOverPendingMigrations(qre.ctx)
+ case sqlparser.SetCutOverThresholdMigrationType:
+ return qre.tsv.onlineDDLExecutor.SetMigrationCutOverThreshold(qre.ctx, alterMigration.UUID, alterMigration.Threshold)
}
return nil, vterrors.New(vtrpcpb.Code_UNIMPLEMENTED, "ALTER VITESS_MIGRATION not implemented")
}
diff --git a/go/vt/vttablet/tabletserver/query_executor_test.go b/go/vt/vttablet/tabletserver/query_executor_test.go
index 33a0a680799..5820b809c8e 100644
--- a/go/vt/vttablet/tabletserver/query_executor_test.go
+++ b/go/vt/vttablet/tabletserver/query_executor_test.go
@@ -87,7 +87,8 @@ func TestQueryExecutorPlans(t *testing.T) {
// If empty, then we should expect the same as logWant.
inTxWant string
// errorWant is the error we expect to get, if any, and should be nil if no error should be returned
- errorWant error
+ errorWant string
+ onlyInTxErr bool
// TxThrottler allows the test case to override the transaction throttler
txThrottler txthrottler.TxThrottler
}{{
@@ -196,9 +197,11 @@ func TestQueryExecutorPlans(t *testing.T) {
query: "alter table test_table add column zipcode int",
result: dmlResult,
}},
- resultWant: dmlResult,
- planWant: "DDL",
- logWant: "alter table test_table add column zipcode int",
+ resultWant: dmlResult,
+ planWant: "DDL",
+ logWant: "alter table test_table add column zipcode int",
+ onlyInTxErr: true,
+ errorWant: "DDL statement executed inside a transaction",
}, {
input: "savepoint a",
dbResponses: []dbResponse{{
@@ -215,20 +218,24 @@ func TestQueryExecutorPlans(t *testing.T) {
query: "alter table `user` add key a (id)",
result: emptyResult,
}},
- resultWant: emptyResult,
- planWant: "DDL",
- logWant: "alter table `user` add key a (id)",
- inTxWant: "alter table `user` add key a (id)",
+ resultWant: emptyResult,
+ planWant: "DDL",
+ logWant: "alter table `user` add key a (id)",
+ inTxWant: "alter table `user` add key a (id)",
+ onlyInTxErr: true,
+ errorWant: "DDL statement executed inside a transaction",
}, {
input: "create index a on user(id1 + id2)",
dbResponses: []dbResponse{{
query: "create index a on user(id1 + id2)",
result: emptyResult,
}},
- resultWant: emptyResult,
- planWant: "DDL",
- logWant: "create index a on user(id1 + id2)",
- inTxWant: "create index a on user(id1 + id2)",
+ resultWant: emptyResult,
+ planWant: "DDL",
+ logWant: "create index a on user(id1 + id2)",
+ inTxWant: "create index a on user(id1 + id2)",
+ onlyInTxErr: true,
+ errorWant: "DDL statement executed inside a transaction",
}, {
input: "ROLLBACK work to SAVEPOINT a",
dbResponses: []dbResponse{{
@@ -282,7 +289,7 @@ func TestQueryExecutorPlans(t *testing.T) {
query: "update test_table set a = 1 limit 10001",
result: dmlResult,
}},
- errorWant: errTxThrottled,
+ errorWant: "Transaction throttled",
txThrottler: &mockTxThrottler{true},
}, {
input: "update test_table set a=1",
@@ -291,7 +298,7 @@ func TestQueryExecutorPlans(t *testing.T) {
query: "update test_table set a = 1 limit 10001",
result: dmlResult,
}},
- errorWant: errTxThrottled,
+ errorWant: "Transaction throttled",
txThrottler: &mockTxThrottler{true},
},
}
@@ -315,13 +322,13 @@ func TestQueryExecutorPlans(t *testing.T) {
// Test outside a transaction.
qre := newTestQueryExecutor(ctx, tsv, tcase.input, 0)
got, err := qre.Execute()
- if tcase.errorWant == nil {
+ if tcase.errorWant != "" && !tcase.onlyInTxErr {
+ assert.EqualError(t, err, tcase.errorWant)
+ } else {
require.NoError(t, err, tcase.input)
assert.Equal(t, tcase.resultWant, got, tcase.input)
assert.Equal(t, tcase.planWant, qre.logStats.PlanType, tcase.input)
assert.Equal(t, tcase.logWant, qre.logStats.RewrittenSQL(), tcase.input)
- } else {
- assert.True(t, vterrors.Equals(err, tcase.errorWant))
}
// Wait for the existing query to be processed by the cache
time.Sleep(100 * time.Millisecond)
@@ -329,25 +336,29 @@ func TestQueryExecutorPlans(t *testing.T) {
// Test inside a transaction.
target := tsv.sm.Target()
state, err := tsv.Begin(ctx, target, nil)
- if tcase.errorWant == nil {
- require.NoError(t, err)
- require.NotNil(t, state.TabletAlias, "alias should not be nil")
- assert.Equal(t, tsv.alias, state.TabletAlias, "Wrong alias returned by Begin")
- defer tsv.Commit(ctx, target, state.TransactionID)
-
- qre = newTestQueryExecutor(ctx, tsv, tcase.input, state.TransactionID)
- got, err = qre.Execute()
- require.NoError(t, err, tcase.input)
- assert.Equal(t, tcase.resultWant, got, "in tx: %v", tcase.input)
- assert.Equal(t, tcase.planWant, qre.logStats.PlanType, "in tx: %v", tcase.input)
- want := tcase.logWant
- if tcase.inTxWant != "" {
- want = tcase.inTxWant
- }
- assert.Equal(t, want, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input)
- } else {
- assert.True(t, vterrors.Equals(err, tcase.errorWant))
+ if tcase.errorWant != "" && !tcase.onlyInTxErr {
+ require.EqualError(t, err, tcase.errorWant)
+ return
}
+ require.NoError(t, err)
+ require.NotNil(t, state.TabletAlias, "alias should not be nil")
+ assert.Equal(t, tsv.alias, state.TabletAlias, "Wrong alias returned by Begin")
+ defer tsv.Commit(ctx, target, state.TransactionID)
+
+ qre = newTestQueryExecutor(ctx, tsv, tcase.input, state.TransactionID)
+ got, err = qre.Execute()
+ if tcase.onlyInTxErr {
+ require.EqualError(t, err, tcase.errorWant)
+ return
+ }
+ require.NoError(t, err, tcase.input)
+ assert.Equal(t, tcase.resultWant, got, "in tx: %v", tcase.input)
+ assert.Equal(t, tcase.planWant, qre.logStats.PlanType, "in tx: %v", tcase.input)
+ want := tcase.logWant
+ if tcase.inTxWant != "" {
+ want = tcase.inTxWant
+ }
+ assert.Equal(t, want, qre.logStats.RewrittenSQL(), "in tx: %v", tcase.input)
})
}
}
@@ -1503,20 +1514,17 @@ func newTestTabletServer(ctx context.Context, flags executorFlags, db *fakesqldb
} else {
cfg.StrictTableACL = false
}
- if flags&noTwopc > 0 {
- cfg.TwoPCEnable = false
- } else {
- cfg.TwoPCEnable = true
- }
if flags&disableOnlineDDL > 0 {
cfg.EnableOnlineDDL = false
} else {
cfg.EnableOnlineDDL = true
}
- if flags&shortTwopcAge > 0 {
- cfg.TwoPCAbandonAge = 0.5
+ if flags&noTwopc > 0 {
+ cfg.TwoPCAbandonAge = 0
+ } else if flags&shortTwopcAge > 0 {
+ cfg.TwoPCAbandonAge = 500 * time.Millisecond
} else {
- cfg.TwoPCAbandonAge = 10
+ cfg.TwoPCAbandonAge = 10 * time.Second
}
if flags&smallResultSize > 0 {
cfg.Oltp.MaxRows = 2
@@ -1532,9 +1540,6 @@ func newTestTabletServer(ctx context.Context, flags executorFlags, db *fakesqldb
tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{}, srvTopoCounts)
target := &querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
err := tsv.StartService(target, dbconfigs, nil /* mysqld */)
- if cfg.TwoPCEnable {
- tsv.TwoPCEngineWait()
- }
if err != nil {
panic(err)
}
diff --git a/go/vt/vttablet/tabletserver/querylogz.go b/go/vt/vttablet/tabletserver/querylogz.go
index 33341d1641b..09f375aa329 100644
--- a/go/vt/vttablet/tabletserver/querylogz.go
+++ b/go/vt/vttablet/tabletserver/querylogz.go
@@ -20,9 +20,10 @@ import (
"net/http"
"strconv"
"strings"
- "text/template"
"time"
+ "github.com/google/safehtml/template"
+
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logz"
diff --git a/go/vt/vttablet/tabletserver/querylogz_test.go b/go/vt/vttablet/tabletserver/querylogz_test.go
index 25f03c762c7..ee26437f330 100644
--- a/go/vt/vttablet/tabletserver/querylogz_test.go
+++ b/go/vt/vttablet/tabletserver/querylogz_test.go
@@ -37,7 +37,7 @@ func TestQuerylogzHandler(t *testing.T) {
req, _ := http.NewRequest("GET", "/querylogz?timeout=10&limit=1", nil)
logStats := tabletenv.NewLogStats(context.Background(), "Execute")
logStats.PlanType = planbuilder.PlanSelect.String()
- logStats.OriginalSQL = "select name from test_table limit 1000"
+ logStats.OriginalSQL = "select name, 'inject ' from test_table limit 1000"
logStats.RowsAffected = 1000
logStats.NumberOfQueries = 1
logStats.StartTime, _ = time.Parse("Jan 2 15:04:05", "Nov 29 13:33:09")
@@ -64,7 +64,7 @@ func TestQuerylogzHandler(t *testing.T) {
`0.001 `,
`1e-08 `,
`Select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`none `,
`1000 `,
@@ -95,7 +95,7 @@ func TestQuerylogzHandler(t *testing.T) {
`0.001 `,
`1e-08 `,
`Select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`none `,
`1000 `,
@@ -126,7 +126,7 @@ func TestQuerylogzHandler(t *testing.T) {
`0.001 `,
`1e-08 `,
`Select `,
- `select name from test_table limit 1000 `,
+ regexp.QuoteMeta(`select name, 'inject <script>alert();</script>' from test_table limit 1000 `),
`1 `,
`none `,
`1000 `,
diff --git a/go/vt/vttablet/tabletserver/schema/cached_size.go b/go/vt/vttablet/tabletserver/schema/cached_size.go
index 8ca75534a62..4db9f313644 100644
--- a/go/vt/vttablet/tabletserver/schema/cached_size.go
+++ b/go/vt/vttablet/tabletserver/schema/cached_size.go
@@ -25,7 +25,7 @@ func (cached *MessageInfo) CachedSize(alloc bool) int64 {
}
size := int64(0)
if alloc {
- size += int64(80)
+ size += int64(96)
}
// field Fields []*vitess.io/vitess/go/vt/proto/query.Field
{
diff --git a/go/vt/vttablet/tabletserver/schema/engine.go b/go/vt/vttablet/tabletserver/schema/engine.go
index 5babed271ca..fb4a9d9367a 100644
--- a/go/vt/vttablet/tabletserver/schema/engine.go
+++ b/go/vt/vttablet/tabletserver/schema/engine.go
@@ -31,6 +31,7 @@ import (
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/mysql/collations/charset"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
@@ -40,6 +41,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/dbconnpool"
"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/servenv"
@@ -85,9 +87,10 @@ type Engine struct {
historian *historian
- conns *connpool.Pool
- ticks *timer.Timer
- reloadTimeout time.Duration
+ conns *connpool.Pool
+ ticks *timer.Timer
+ reloadTimeout time.Duration
+ throttledLogger *logutil.ThrottledLogger
// dbCreationFailed is for preventing log spam.
dbCreationFailed bool
@@ -109,7 +112,8 @@ func NewEngine(env tabletenv.Env) *Engine {
Size: 3,
IdleTimeout: env.Config().OltpReadPool.IdleTimeout,
}),
- ticks: timer.NewTimer(reloadTime),
+ ticks: timer.NewTimer(reloadTime),
+ throttledLogger: logutil.NewThrottledLogger("schema-tracker", 1*time.Minute),
}
se.schemaCopy = env.Config().SignalWhenSchemaChange
_ = env.Exporter().NewGaugeDurationFunc("SchemaReloadTime", "vttablet keeps table schemas in its own memory and periodically refreshes it from MySQL. This config controls the reload time.", se.ticks.Interval)
@@ -177,14 +181,15 @@ func (se *Engine) syncSidecarDB(ctx context.Context, conn *dbconnpool.DBConnecti
// EnsureConnectionAndDB ensures that we can connect to mysql.
// If tablet type is primary and there is no db, then the database is created.
// This function can be called before opening the Engine.
-func (se *Engine) EnsureConnectionAndDB(tabletType topodatapb.TabletType) error {
+func (se *Engine) EnsureConnectionAndDB(tabletType topodatapb.TabletType, serving bool) error {
ctx := tabletenv.LocalContext()
// We use AllPrivs since syncSidecarDB() might need to upgrade the schema
conn, err := dbconnpool.NewDBConnection(ctx, se.env.Config().DB.AllPrivsWithDB())
if err == nil {
se.dbCreationFailed = false
// upgrade sidecar db if required, for a tablet with an existing database
- if tabletType == topodatapb.TabletType_PRIMARY {
+ // only run DDL updates when a PRIMARY is transitioning to serving state.
+ if tabletType == topodatapb.TabletType_PRIMARY && serving {
if err := se.syncSidecarDB(ctx, conn); err != nil {
conn.Close()
return err
@@ -193,7 +198,7 @@ func (se *Engine) EnsureConnectionAndDB(tabletType topodatapb.TabletType) error
conn.Close()
return nil
}
- if tabletType != topodatapb.TabletType_PRIMARY {
+ if tabletType != topodatapb.TabletType_PRIMARY || !serving {
return err
}
if merr, isSQLErr := err.(*sqlerror.SQLError); !isSQLErr || merr.Num != sqlerror.ERBadDb {
@@ -420,10 +425,55 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
return err
}
+ var innodbTablesStats map[string]*Table
+ if includeStats {
+ if innodbTableSizesQuery := conn.Conn.BaseShowInnodbTableSizes(); innodbTableSizesQuery != "" {
+ // Since the InnoDB table size query is available to us on this MySQL version, we should use it.
+ // We therefore don't want to query for table sizes in getTableData()
+ includeStats = false
+
+ innodbResults, err := conn.Conn.Exec(ctx, innodbTableSizesQuery, maxTableCount, false)
+ if err != nil {
+ return vterrors.Wrapf(err, "in Engine.reload(), reading innodb tables")
+ }
+ innodbTablesStats = make(map[string]*Table, len(innodbResults.Rows))
+ for _, row := range innodbResults.Rows {
+ innodbTableName := row[0].ToString() // In the form of encoded `schema/table`
+ fileSize, _ := row[1].ToCastUint64()
+ allocatedSize, _ := row[2].ToCastUint64()
+
+ if _, ok := innodbTablesStats[innodbTableName]; !ok {
+ innodbTablesStats[innodbTableName] = &Table{}
+ }
+ // There could be multiple appearances of the same table in the result set:
+ // A table that has FULLTEXT indexes will appear once for the table itself,
+ // with total size of row data, and once for the aggregates size of all
+ // FULLTEXT indexes. We aggregate the sizes of all appearances of the same table.
+ table := innodbTablesStats[innodbTableName]
+ table.FileSize += fileSize
+ table.AllocatedSize += allocatedSize
+
+ if originalTableName, _, found := strings.Cut(innodbTableName, "#p#"); found {
+ // innodbTableName is encoded any special characters are turned into some @0-f0-f0-f value.
+ // Therefore this "#p#" here is a clear indication that we are looking at a partitioned table.
+ // We turn `my@002ddb/tbl_part#p#p0` into `my@002ddb/tbl_part`
+ // and aggregate the total partition sizes.
+ if _, ok := innodbTablesStats[originalTableName]; !ok {
+ innodbTablesStats[originalTableName] = &Table{}
+ originalTable := innodbTablesStats[originalTableName]
+ originalTable.FileSize += fileSize
+ originalTable.AllocatedSize += allocatedSize
+ }
+ }
+ }
+ // See testing in TestEngineReload
+ }
+ }
tableData, err := getTableData(ctx, conn.Conn, includeStats)
if err != nil {
return vterrors.Wrapf(err, "in Engine.reload(), reading tables")
}
+
// On the primary tablet, we also check the data we have stored in our schema tables to see what all needs reloading.
shouldUseDatabase := se.isServingPrimary && se.schemaCopy
@@ -448,7 +498,7 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
udfsChanged, err := getChangedUserDefinedFunctions(ctx, conn.Conn, shouldUseDatabase)
if err != nil {
- return err
+ se.throttledLogger.Errorf("error in getting changed UDFs: %v", err)
}
rec := concurrency.AllErrorRecorder{}
@@ -458,8 +508,14 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
changedTables := make(map[string]*Table)
// created and altered contain the names of created and altered tables for broadcast.
var created, altered []*Table
+ databaseName := se.cp.DBName()
for _, row := range tableData.Rows {
tableName := row[0].ToString()
+ var innodbTable *Table
+ if innodbTablesStats != nil {
+ innodbTableName := fmt.Sprintf("%s/%s", charset.TablenameToFilename(databaseName), charset.TablenameToFilename(tableName))
+ innodbTable = innodbTablesStats[innodbTableName]
+ }
curTables[tableName] = true
createTime, _ := row[2].ToCastInt64()
var fileSize, allocatedSize uint64
@@ -470,6 +526,9 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
// publish the size metrics
se.tableFileSizeGauge.Set(tableName, int64(fileSize))
se.tableAllocatedSizeGauge.Set(tableName, int64(allocatedSize))
+ } else if innodbTable != nil {
+ se.tableFileSizeGauge.Set(tableName, int64(innodbTable.FileSize))
+ se.tableAllocatedSizeGauge.Set(tableName, int64(innodbTable.AllocatedSize))
}
// Table schemas are cached by tabletserver. For each table we cache `information_schema.tables.create_time` (`tbl.CreateTime`).
@@ -497,6 +556,9 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
if includeStats {
tbl.FileSize = fileSize
tbl.AllocatedSize = allocatedSize
+ } else if innodbTable != nil {
+ tbl.FileSize = innodbTable.FileSize
+ tbl.AllocatedSize = innodbTable.AllocatedSize
}
continue
}
@@ -516,6 +578,9 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
if includeStats {
table.FileSize = fileSize
table.AllocatedSize = allocatedSize
+ } else if innodbTable != nil {
+ table.FileSize = innodbTable.FileSize
+ table.AllocatedSize = innodbTable.AllocatedSize
}
table.CreateTime = createTime
changedTables[tableName] = table
@@ -800,6 +865,13 @@ func (se *Engine) broadcast(created, altered, dropped []*Table, udfsChanged bool
}
}
+// BroadcastForTesting is meant to be a testing function that triggers a broadcast call.
+func (se *Engine) BroadcastForTesting(created, altered, dropped []*Table, udfsChanged bool) {
+ se.mu.Lock()
+ defer se.mu.Unlock()
+ se.broadcast(created, altered, dropped, udfsChanged)
+}
+
// GetTable returns the info for a table.
func (se *Engine) GetTable(tableName sqlparser.IdentifierCS) *Table {
se.mu.Lock()
@@ -886,6 +958,7 @@ func NewEngineForTests() *Engine {
tables: make(map[string]*Table),
historian: newHistorian(false, 0, nil),
env: tabletenv.NewEnv(vtenv.NewTestEnv(), tabletenv.NewDefaultConfig(), "SchemaEngineForTests"),
+ notifiers: make(map[string]notifier),
}
return se
}
diff --git a/go/vt/vttablet/tabletserver/schema/engine_test.go b/go/vt/vttablet/tabletserver/schema/engine_test.go
index d4271dee876..3faa7a10554 100644
--- a/go/vt/vttablet/tabletserver/schema/engine_test.go
+++ b/go/vt/vttablet/tabletserver/schema/engine_test.go
@@ -42,6 +42,7 @@ import (
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/dbconfigs"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
@@ -53,11 +54,15 @@ import (
)
const baseShowTablesWithSizesPattern = `SELECT t\.table_name.*SUM\(i\.file_size\).*`
+const baseInnoDBTableSizesPattern = `(?s).*SELECT.*its\.space = it\.space.*SUM\(its\.file_size\).*`
var mustMatch = utils.MustMatchFn(".Mutex")
-func TestOpenAndReload(t *testing.T) {
- db := fakesqldb.New(t)
+// TestOpenAndReloadLegacy
+//
+// Runs with 5.7 env
+func TestOpenAndReloadLegacy(t *testing.T) {
+ db := fakesqldb.NewWithEnv(t, vtenv.NewLegacyTestEnv())
defer db.Close()
schematest.AddDefaultQueries(db)
@@ -86,7 +91,7 @@ func TestOpenAndReload(t *testing.T) {
))
firstReadRowsValue := 12
AddFakeInnoDBReadRowsResult(db, firstReadRowsValue)
- se := newEngine(10*time.Second, 10*time.Second, 0, db)
+ se := newEngine(10*time.Second, 10*time.Second, 0, db, vtenv.NewLegacyTestEnv())
se.Open()
defer se.Close()
@@ -95,6 +100,19 @@ func TestOpenAndReload(t *testing.T) {
assert.Equal(t, int64(0), se.tableFileSizeGauge.Counts()["msg"])
assert.Equal(t, int64(0), se.tableAllocatedSizeGauge.Counts()["msg"])
+ t.Run("EnsureConnectionAndDB", func(t *testing.T) {
+ // Verify that none of the following configurations run any schema change detection queries -
+ // 1. REPLICA serving
+ // 2. REPLICA non-serving
+ // 3. PRIMARY serving
+ err := se.EnsureConnectionAndDB(topodatapb.TabletType_REPLICA, true)
+ require.NoError(t, err)
+ err = se.EnsureConnectionAndDB(topodatapb.TabletType_PRIMARY, false)
+ require.NoError(t, err)
+ err = se.EnsureConnectionAndDB(topodatapb.TabletType_REPLICA, false)
+ require.NoError(t, err)
+ })
+
// Advance time some more.
db.AddQuery("select unix_timestamp()", sqltypes.MakeTestResult(sqltypes.MakeTestFields(
"t",
@@ -278,23 +296,279 @@ func TestOpenAndReload(t *testing.T) {
assert.Equal(t, want, se.GetSchema())
}
-func TestReloadWithSwappedTables(t *testing.T) {
+func TestOpenAndReload(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
schematest.AddDefaultQueries(db)
db.RejectQueryPattern(baseShowTablesWithSizesPattern, "Opening schema engine should query tables without size information")
+ db.RejectQueryPattern(baseInnoDBTableSizesPattern, "Opening schema engine should query tables without size information")
db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
Fields: mysql.BaseShowTablesFields,
RowsAffected: 0,
InsertID: 0,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
+ mysql.BaseShowTablesRow("test_table_03", false, ""),
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ mysql.BaseShowTablesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
+ },
+ SessionStateChanges: "",
+ StatusFlags: 0,
+ })
+
+ // advance to one second after the default 1427325875.
+ db.AddQuery("select unix_timestamp()", sqltypes.MakeTestResult(sqltypes.MakeTestFields(
+ "t",
+ "int64"),
+ "1427325876",
+ ))
+ firstReadRowsValue := 12
+ AddFakeInnoDBReadRowsResult(db, firstReadRowsValue)
+ se := newEngine(10*time.Second, 10*time.Second, 0, db, nil)
+ se.Open()
+ defer se.Close()
+
+ want := initialSchema()
+ mustMatch(t, want, se.GetSchema())
+ assert.Equal(t, int64(0), se.tableFileSizeGauge.Counts()["msg"])
+ assert.Equal(t, int64(0), se.tableAllocatedSizeGauge.Counts()["msg"])
+
+ t.Run("EnsureConnectionAndDB", func(t *testing.T) {
+ // Verify that none of the following configurations run any schema change detection queries -
+ // 1. REPLICA serving
+ // 2. REPLICA non-serving
+ // 3. PRIMARY serving
+ err := se.EnsureConnectionAndDB(topodatapb.TabletType_REPLICA, true)
+ require.NoError(t, err)
+ err = se.EnsureConnectionAndDB(topodatapb.TabletType_PRIMARY, false)
+ require.NoError(t, err)
+ err = se.EnsureConnectionAndDB(topodatapb.TabletType_REPLICA, false)
+ require.NoError(t, err)
+ })
+
+ // Advance time some more.
+ db.AddQuery("select unix_timestamp()", sqltypes.MakeTestResult(sqltypes.MakeTestFields(
+ "t",
+ "int64"),
+ "1427325877",
+ ))
+ assert.EqualValues(t, firstReadRowsValue, se.innoDbReadRowsCounter.Get())
+
+ // Modify test_table_03
+ // Add test_table_04
+ // Drop msg
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ RowsAffected: 0,
+ InsertID: 0,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("test_table_03")), // table_name
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("BASE TABLE")), // table_type
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("1427325877")), // unix_timestamp(t.create_time)
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("")), // table_comment
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ },
+ mysql.BaseShowTablesRow("test_table_04", false, ""),
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ },
+ SessionStateChanges: "",
+ StatusFlags: 0,
+ })
+ // Modify test_table_03
+ // Add test_table_04
+ // Drop msg
+ db.AddQueryPattern(baseInnoDBTableSizesPattern, &sqltypes.Result{
+ Fields: mysql.BaseInnoDBTableSizesFields,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_01"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_02"),
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("fakesqldb/test_table_03")), // table_name
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ },
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_04"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "seq"),
+ },
+ })
+ db.RejectQueryPattern(baseShowTablesWithSizesPattern, "Opening schema engine should query tables without size information")
+
+ db.MockQueriesForTable("test_table_03", &sqltypes.Result{
+ Fields: []*querypb.Field{{
+ Name: "pk1",
+ Type: sqltypes.Int32,
+ }, {
+ Name: "pk2",
+ Type: sqltypes.Int32,
+ }, {
+ Name: "val",
+ Type: sqltypes.Int32,
+ }},
+ })
+
+ db.MockQueriesForTable("test_table_04", &sqltypes.Result{
+ Fields: []*querypb.Field{{
+ Name: "pk",
+ Type: sqltypes.Int32,
+ }},
+ })
+
+ db.AddQuery(mysql.BaseShowPrimary, &sqltypes.Result{
+ Fields: mysql.ShowPrimaryFields,
+ Rows: [][]sqltypes.Value{
+ mysql.ShowPrimaryRow("test_table_01", "pk"),
+ mysql.ShowPrimaryRow("test_table_02", "pk"),
+ mysql.ShowPrimaryRow("test_table_03", "pk1"),
+ mysql.ShowPrimaryRow("test_table_03", "pk2"),
+ mysql.ShowPrimaryRow("test_table_04", "pk"),
+ mysql.ShowPrimaryRow("seq", "id"),
+ },
+ })
+ secondReadRowsValue := 123
+ AddFakeInnoDBReadRowsResult(db, secondReadRowsValue)
+
+ firstTime := true
+ notifier := func(full map[string]*Table, created, altered, dropped []*Table, _ bool) {
+ if firstTime {
+ firstTime = false
+ createTables := extractNamesFromTablesList(created)
+ sort.Strings(createTables)
+ assert.Equal(t, []string{"dual", "msg", "seq", "test_table_01", "test_table_02", "test_table_03"}, createTables)
+ assert.Equal(t, []*Table(nil), altered)
+ assert.Equal(t, []*Table(nil), dropped)
+ } else {
+ assert.Equal(t, []string{"test_table_04"}, extractNamesFromTablesList(created))
+ assert.Equal(t, []string{"test_table_03"}, extractNamesFromTablesList(altered))
+ assert.Equal(t, []string{"msg"}, extractNamesFromTablesList(dropped))
+ }
+ }
+ se.RegisterNotifier("test", notifier, true)
+ err := se.Reload(context.Background())
+ require.NoError(t, err)
+
+ assert.EqualValues(t, secondReadRowsValue, se.innoDbReadRowsCounter.Get())
+
+ want["seq"].FileSize = 100
+ want["seq"].AllocatedSize = 150
+
+ want["test_table_01"].FileSize = 100
+ want["test_table_01"].AllocatedSize = 150
+
+ want["test_table_02"].FileSize = 100
+ want["test_table_02"].AllocatedSize = 150
+
+ want["test_table_03"] = &Table{
+ Name: sqlparser.NewIdentifierCS("test_table_03"),
+ Fields: []*querypb.Field{{
+ Name: "pk1",
+ Type: sqltypes.Int32,
+ }, {
+ Name: "pk2",
+ Type: sqltypes.Int32,
+ }, {
+ Name: "val",
+ Type: sqltypes.Int32,
+ }},
+ PKColumns: []int{0, 1},
+ CreateTime: 1427325877,
+ FileSize: 128,
+ AllocatedSize: 256,
+ }
+ want["test_table_04"] = &Table{
+ Name: sqlparser.NewIdentifierCS("test_table_04"),
+ Fields: []*querypb.Field{{
+ Name: "pk",
+ Type: sqltypes.Int32,
+ }},
+ PKColumns: []int{0},
+ CreateTime: 1427325875,
+ FileSize: 100,
+ AllocatedSize: 150,
+ }
+ delete(want, "msg")
+ assert.Equal(t, want, se.GetSchema())
+ assert.Equal(t, int64(0), se.tableAllocatedSizeGauge.Counts()["msg"])
+ assert.Equal(t, int64(0), se.tableFileSizeGauge.Counts()["msg"])
+
+ // ReloadAt tests
+ pos1, err := replication.DecodePosition("MariaDB/0-41983-20")
+ require.NoError(t, err)
+ pos2, err := replication.DecodePosition("MariaDB/0-41983-40")
+ require.NoError(t, err)
+ se.UnregisterNotifier("test")
+
+ err = se.ReloadAt(context.Background(), replication.Position{})
+ require.NoError(t, err)
+ assert.Equal(t, want, se.GetSchema())
+
+ err = se.ReloadAt(context.Background(), pos1)
+ require.NoError(t, err)
+ assert.Equal(t, want, se.GetSchema())
+
+ db.AddQueryPattern(baseShowTablesWithSizesPattern, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesWithSizesFields,
Rows: [][]sqltypes.Value{
mysql.BaseShowTablesWithSizesRow("test_table_01", false, ""),
mysql.BaseShowTablesWithSizesRow("test_table_02", false, ""),
- mysql.BaseShowTablesWithSizesRow("test_table_03", false, ""),
+ mysql.BaseShowTablesWithSizesRow("test_table_04", false, ""),
mysql.BaseShowTablesWithSizesRow("seq", false, "vitess_sequence"),
- mysql.BaseShowTablesWithSizesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
+ },
+ })
+
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
+ mysql.BaseShowTablesRow("test_table_04", false, ""),
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ },
+ })
+
+ db.AddQuery(mysql.BaseShowPrimary, &sqltypes.Result{
+ Fields: mysql.ShowPrimaryFields,
+ Rows: [][]sqltypes.Value{
+ mysql.ShowPrimaryRow("test_table_01", "pk"),
+ mysql.ShowPrimaryRow("test_table_02", "pk"),
+ mysql.ShowPrimaryRow("test_table_04", "pk"),
+ mysql.ShowPrimaryRow("seq", "id"),
+ },
+ })
+ err = se.ReloadAt(context.Background(), pos1)
+ require.NoError(t, err)
+ assert.Equal(t, want, se.GetSchema())
+
+ delete(want, "test_table_03")
+ err = se.ReloadAt(context.Background(), pos2)
+ require.NoError(t, err)
+ assert.Equal(t, want, se.GetSchema())
+}
+
+func TestReloadWithSwappedTables(t *testing.T) {
+ db := fakesqldb.New(t)
+ defer db.Close()
+ schematest.AddDefaultQueries(db)
+
+ db.RejectQueryPattern(baseShowTablesWithSizesPattern, "Opening schema engine should query tables without size information")
+ db.RejectQueryPattern(baseInnoDBTableSizesPattern, "Opening schema engine should query tables without size information")
+
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ RowsAffected: 0,
+ InsertID: 0,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
+ mysql.BaseShowTablesRow("test_table_03", false, ""),
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ mysql.BaseShowTablesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
},
SessionStateChanges: "",
StatusFlags: 0,
@@ -302,7 +576,7 @@ func TestReloadWithSwappedTables(t *testing.T) {
firstReadRowsValue := 12
AddFakeInnoDBReadRowsResult(db, firstReadRowsValue)
- se := newEngine(10*time.Second, 10*time.Second, 0, db)
+ se := newEngine(10*time.Second, 10*time.Second, 0, db, nil)
se.Open()
defer se.Close()
want := initialSchema()
@@ -315,24 +589,43 @@ func TestReloadWithSwappedTables(t *testing.T) {
"int64"),
"1427325876",
))
- db.AddQueryPattern(baseShowTablesWithSizesPattern, &sqltypes.Result{
- Fields: mysql.BaseShowTablesWithSizesFields,
+ db.AddQueryPattern(baseInnoDBTableSizesPattern, &sqltypes.Result{
+ Fields: mysql.BaseInnoDBTableSizesFields,
Rows: [][]sqltypes.Value{
- mysql.BaseShowTablesWithSizesRow("test_table_01", false, ""),
- mysql.BaseShowTablesWithSizesRow("test_table_02", false, ""),
- mysql.BaseShowTablesWithSizesRow("test_table_03", false, ""),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_01"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_02"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_03"),
{
- sqltypes.MakeTrusted(sqltypes.VarChar, []byte("test_table_04")),
- sqltypes.MakeTrusted(sqltypes.VarChar, []byte("BASE TABLE")),
- sqltypes.MakeTrusted(sqltypes.Int64, []byte("1427325877")), // unix_timestamp(create_time)
- sqltypes.MakeTrusted(sqltypes.VarChar, []byte("")),
- sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
- sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("fakesqldb/test_table_04")), // table_name
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
},
- mysql.BaseShowTablesWithSizesRow("seq", false, "vitess_sequence"),
- mysql.BaseShowTablesWithSizesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "seq"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "msg"),
},
})
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ RowsAffected: 0,
+ InsertID: 0,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
+ mysql.BaseShowTablesRow("test_table_03", false, ""),
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("test_table_04")), // table_name
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("BASE TABLE")), // table_type
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("1427325877")), // unix_timestamp(t.create_time)
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("")), // table_comment
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ },
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ mysql.BaseShowTablesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
+ },
+ SessionStateChanges: "",
+ StatusFlags: 0,
+ })
db.MockQueriesForTable("test_table_04", &sqltypes.Result{
Fields: []*querypb.Field{{
Name: "mypk",
@@ -389,11 +682,28 @@ func TestReloadWithSwappedTables(t *testing.T) {
"int64"),
"1427325877",
))
- db.AddQueryPattern(baseShowTablesWithSizesPattern, &sqltypes.Result{
- Fields: mysql.BaseShowTablesWithSizesFields,
+ db.AddQueryPattern(baseInnoDBTableSizesPattern, &sqltypes.Result{
+ Fields: mysql.BaseInnoDBTableSizesFields,
+ Rows: [][]sqltypes.Value{
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_01"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_02"),
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("fakesqldb/test_table_03")), // table_name
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ },
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "test_table_04"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "seq"),
+ mysql.BaseInnoDBTableSizesRow("fakesqldb", "msg"),
+ },
+ })
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ RowsAffected: 0,
+ InsertID: 0,
Rows: [][]sqltypes.Value{
- mysql.BaseShowTablesWithSizesRow("test_table_01", false, ""),
- mysql.BaseShowTablesWithSizesRow("test_table_02", false, ""),
+ mysql.BaseShowTablesRow("test_table_01", false, ""),
+ mysql.BaseShowTablesRow("test_table_02", false, ""),
{
sqltypes.MakeTrusted(sqltypes.VarChar, []byte("test_table_03")),
sqltypes.MakeTrusted(sqltypes.VarChar, []byte("BASE TABLE")),
@@ -402,10 +712,12 @@ func TestReloadWithSwappedTables(t *testing.T) {
sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
},
- mysql.BaseShowTablesWithSizesRow("test_table_04", false, ""),
- mysql.BaseShowTablesWithSizesRow("seq", false, "vitess_sequence"),
- mysql.BaseShowTablesWithSizesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
+ mysql.BaseShowTablesRow("test_table_04", false, ""),
+ mysql.BaseShowTablesRow("seq", false, "vitess_sequence"),
+ mysql.BaseShowTablesRow("msg", false, "vitess_message,vt_ack_wait=30,vt_purge_after=120,vt_batch_size=1,vt_cache_size=10,vt_poller_interval=30"),
},
+ SessionStateChanges: "",
+ StatusFlags: 0,
})
db.MockQueriesForTable("test_table_03", &sqltypes.Result{
Fields: []*querypb.Field{{
@@ -467,8 +779,8 @@ func TestOpenFailedDueToExecErr(t *testing.T) {
defer db.Close()
schematest.AddDefaultQueries(db)
want := "injected error"
- db.AddRejectedQuery(mysql.BaseShowTables, fmt.Errorf(want))
- se := newEngine(1*time.Second, 1*time.Second, 0, db)
+ db.AddRejectedQuery(mysql.BaseShowTables, errors.New(want))
+ se := newEngine(1*time.Second, 1*time.Second, 0, db, nil)
err := se.Open()
if err == nil || !strings.Contains(err.Error(), want) {
t.Errorf("se.Open: %v, want %s", err, want)
@@ -499,7 +811,7 @@ func TestOpenFailedDueToLoadTableErr(t *testing.T) {
db.AddRejectedQuery("SELECT * FROM `fakesqldb`.`test_view` WHERE 1 != 1", sqlerror.NewSQLErrorFromError(errors.New("The user specified as a definer ('root'@'%') does not exist (errno 1449) (sqlstate HY000)")))
AddFakeInnoDBReadRowsResult(db, 0)
- se := newEngine(1*time.Second, 1*time.Second, 0, db)
+ se := newEngine(1*time.Second, 1*time.Second, 0, db, nil)
err := se.Open()
// failed load should return an error because of test_table
assert.ErrorContains(t, err, "Row count exceeded")
@@ -534,7 +846,7 @@ func TestOpenNoErrorDueToInvalidViews(t *testing.T) {
db.AddRejectedQuery("SELECT `col1`, `col2` FROM `fakesqldb`.`bar_view` WHERE 1 != 1", sqlerror.NewSQLError(sqlerror.ERWrongFieldWithGroup, sqlerror.SSClientError, "random error for table bar_view"))
AddFakeInnoDBReadRowsResult(db, 0)
- se := newEngine(1*time.Second, 1*time.Second, 0, db)
+ se := newEngine(1*time.Second, 1*time.Second, 0, db, nil)
err := se.Open()
require.NoError(t, err)
@@ -550,7 +862,7 @@ func TestExportVars(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
schematest.AddDefaultQueries(db)
- se := newEngine(1*time.Second, 1*time.Second, 0, db)
+ se := newEngine(1*time.Second, 1*time.Second, 0, db, nil)
se.Open()
defer se.Close()
expvar.Do(func(kv expvar.KeyValue) {
@@ -562,7 +874,7 @@ func TestStatsURL(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
schematest.AddDefaultQueries(db)
- se := newEngine(1*time.Second, 1*time.Second, 0, db)
+ se := newEngine(1*time.Second, 1*time.Second, 0, db, nil)
se.Open()
defer se.Close()
@@ -592,7 +904,7 @@ func TestSchemaEngineCloseTickRace(t *testing.T) {
})
AddFakeInnoDBReadRowsResult(db, 12)
// Start the engine with a small reload tick
- se := newEngine(100*time.Millisecond, 1*time.Second, 0, db)
+ se := newEngine(100*time.Millisecond, 1*time.Second, 0, db, nil)
err := se.Open()
require.NoError(t, err)
@@ -619,15 +931,20 @@ func TestSchemaEngineCloseTickRace(t *testing.T) {
}
}
-func newEngine(reloadTime time.Duration, idleTimeout time.Duration, schemaMaxAgeSeconds int64, db *fakesqldb.DB) *Engine {
+func newEngine(reloadTime time.Duration, idleTimeout time.Duration, schemaMaxAgeSeconds int64, db *fakesqldb.DB, env *vtenv.Environment) *Engine {
cfg := tabletenv.NewDefaultConfig()
cfg.SchemaReloadInterval = reloadTime
cfg.OltpReadPool.IdleTimeout = idleTimeout
cfg.OlapReadPool.IdleTimeout = idleTimeout
cfg.TxPool.IdleTimeout = idleTimeout
cfg.SchemaVersionMaxAgeSeconds = schemaMaxAgeSeconds
- se := NewEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "SchemaTest"))
- se.InitDBConfig(newDBConfigs(db).DbaWithDB())
+ dbConfigs := newDBConfigs(db)
+ cfg.DB = dbConfigs
+ if env == nil {
+ env = vtenv.NewTestEnv()
+ }
+ se := NewEngine(tabletenv.NewEnv(env, cfg, "SchemaTest"))
+ se.InitDBConfig(dbConfigs.DbaWithDB())
return se
}
@@ -737,6 +1054,7 @@ func initialSchema() map[string]*Table {
BatchSize: 1,
CacheSize: 10,
PollInterval: 30 * time.Second,
+ IDType: sqltypes.Int64,
},
},
}
@@ -1015,7 +1333,6 @@ func TestEngineGetTableData(t *testing.T) {
name string
expectedQueries map[string]*sqltypes.Result
queriesToReject map[string]error
- includeStats bool
expectedError string
}{
{
@@ -1023,13 +1340,6 @@ func TestEngineGetTableData(t *testing.T) {
expectedQueries: map[string]*sqltypes.Result{
conn.BaseShowTables(): {},
},
- includeStats: false,
- }, {
- name: "Success with include stats",
- expectedQueries: map[string]*sqltypes.Result{
- conn.BaseShowTablesWithSizes(): {},
- },
- includeStats: true,
}, {
name: "Error in query",
queriesToReject: map[string]error{
@@ -1051,7 +1361,7 @@ func TestEngineGetTableData(t *testing.T) {
defer db.DeleteRejectedQuery(query)
}
- _, err = getTableData(context.Background(), conn, tt.includeStats)
+ _, err = getTableData(context.Background(), conn, false)
if tt.expectedError != "" {
require.ErrorContains(t, err, tt.expectedError)
return
@@ -1178,183 +1488,204 @@ func TestEngineGetDroppedTables(t *testing.T) {
// TestEngineReload tests the entire functioning of engine.Reload testing all the queries that we end up running against MySQL
// while simulating the responses and verifies the final list of created, altered and dropped tables.
func TestEngineReload(t *testing.T) {
- db := fakesqldb.New(t)
- cfg := tabletenv.NewDefaultConfig()
- cfg.DB = newDBConfigs(db)
- cfg.SignalWhenSchemaChange = true
- env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestEngineReload")
- conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
- require.NoError(t, err)
-
- se := newEngine(10*time.Second, 10*time.Second, 0, db)
- se.conns.Open(se.cp, se.cp, se.cp)
- se.isOpen = true
- se.notifiers = make(map[string]notifier)
- se.MakePrimary(true)
+ envs := []*vtenv.Environment{
+ vtenv.NewTestEnv(),
+ vtenv.NewLegacyTestEnv(),
+ }
+ for _, venv := range envs {
+ t.Run(venv.MySQLVersion(), func(t *testing.T) {
+ db := fakesqldb.NewWithEnv(t, venv)
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ cfg.SignalWhenSchemaChange = true
+
+ env := tabletenv.NewEnv(venv, nil, "TestEngineReload")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
+ require.NoError(t, err)
- // If we have to skip the meta check, then there is nothing to do
- se.SkipMetaCheck = true
- err = se.reload(context.Background(), false)
- require.NoError(t, err)
+ se := newEngine(10*time.Second, 10*time.Second, 0, db, venv)
+ se.conns.Open(se.cp, se.cp, se.cp)
+ se.isOpen = true
+ se.notifiers = make(map[string]notifier)
+ se.MakePrimary(true)
- se.SkipMetaCheck = false
- se.lastChange = 987654321
+ // If we have to skip the meta check, then there is nothing to do
+ se.SkipMetaCheck = true
+ err = se.reload(context.Background(), false)
+ require.NoError(t, err)
- // Initial tables in the schema engine
- se.tables = map[string]*Table{
- "t1": {
- Name: sqlparser.NewIdentifierCS("t1"),
- Type: NoType,
- CreateTime: 123456789,
- },
- "t2": {
- Name: sqlparser.NewIdentifierCS("t2"),
- Type: NoType,
- CreateTime: 123456789,
- },
- "t4": {
- Name: sqlparser.NewIdentifierCS("t4"),
- Type: NoType,
- CreateTime: 123456789,
- },
- "v1": {
- Name: sqlparser.NewIdentifierCS("v1"),
- Type: View,
- CreateTime: 123456789,
- },
- "v2": {
- Name: sqlparser.NewIdentifierCS("v2"),
- Type: View,
- CreateTime: 123456789,
- },
- "v4": {
- Name: sqlparser.NewIdentifierCS("v4"),
- Type: View,
- CreateTime: 123456789,
- },
- }
- // MySQL unix timestamp query.
- db.AddQuery("SELECT UNIX_TIMESTAMP()", sqltypes.MakeTestResult(sqltypes.MakeTestFields("UNIX_TIMESTAMP", "int64"), "987654326"))
- // Table t2 is updated, T2 is created and t4 is deleted.
- // View v2 is updated, V2 is created and v4 is deleted.
- db.AddQuery(conn.BaseShowTables(), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|table_type|unix_timestamp(create_time)|table_comment",
- "varchar|varchar|int64|varchar"),
- "t1|BASE_TABLE|123456789|",
- "t2|BASE_TABLE|123456790|",
- "T2|BASE_TABLE|123456789|",
- "v1|VIEW|123456789|",
- "v2|VIEW|123456789|",
- "V2|VIEW|123456789|",
- ))
+ se.SkipMetaCheck = false
+ se.lastChange = 987654321
- // Detecting view changes.
- // According to the database, v2, V2, v4, and v5 require updating.
- db.AddQuery(fmt.Sprintf(detectViewChange, sidecar.GetIdentifier()), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name", "varchar"),
- "v2",
- "V2",
- "v4",
- "v5",
- ))
+ // Initial tables in the schema engine
+ se.tables = map[string]*Table{
+ "t1": {
+ Name: sqlparser.NewIdentifierCS("t1"),
+ Type: NoType,
+ CreateTime: 123456789,
+ },
+ "t2": {
+ Name: sqlparser.NewIdentifierCS("t2"),
+ Type: NoType,
+ CreateTime: 123456789,
+ },
+ "t4": {
+ Name: sqlparser.NewIdentifierCS("t4"),
+ Type: NoType,
+ CreateTime: 123456789,
+ },
+ "v1": {
+ Name: sqlparser.NewIdentifierCS("v1"),
+ Type: View,
+ CreateTime: 123456789,
+ },
+ "v2": {
+ Name: sqlparser.NewIdentifierCS("v2"),
+ Type: View,
+ CreateTime: 123456789,
+ },
+ "v4": {
+ Name: sqlparser.NewIdentifierCS("v4"),
+ Type: View,
+ CreateTime: 123456789,
+ },
+ }
+ // MySQL unix timestamp query.
+ db.AddQuery("SELECT UNIX_TIMESTAMP()", sqltypes.MakeTestResult(sqltypes.MakeTestFields("UNIX_TIMESTAMP", "int64"), "987654326"))
+ // Table t2 is updated, T2 is created and t4 is deleted.
+ // View v2 is updated, V2 is created and v4 is deleted.
+ db.AddQuery(conn.BaseShowTables(), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|table_type|unix_timestamp(create_time)|table_comment",
+ "varchar|varchar|int64|varchar"),
+ "t1|BASE_TABLE|123456789|",
+ "t2|BASE_TABLE|123456790|",
+ "T2|BASE_TABLE|123456789|",
+ "v1|VIEW|123456789|",
+ "v2|VIEW|123456789|",
+ "V2|VIEW|123456789|",
+ ))
+ // Detecting view changes.
+ // According to the database, v2, V2, v4, and v5 require updating.
+ db.AddQuery(fmt.Sprintf(detectViewChange, sidecar.GetIdentifier()), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name", "varchar"),
+ "v2",
+ "V2",
+ "v4",
+ "v5",
+ ))
- // Finding mismatches in the tables.
- // t5 exists in the database.
- db.AddQuery("SELECT TABLE_NAME, CREATE_TIME FROM _vt.`tables`", sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|create_time", "varchar|int64"),
- "t1|123456789",
- "t2|123456789",
- "t4|123456789",
- "t5|123456789",
- ))
+ // Finding mismatches in the tables.
+ // t5 exists in the database.
+ db.AddQuery("SELECT TABLE_NAME, CREATE_TIME FROM _vt.`tables`", sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|create_time", "varchar|int64"),
+ "t1|123456789",
+ "t2|123456789",
+ "t4|123456789",
+ "t5|123456789",
+ ))
- // Read Innodb_rows_read.
- db.AddQuery(mysql.ShowRowsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|int64"),
- "Innodb_rows_read|35"))
+ // Read Innodb_rows_read.
+ db.AddQuery(mysql.ShowRowsRead, sqltypes.MakeTestResult(sqltypes.MakeTestFields("Variable_name|Value", "varchar|int64"),
+ "Innodb_rows_read|35"))
- // Queries to load the tables' information.
- for _, tableName := range []string{"t2", "T2", "v2", "V2"} {
- db.AddQuery(fmt.Sprintf(`SELECT COLUMN_NAME as column_name
+ // Queries to load the tables' information.
+ for _, tableName := range []string{"t2", "T2", "v2", "V2"} {
+ db.AddQuery(fmt.Sprintf(`SELECT COLUMN_NAME as column_name
FROM INFORMATION_SCHEMA.COLUMNS
WHERE TABLE_SCHEMA = 'fakesqldb' AND TABLE_NAME = '%s'
ORDER BY ORDINAL_POSITION`, tableName),
- sqltypes.MakeTestResult(sqltypes.MakeTestFields("column_name", "varchar"),
- "col1"))
- db.AddQuery(fmt.Sprintf("SELECT `col1` FROM `fakesqldb`.`%v` WHERE 1 != 1", tableName), sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1", "varchar")))
- }
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("column_name", "varchar"),
+ "col1"))
+ db.AddQuery(fmt.Sprintf("SELECT `col1` FROM `fakesqldb`.`%v` WHERE 1 != 1", tableName), sqltypes.MakeTestResult(sqltypes.MakeTestFields("col1", "varchar")))
+ }
- // Primary key information.
- db.AddQuery(mysql.BaseShowPrimary, sqltypes.MakeTestResult(mysql.ShowPrimaryFields,
- "t1|col1",
- "t2|col1",
- "T2|col1",
- ))
+ // Primary key information.
+ db.AddQuery(mysql.BaseShowPrimary, sqltypes.MakeTestResult(mysql.ShowPrimaryFields,
+ "t1|col1",
+ "t2|col1",
+ "T2|col1",
+ ))
- // Queries for reloading the tables' information.
- {
- for _, tableName := range []string{"t2", "T2"} {
- db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
- sqltypes.MakeTestResult(sqltypes.MakeTestFields("Table | Create Table", "varchar|varchar"),
- fmt.Sprintf("%v|create_table_%v", tableName, tableName)))
- }
- db.AddQuery("begin", &sqltypes.Result{})
- db.AddQuery("commit", &sqltypes.Result{})
- db.AddQuery("rollback", &sqltypes.Result{})
- // We are adding both the variants of the delete statements that we can see in the test, since the deleted tables are initially stored as a map, the order is not defined.
- db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t5', 't4', 'T2', 't2')", &sqltypes.Result{})
- db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t4', 't5', 'T2', 't2')", &sqltypes.Result{})
- db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 't2', 'create_table_t2', 123456790)", &sqltypes.Result{})
- db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 'T2', 'create_table_T2', 123456789)", &sqltypes.Result{})
- }
+ // Queries for reloading the tables' information.
+ {
+ for _, tableName := range []string{"t2", "T2"} {
+ db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("Table | Create Table", "varchar|varchar"),
+ fmt.Sprintf("%v|create_table_%v", tableName, tableName)))
+ }
+ db.AddQuery("begin", &sqltypes.Result{})
+ db.AddQuery("commit", &sqltypes.Result{})
+ db.AddQuery("rollback", &sqltypes.Result{})
+ // We are adding both the variants of the delete statements that we can see in the test, since the deleted tables are initially stored as a map, the order is not defined.
+ db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t5', 't4', 'T2', 't2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t4', 't5', 'T2', 't2')", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 't2', 'create_table_t2', 123456790)", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 'T2', 'create_table_T2', 123456789)", &sqltypes.Result{})
+ }
- // Queries for reloading the views' information.
- {
- for _, tableName := range []string{"v2", "V2"} {
- db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
- sqltypes.MakeTestResult(sqltypes.MakeTestFields(" View | Create View | character_set_client | collation_connection", "varchar|varchar|varchar|varchar"),
- fmt.Sprintf("%v|create_table_%v|utf8mb4|utf8mb4_0900_ai_ci", tableName, tableName)))
- }
- // We are adding both the variants of the select statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
- db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v4', 'v5', 'V2', 'v2')",
- sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
- "v2|select_v2",
- "V2|select_V2",
- ))
- db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v5', 'v4', 'V2', 'v2')",
- sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
- "v2|select_v2",
- "V2|select_V2",
- ))
+ // Queries for reloading the views' information.
+ {
+ for _, tableName := range []string{"v2", "V2"} {
+ db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields(" View | Create View | character_set_client | collation_connection", "varchar|varchar|varchar|varchar"),
+ fmt.Sprintf("%v|create_table_%v|utf8mb4|utf8mb4_0900_ai_ci", tableName, tableName)))
+ }
+ // We are adding both the variants of the select statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
+ db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v4', 'v5', 'V2', 'v2')",
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
+ "v2|select_v2",
+ "V2|select_V2",
+ ))
+ db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v5', 'v4', 'V2', 'v2')",
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
+ "v2|select_v2",
+ "V2|select_V2",
+ ))
+
+ // We are adding both the variants of the delete statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
+ db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v4', 'v5', 'V2', 'v2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v5', 'v4', 'V2', 'v2')", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'v2', 'create_table_v2', 'select_v2')", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'V2', 'create_table_V2', 'select_V2')", &sqltypes.Result{})
+ }
- // We are adding both the variants of the delete statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
- db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v4', 'v5', 'V2', 'v2')", &sqltypes.Result{})
- db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v5', 'v4', 'V2', 'v2')", &sqltypes.Result{})
- db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'v2', 'create_table_v2', 'select_v2')", &sqltypes.Result{})
- db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'V2', 'create_table_V2', 'select_V2')", &sqltypes.Result{})
- }
+ // adding query pattern for udfs
+ udfQueryPattern := "SELECT name.*mysql.func.*"
+ db.AddQueryPattern(udfQueryPattern, &sqltypes.Result{})
- // adding query pattern for udfs
- db.AddQueryPattern("SELECT name.*", &sqltypes.Result{})
+ // Verify the list of created, altered and dropped tables seen.
+ se.RegisterNotifier("test", func(full map[string]*Table, created, altered, dropped []*Table, _ bool) {
+ require.ElementsMatch(t, extractNamesFromTablesList(created), []string{"T2", "V2"})
+ require.ElementsMatch(t, extractNamesFromTablesList(altered), []string{"t2", "v2"})
+ require.ElementsMatch(t, extractNamesFromTablesList(dropped), []string{"t4", "v4", "t5", "v5"})
+ }, false)
- // Verify the list of created, altered and dropped tables seen.
- se.RegisterNotifier("test", func(full map[string]*Table, created, altered, dropped []*Table, _ bool) {
- require.ElementsMatch(t, extractNamesFromTablesList(created), []string{"T2", "V2"})
- require.ElementsMatch(t, extractNamesFromTablesList(altered), []string{"t2", "v2"})
- require.ElementsMatch(t, extractNamesFromTablesList(dropped), []string{"t4", "v4", "t5", "v5"})
- }, false)
+ // Run the reload.
+ err = se.reload(context.Background(), false)
+ require.NoError(t, err)
+ require.NoError(t, db.LastError())
+ require.Zero(t, se.throttledLogger.GetLastLogTime())
- // Run the reload.
- err = se.reload(context.Background(), false)
- require.NoError(t, err)
- require.NoError(t, db.LastError())
+ // Now if we remove the query pattern for udfs, schema engine shouldn't fail.
+ // Instead we should see a log message with the error.
+ db.RemoveQueryPattern(udfQueryPattern)
+ se.UnregisterNotifier("test")
+ err = se.reload(context.Background(), false)
+ require.NoError(t, err)
+ // Check for the udf error being logged. The last log time should be less than a second.
+ require.Less(t, time.Since(se.throttledLogger.GetLastLogTime()), 1*time.Second)
+ })
+ }
}
-// TestEngineReload tests the vreplication specific GetTableForPos function to ensure
+// TestGetTableForPosLegacy tests the vreplication specific GetTableForPos function to ensure
// that it conforms to the intended/expected behavior in various scenarios.
// This more specifically tests the behavior of the function when the historian is
// disabled or otherwise unable to get a table schema for the given position. When it
// CAN, that is tested indepenently in the historian tests.
-func TestGetTableForPos(t *testing.T) {
+//
+// Runs with 5.7 env
+func TestGetTableForPosLegacy(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- fakedb := fakesqldb.New(t)
+ fakedb := fakesqldb.NewWithEnv(t, vtenv.NewLegacyTestEnv())
cfg := tabletenv.NewDefaultConfig()
cfg.DB = newDBConfigs(fakedb)
table := sqlparser.NewIdentifierCS("t1")
@@ -1372,7 +1703,7 @@ func TestGetTableForPos(t *testing.T) {
}
// Don't do any automatic / TTL based cache refreshes.
- se := newEngine(1*time.Hour, 1*time.Hour, 0, fakedb)
+ se := newEngine(1*time.Hour, 1*time.Hour, 0, fakedb, vtenv.NewLegacyTestEnv())
se.conns.Open(se.cp, se.cp, se.cp)
se.isOpen = true
se.notifiers = make(map[string]notifier)
@@ -1534,3 +1865,197 @@ func TestGetTableForPos(t *testing.T) {
})
}
}
+
+// TestGetTableForPos tests the vreplication specific GetTableForPos function to ensure
+// that it conforms to the intended/expected behavior in various scenarios.
+// This more specifically tests the behavior of the function when the historian is
+// disabled or otherwise unable to get a table schema for the given position. When it
+// CAN, that is tested indepenently in the historian tests.
+func TestGetTableForPos(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ fakedb := fakesqldb.New(t)
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(fakedb)
+ table := sqlparser.NewIdentifierCS("t1")
+ column := "col1"
+ tableSchema := fmt.Sprintf("create table %s (%s varchar(50), primary key(col1))", table.String(), column)
+ tableMt := &binlogdatapb.MinimalTable{
+ Name: table.String(),
+ Fields: []*querypb.Field{
+ {
+ Name: column,
+ Type: sqltypes.VarChar,
+ },
+ },
+ PKColumns: []int64{0}, // First column: col1
+ }
+
+ // Don't do any automatic / TTL based cache refreshes.
+ se := newEngine(1*time.Hour, 1*time.Hour, 0, fakedb, nil)
+ se.conns.Open(se.cp, se.cp, se.cp)
+ se.isOpen = true
+ se.notifiers = make(map[string]notifier)
+ se.MakePrimary(true)
+ se.historian.enabled = false
+
+ addExpectedReloadQueries := func(db *fakesqldb.DB) {
+ db.AddQuery("SELECT UNIX_TIMESTAMP()", sqltypes.MakeTestResult(sqltypes.MakeTestFields(
+ "UNIX_TIMESTAMP()",
+ "int64"),
+ fmt.Sprintf("%d", time.Now().Unix()),
+ ))
+ db.AddQuery(fmt.Sprintf(detectViewChange, sidecar.GetIdentifier()), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name", "varchar")))
+ db.AddQuery(fmt.Sprintf(readTableCreateTimes, sidecar.GetIdentifier()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|create_time", "varchar|int64")))
+ db.AddQuery(fmt.Sprintf(detectUdfChange, sidecar.GetIdentifier()), &sqltypes.Result{})
+ db.AddQueryPattern(baseInnoDBTableSizesPattern, &sqltypes.Result{
+ Fields: mysql.BaseInnoDBTableSizesFields,
+ Rows: [][]sqltypes.Value{
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("fakesqldb/"+table.String())), // table_name
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("128")), // file_size
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte("256")), // allocated_size
+ },
+ },
+ })
+ db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{
+ Fields: mysql.BaseShowTablesFields,
+ Rows: [][]sqltypes.Value{
+ {
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte(table.String())), // table_name
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("BASE TABLE")), // table_type
+ sqltypes.MakeTrusted(sqltypes.Int64, []byte(fmt.Sprintf("%d", time.Now().Unix()-1000))), // unix_timestamp(t.create_time)
+ sqltypes.MakeTrusted(sqltypes.VarChar, []byte("")), // table_comment
+ },
+ },
+ SessionStateChanges: "",
+ StatusFlags: 0,
+ })
+ db.RejectQueryPattern(baseShowTablesWithSizesPattern, "we should expect to get sizes by InnoDBTableSizes")
+ db.AddQuery(mysql.BaseShowPrimary, &sqltypes.Result{
+ Fields: mysql.ShowPrimaryFields,
+ Rows: [][]sqltypes.Value{
+ mysql.ShowPrimaryRow(table.String(), column),
+ },
+ })
+ db.AddQueryPattern(fmt.Sprintf(mysql.GetColumnNamesQueryPatternForTable, table.String()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("column_name", "varchar"), column))
+ db.AddQuery(fmt.Sprintf("SELECT `%s` FROM `fakesqldb`.`%v` WHERE 1 != 1", column, table.String()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields(column, "varchar")))
+ db.AddQuery(fmt.Sprintf(`show create table %s`, table.String()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("Table|Create Table", "varchar|varchar"), table.String(), tableSchema))
+ db.AddQuery("begin", &sqltypes.Result{})
+ db.AddQuery(fmt.Sprintf("delete from %s.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('%s')",
+ sidecar.GetIdentifier(), table.String()), &sqltypes.Result{})
+ db.AddQuery(fmt.Sprintf("insert into %s.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), '%s', '%s', %d)",
+ sidecar.GetIdentifier(), table.String(), tableSchema, time.Now().Unix()), &sqltypes.Result{RowsAffected: 1})
+ db.AddQuery("rollback", &sqltypes.Result{})
+ }
+
+ type testcase struct {
+ name string
+ initialCacheState map[string]*Table
+ expectedQueriesFunc func(db *fakesqldb.DB)
+ expectFunc func()
+ }
+ tests := []testcase{
+ {
+ name: "GetTableForPos with cache uninitialized",
+ initialCacheState: make(map[string]*Table), // empty
+ expectedQueriesFunc: func(db *fakesqldb.DB) {
+ // We do a reload to initialize the cache.
+ addExpectedReloadQueries(db)
+ },
+ expectFunc: func() {
+ tbl, err := se.GetTableForPos(ctx, table, "")
+ require.NoError(t, err)
+ require.Equal(t, tableMt, tbl)
+ },
+ },
+ {
+ name: "GetTableForPos with cache uninitialized, table not found",
+ initialCacheState: make(map[string]*Table), // empty
+ expectedQueriesFunc: func(db *fakesqldb.DB) {
+ // We do a reload to initialize the cache and in doing so get the missing table.
+ addExpectedReloadQueries(db)
+ },
+ expectFunc: func() {
+ tbl, err := se.GetTableForPos(ctx, sqlparser.NewIdentifierCS("nobueno"), "")
+ require.EqualError(t, err, "table nobueno not found in vttablet schema")
+ require.Nil(t, tbl)
+ },
+ },
+ {
+ name: "GetTableForPos with cache initialized, table not found",
+ initialCacheState: map[string]*Table{"t2": {Name: sqlparser.NewIdentifierCS("t2")}},
+ expectedQueriesFunc: func(db *fakesqldb.DB) {
+ // We do a reload to try and get this missing table and any other recently created ones.
+ addExpectedReloadQueries(db)
+ },
+ expectFunc: func() {
+ tbl, err := se.GetTableForPos(ctx, table, "")
+ require.NoError(t, err)
+ require.Equal(t, tableMt, tbl)
+ },
+ },
+ {
+ name: "GetTableForPos with cache initialized, table found",
+ initialCacheState: map[string]*Table{table.String(): {Name: table}},
+ expectedQueriesFunc: func(db *fakesqldb.DB) {
+ // We only reload the column and PK info for the table in our cache. A new column
+ // called col2 has been added to the table schema and it is the new PK.
+ newTableSchema := fmt.Sprintf("create table %s (%s varchar(50), col2 varchar(50), primary key(col2))", table.String(), column)
+ db.AddQuery(mysql.BaseShowPrimary, &sqltypes.Result{
+ Fields: mysql.ShowPrimaryFields,
+ Rows: [][]sqltypes.Value{
+ mysql.ShowPrimaryRow(table.String(), "col2"),
+ },
+ })
+ db.AddQueryPattern(fmt.Sprintf(mysql.GetColumnNamesQueryPatternForTable, table.String()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("column_name", "varchar"), column, "col2"))
+ db.AddQuery(fmt.Sprintf("SELECT `%s`, `%s` FROM `fakesqldb`.`%v` WHERE 1 != 1",
+ column, "col2", table.String()), sqltypes.MakeTestResult(sqltypes.MakeTestFields(fmt.Sprintf("%s|%s", column, "col2"), "varchar|varchar")))
+ db.AddQuery(fmt.Sprintf(`show create table %s`, table.String()),
+ sqltypes.MakeTestResult(sqltypes.MakeTestFields("Table|Create Table", "varchar|varchar"), table.String(), newTableSchema))
+ db.AddQuery("begin", &sqltypes.Result{})
+ db.AddQuery(fmt.Sprintf("delete from %s.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('%s')",
+ sidecar.GetIdentifier(), table.String()), &sqltypes.Result{})
+ db.AddQuery(fmt.Sprintf("insert into %s.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), '%s', '%s', %d)",
+ sidecar.GetIdentifier(), table.String(), newTableSchema, time.Now().Unix()), &sqltypes.Result{})
+ db.AddQuery("rollback", &sqltypes.Result{})
+ },
+ expectFunc: func() {
+ tbl, err := se.GetTableForPos(ctx, table, "MySQL56/1497ddb0-7cb9-11ed-a1eb-0242ac120002:1-891")
+ require.NoError(t, err)
+ require.NotNil(t, tbl)
+ require.Equal(t, &binlogdatapb.MinimalTable{
+ Name: table.String(),
+ Fields: []*querypb.Field{
+ {
+ Name: column,
+ Type: sqltypes.VarChar,
+ },
+ {
+ Name: "col2",
+ Type: sqltypes.VarChar,
+ },
+ },
+ PKColumns: []int64{1}, // Second column: col2
+ }, tbl)
+ },
+ },
+ }
+
+ for _, tc := range tests {
+ t.Run(tc.name, func(t *testing.T) {
+ fakedb.DeleteAllQueries()
+ AddFakeInnoDBReadRowsResult(fakedb, int(rand.Int32N(1000000)))
+ tc.expectedQueriesFunc(fakedb)
+ se.tables = tc.initialCacheState
+ tc.expectFunc()
+ fakedb.VerifyAllExecutedOrFail()
+ require.NoError(t, fakedb.LastError())
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletserver/schema/load_table.go b/go/vt/vttablet/tabletserver/schema/load_table.go
index 6022f8724eb..2a2be33a208 100644
--- a/go/vt/vttablet/tabletserver/schema/load_table.go
+++ b/go/vt/vttablet/tabletserver/schema/load_table.go
@@ -157,6 +157,14 @@ func loadMessageInfo(ta *Table, comment string, collationEnv *collations.Environ
ta.MessageInfo.Fields = getDefaultMessageFields(ta.Fields, hiddenCols)
}
+ ta.MessageInfo.IDType = sqltypes.VarBinary
+ for _, field := range ta.MessageInfo.Fields {
+ if field.Name == "id" {
+ ta.MessageInfo.IDType = field.Type
+ break
+ }
+ }
+
return nil
}
diff --git a/go/vt/vttablet/tabletserver/schema/load_table_test.go b/go/vt/vttablet/tabletserver/schema/load_table_test.go
index 6416e2e306e..e451624c0d7 100644
--- a/go/vt/vttablet/tabletserver/schema/load_table_test.go
+++ b/go/vt/vttablet/tabletserver/schema/load_table_test.go
@@ -149,6 +149,7 @@ func TestLoadTableMessage(t *testing.T) {
BatchSize: 1,
CacheSize: 10,
PollInterval: 30 * time.Second,
+ IDType: sqltypes.Int64,
},
}
assert.Equal(t, want, table)
diff --git a/go/vt/vttablet/tabletserver/schema/main_test.go b/go/vt/vttablet/tabletserver/schema/main_test.go
index 7eaca5f18e5..8b705b9bfbd 100644
--- a/go/vt/vttablet/tabletserver/schema/main_test.go
+++ b/go/vt/vttablet/tabletserver/schema/main_test.go
@@ -34,11 +34,11 @@ func getTestSchemaEngine(t *testing.T, schemaMaxAgeSeconds int64) (*Engine, *fak
"int64"),
"1427325876",
))
- db.AddQueryPattern(baseShowTablesWithSizesPattern, &sqltypes.Result{})
+ db.AddQueryPattern(baseInnoDBTableSizesPattern, &sqltypes.Result{})
db.AddQuery(mysql.BaseShowTables, &sqltypes.Result{})
db.AddQuery(mysql.BaseShowPrimary, &sqltypes.Result{})
AddFakeInnoDBReadRowsResult(db, 1)
- se := newEngine(10*time.Second, 10*time.Second, schemaMaxAgeSeconds, db)
+ se := newEngine(10*time.Second, 10*time.Second, schemaMaxAgeSeconds, db, nil)
require.NoError(t, se.Open())
cancel := func() {
defer db.Close()
diff --git a/go/vt/vttablet/tabletserver/schema/schema.go b/go/vt/vttablet/tabletserver/schema/schema.go
index 4b3d9c88fb5..e800477da3b 100644
--- a/go/vt/vttablet/tabletserver/schema/schema.go
+++ b/go/vt/vttablet/tabletserver/schema/schema.go
@@ -20,6 +20,7 @@ import (
"sync"
"time"
+ "vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/sqlparser"
@@ -125,6 +126,9 @@ type MessageInfo struct {
// MaxBackoff specifies the longest duration message manager
// should wait before rescheduling a message
MaxBackoff time.Duration
+
+ // IDType specifies the type of the ID column
+ IDType sqltypes.Type
}
// NewTable creates a new Table.
diff --git a/go/vt/vttablet/tabletserver/schema/tracker.go b/go/vt/vttablet/tabletserver/schema/tracker.go
index 8db202efa13..252a81f3493 100644
--- a/go/vt/vttablet/tabletserver/schema/tracker.go
+++ b/go/vt/vttablet/tabletserver/schema/tracker.go
@@ -39,7 +39,8 @@ import (
// VStreamer defines the functions of VStreamer
// that the replicationWatcher needs.
type VStreamer interface {
- Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error
+ Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter,
+ throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error
}
// Tracker watches the replication and saves the latest schema into the schema_version table when a DDL is encountered.
@@ -144,7 +145,7 @@ func (tr *Tracker) process(ctx context.Context) {
}
}
return nil
- })
+ }, nil)
select {
case <-ctx.Done():
return
diff --git a/go/vt/vttablet/tabletserver/schema/tracker_test.go b/go/vt/vttablet/tabletserver/schema/tracker_test.go
index 32f68597779..0e7444b39b9 100644
--- a/go/vt/vttablet/tabletserver/schema/tracker_test.go
+++ b/go/vt/vttablet/tabletserver/schema/tracker_test.go
@@ -139,7 +139,8 @@ type fakeVstreamer struct {
events [][]*binlogdatapb.VEvent
}
-func (f *fakeVstreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error {
+func (f *fakeVstreamer) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error {
for _, events := range f.events {
err := send(events)
if err != nil {
diff --git a/go/vt/vttablet/tabletserver/state_manager.go b/go/vt/vttablet/tabletserver/state_manager.go
index e231a628748..6e03cd7ab62 100644
--- a/go/vt/vttablet/tabletserver/state_manager.go
+++ b/go/vt/vttablet/tabletserver/state_manager.go
@@ -140,7 +140,7 @@ type stateManager struct {
type (
schemaEngine interface {
- EnsureConnectionAndDB(topodatapb.TabletType) error
+ EnsureConnectionAndDB(topodatapb.TabletType, bool) error
Open() error
MakeNonPrimary()
MakePrimary(bool)
@@ -164,6 +164,7 @@ type (
AcceptReadWrite()
AcceptReadOnly()
Close()
+ RollbackPrepared()
}
subComponent interface {
@@ -448,7 +449,7 @@ func (sm *stateManager) verifyTargetLocked(ctx context.Context, target *querypb.
func (sm *stateManager) servePrimary() error {
sm.watcher.Close()
- if err := sm.connect(topodatapb.TabletType_PRIMARY); err != nil {
+ if err := sm.connect(topodatapb.TabletType_PRIMARY, true); err != nil {
return err
}
@@ -478,7 +479,7 @@ func (sm *stateManager) unservePrimary() error {
sm.watcher.Close()
- if err := sm.connect(topodatapb.TabletType_PRIMARY); err != nil {
+ if err := sm.connect(topodatapb.TabletType_PRIMARY, false); err != nil {
return err
}
@@ -504,7 +505,7 @@ func (sm *stateManager) serveNonPrimary(wantTabletType topodatapb.TabletType) er
sm.hs.MakeNonPrimary()
sm.txThrottler.MakeNonPrimary()
- if err := sm.connect(wantTabletType); err != nil {
+ if err := sm.connect(wantTabletType, true); err != nil {
return err
}
@@ -523,7 +524,7 @@ func (sm *stateManager) unserveNonPrimary(wantTabletType topodatapb.TabletType)
sm.hs.MakeNonPrimary()
sm.txThrottler.MakeNonPrimary()
- if err := sm.connect(wantTabletType); err != nil {
+ if err := sm.connect(wantTabletType, false); err != nil {
return err
}
@@ -533,8 +534,8 @@ func (sm *stateManager) unserveNonPrimary(wantTabletType topodatapb.TabletType)
return nil
}
-func (sm *stateManager) connect(tabletType topodatapb.TabletType) error {
- if err := sm.se.EnsureConnectionAndDB(tabletType); err != nil {
+func (sm *stateManager) connect(tabletType topodatapb.TabletType, serving bool) error {
+ if err := sm.se.EnsureConnectionAndDB(tabletType, serving); err != nil {
return err
}
if err := sm.se.Open(); err != nil {
@@ -616,6 +617,12 @@ func (sm *stateManager) terminateAllQueries(wg *sync.WaitGroup) (cancel func())
log.Infof("Killed all stateless OLTP queries.")
sm.statefulql.TerminateAll()
log.Infof("Killed all OLTP queries.")
+ // We can rollback prepared transactions only after we have killed all the write queries in progress.
+ // This is essential because when we rollback a prepared transaction, it lets go of the locks it was holding.
+ // If there were some other conflicting write in progress that hadn't been killed, then it could potentially go through
+ // and cause data corruption since we won't be able to prepare the transaction again.
+ sm.te.RollbackPrepared()
+ log.Infof("Rollbacked all prepared transactions")
}()
return cancel
}
diff --git a/go/vt/vttablet/tabletserver/state_manager_test.go b/go/vt/vttablet/tabletserver/state_manager_test.go
index 45257c995ad..d06bbde4aea 100644
--- a/go/vt/vttablet/tabletserver/state_manager_test.go
+++ b/go/vt/vttablet/tabletserver/state_manager_test.go
@@ -378,6 +378,9 @@ func (te *delayedTxEngine) Close() {
time.Sleep(50 * time.Millisecond)
}
+func (te *delayedTxEngine) RollbackPrepared() {
+}
+
type killableConn struct {
id int64
killed atomic.Bool
@@ -805,7 +808,7 @@ type testSchemaEngine struct {
failMySQL bool
}
-func (te *testSchemaEngine) EnsureConnectionAndDB(tabletType topodatapb.TabletType) error {
+func (te *testSchemaEngine) EnsureConnectionAndDB(topodatapb.TabletType, bool) error {
if te.failMySQL {
te.failMySQL = false
return errors.New("intentional error")
@@ -902,6 +905,8 @@ func (te *testTxEngine) Close() {
te.state = testStateClosed
}
+func (te *testTxEngine) RollbackPrepared() {}
+
type testSubcomponent struct {
testOrderState
}
diff --git a/go/vt/vttablet/tabletserver/stateful_connection.go b/go/vt/vttablet/tabletserver/stateful_connection.go
index 067f2194655..10fc763984f 100644
--- a/go/vt/vttablet/tabletserver/stateful_connection.go
+++ b/go/vt/vttablet/tabletserver/stateful_connection.go
@@ -150,7 +150,7 @@ func (sc *StatefulConnection) unlock(updateTime bool) {
return
}
if sc.dbConn.Conn.IsClosed() {
- sc.Releasef("unlocked closed connection")
+ sc.ReleaseString("unlocked closed connection")
} else {
sc.pool.markAsNotInUse(sc, updateTime)
}
@@ -159,16 +159,24 @@ func (sc *StatefulConnection) unlock(updateTime bool) {
// Release is used when the connection will not be used ever again.
// The underlying dbConn is removed so that this connection cannot be used by mistake.
func (sc *StatefulConnection) Release(reason tx.ReleaseReason) {
- sc.Releasef(reason.String())
+ sc.ReleaseString(reason.String())
}
// Releasef is used when the connection will not be used ever again.
// The underlying dbConn is removed so that this connection cannot be used by mistake.
func (sc *StatefulConnection) Releasef(reasonFormat string, a ...any) {
+ sc.ReleaseString(fmt.Sprintf(reasonFormat, a...))
+}
+
+// ReleaseString is used when the connection will not be used ever again.
+// The underlying dbConn is removed so that this connection cannot be used by mistake.
+func (sc *StatefulConnection) ReleaseString(reason string) {
if sc.dbConn == nil {
return
}
- sc.pool.unregister(sc.ConnID, fmt.Sprintf(reasonFormat, a...))
+ if sc.pool != nil {
+ sc.pool.unregister(sc.ConnID, reason)
+ }
sc.dbConn.Recycle()
sc.dbConn = nil
sc.logReservedConn()
@@ -264,7 +272,7 @@ func (sc *StatefulConnection) IsTainted() bool {
// LogTransaction logs transaction related stats
func (sc *StatefulConnection) LogTransaction(reason tx.ReleaseReason) {
if sc.txProps == nil {
- return //Nothing to log as no transaction exists on this connection.
+ return // Nothing to log as no transaction exists on this connection.
}
sc.txProps.Conclusion = reason.Name()
sc.txProps.EndTime = time.Now()
@@ -288,7 +296,7 @@ func (sc *StatefulConnection) SetTimeout(timeout time.Duration) {
// logReservedConn logs reserved connection related stats.
func (sc *StatefulConnection) logReservedConn() {
if sc.reservedProps == nil {
- return //Nothing to log as this connection is not reserved.
+ return // Nothing to log as this connection is not reserved.
}
duration := time.Since(sc.reservedProps.StartTime)
username := sc.getUsername()
@@ -305,13 +313,19 @@ func (sc *StatefulConnection) getUsername() string {
return callerid.GetUsername(sc.reservedProps.ImmediateCaller)
}
-func (sc *StatefulConnection) ApplySetting(ctx context.Context, setting *smartconnpool.Setting) error {
+// ApplySetting returns whether the settings where applied or not. It also returns an error, if encountered.
+func (sc *StatefulConnection) ApplySetting(ctx context.Context, setting *smartconnpool.Setting) (bool, error) {
if sc.dbConn.Conn.Setting() == setting {
- return nil
+ return false, nil
}
- return sc.dbConn.Conn.ApplySetting(ctx, setting)
+ return true, sc.dbConn.Conn.ApplySetting(ctx, setting)
}
func (sc *StatefulConnection) resetExpiryTime() {
sc.expiryTime = time.Now().Add(sc.timeout)
}
+
+// IsUnixSocket returns true if the connection is using a unix socket
+func (sc *StatefulConnection) IsUnixSocket() bool {
+ return sc.dbConn.Conn.IsUnixSocket()
+}
diff --git a/go/vt/vttablet/tabletserver/stateful_connection_pool.go b/go/vt/vttablet/tabletserver/stateful_connection_pool.go
index 64268825b70..88fbc56fd0c 100644
--- a/go/vt/vttablet/tabletserver/stateful_connection_pool.go
+++ b/go/vt/vttablet/tabletserver/stateful_connection_pool.go
@@ -96,7 +96,7 @@ func (sf *StatefulConnectionPool) Close() {
log.Warningf("killing %s for shutdown: %s", thing, conn.String(sf.env.Config().SanitizeLogMessages, sf.env.Environment().Parser()))
sf.env.Stats().InternalErrors.Add("StrayTransactions", 1)
conn.Close()
- conn.Releasef("pool closed")
+ conn.ReleaseString("pool closed")
}
sf.conns.Close()
sf.foundRowsPool.Close()
@@ -111,7 +111,7 @@ func (sf *StatefulConnectionPool) ShutdownNonTx() {
return !sc.(*StatefulConnection).IsInTransaction()
}))
for _, sc := range conns {
- sc.Releasef("kill non-tx")
+ sc.ReleaseString("kill non-tx")
}
}
@@ -231,14 +231,14 @@ func (sf *StatefulConnectionPool) markAsNotInUse(sc *StatefulConnection, updateT
switch sf.state.Load() {
case scpKillingNonTx:
if !sc.IsInTransaction() {
- sc.Releasef("kill non-tx")
+ sc.ReleaseString("kill non-tx")
return
}
case scpKillingAll:
if sc.IsInTransaction() {
sc.Close()
}
- sc.Releasef("kill all")
+ sc.ReleaseString("kill all")
return
}
if updateTime {
diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go
index 158f40d5202..994999f2368 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/config.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/config.go
@@ -156,8 +156,12 @@ func registerTabletEnvFlags(fs *pflag.FlagSet) {
fs.BoolVar(¤tConfig.WatchReplication, "watch_replication_stream", false, "When enabled, vttablet will stream the MySQL replication stream from the local server, and use it to update schema when it sees a DDL.")
fs.BoolVar(¤tConfig.TrackSchemaVersions, "track_schema_versions", false, "When enabled, vttablet will store versions of schemas at each position that a DDL is applied and allow retrieval of the schema corresponding to a position")
fs.Int64Var(¤tConfig.SchemaVersionMaxAgeSeconds, "schema-version-max-age-seconds", 0, "max age of schema version records to kept in memory by the vreplication historian")
- fs.BoolVar(¤tConfig.TwoPCEnable, "twopc_enable", defaultConfig.TwoPCEnable, "if the flag is on, 2pc is enabled. Other 2pc flags must be supplied.")
- SecondsVar(fs, ¤tConfig.TwoPCAbandonAge, "twopc_abandon_age", defaultConfig.TwoPCAbandonAge, "time in seconds. Any unresolved transaction older than this time will be sent to the coordinator to be resolved.")
+
+ _ = fs.Bool("twopc_enable", true, "TwoPC is enabled")
+ _ = fs.MarkDeprecated("twopc_enable", "TwoPC is always enabled, the transaction abandon age can be configured")
+ flagutil.FloatDuration(fs, ¤tConfig.TwoPCAbandonAge, "twopc_abandon_age", defaultConfig.TwoPCAbandonAge,
+ "Any unresolved transaction older than this time will be sent to the coordinator to be resolved. NOTE: Providing time as seconds (float64) is deprecated. Use time.Duration format (e.g., '1s', '2m', '1h').")
+
// Tx throttler config
flagutil.DualFormatBoolVar(fs, ¤tConfig.EnableTxThrottler, "enable_tx_throttler", defaultConfig.EnableTxThrottler, "If true replication-lag-based throttling on transactions will be enabled.")
flagutil.DualFormatVar(fs, currentConfig.TxThrottlerConfig, "tx_throttler_config", "The configuration of the transaction throttler as a text-formatted throttlerdata.Configuration protocol buffer message.")
@@ -331,12 +335,11 @@ type TabletConfig struct {
ExternalConnections map[string]*dbconfigs.DBConfigs `json:"externalConnections,omitempty"`
- SanitizeLogMessages bool `json:"-"`
- StrictTableACL bool `json:"-"`
- EnableTableACLDryRun bool `json:"-"`
- TableACLExemptACL string `json:"-"`
- TwoPCEnable bool `json:"-"`
- TwoPCAbandonAge Seconds `json:"-"`
+ SanitizeLogMessages bool `json:"-"`
+ StrictTableACL bool `json:"-"`
+ EnableTableACLDryRun bool `json:"-"`
+ TableACLExemptACL string `json:"-"`
+ TwoPCAbandonAge time.Duration `json:"-"`
EnableTxThrottler bool `json:"-"`
TxThrottlerConfig *TxThrottlerConfigFlag `json:"-"`
@@ -1054,6 +1057,8 @@ var defaultConfig = TabletConfig{
},
EnablePerWorkloadTableMetrics: false,
+
+ TwoPCAbandonAge: 15 * time.Minute,
}
// defaultTxThrottlerConfig returns the default TxThrottlerConfigFlag object based on
diff --git a/go/vt/vttablet/tabletserver/tabletenv/seconds.go b/go/vt/vttablet/tabletserver/tabletenv/seconds.go
deleted file mode 100644
index ae11121f2de..00000000000
--- a/go/vt/vttablet/tabletserver/tabletenv/seconds.go
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
-Copyright 2020 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 tabletenv
-
-import (
- "time"
-
- "github.com/spf13/pflag"
-)
-
-// Seconds provides convenience functions for extracting
-// duration from float64 seconds values.
-type Seconds float64
-
-// SecondsVar is like a flag.Float64Var, but it works for Seconds.
-func SecondsVar(fs *pflag.FlagSet, p *Seconds, name string, value Seconds, usage string) {
- fs.Float64Var((*float64)(p), name, float64(value), usage)
-}
-
-// Get converts Seconds to time.Duration
-func (s Seconds) Get() time.Duration {
- return time.Duration(s * Seconds(1*time.Second))
-}
-
-// Set sets the value from time.Duration
-func (s *Seconds) Set(d time.Duration) {
- *s = Seconds(d) / Seconds(1*time.Second)
-}
diff --git a/go/vt/vttablet/tabletserver/tabletenv/stats.go b/go/vt/vttablet/tabletserver/tabletenv/stats.go
index 1ad93532719..52bb6a5a3b0 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/stats.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/stats.go
@@ -34,7 +34,6 @@ type Stats struct {
ErrorCounters *stats.CountersWithSingleLabel
InternalErrors *stats.CountersWithSingleLabel
Warnings *stats.CountersWithSingleLabel
- Unresolved *stats.GaugesWithSingleLabel // For now, only Prepares are tracked
UserTableQueryCount *stats.CountersWithMultiLabels // Per CallerID/table counts
UserTableQueryTimesNs *stats.CountersWithMultiLabels // Per CallerID/table latencies
UserTransactionCount *stats.CountersWithMultiLabels // Per CallerID transaction counts
@@ -49,6 +48,11 @@ type Stats struct {
UserReservedTimesNs *stats.CountersWithSingleLabel // Per CallerID reserved connection duration
QueryTimingsByTabletType *servenv.TimingsWrapper // Query timings split by current tablet type
+
+ // Atomic Transactions
+ Unresolved *stats.GaugesWithSingleLabel
+ CommitPreparedFail *stats.CountersWithSingleLabel
+ RedoPreparedFail *stats.CountersWithSingleLabel
}
// NewStats instantiates a new set of stats scoped by exporter.
@@ -83,7 +87,6 @@ func NewStats(exporter *servenv.Exporter) *Stats {
),
InternalErrors: exporter.NewCountersWithSingleLabel("InternalErrors", "Internal component errors", "type", "Task", "StrayTransactions", "Panic", "HungQuery", "Schema", "TwopcCommit", "TwopcResurrection", "WatchdogFail", "Messages"),
Warnings: exporter.NewCountersWithSingleLabel("Warnings", "Warnings", "type", "ResultsExceeded"),
- Unresolved: exporter.NewGaugesWithSingleLabel("Unresolved", "Unresolved items", "item_type", "Prepares"),
UserTableQueryCount: exporter.NewCountersWithMultiLabels("UserTableQueryCount", "Queries received for each CallerID/table combination", []string{"TableName", "CallerID", "Type"}),
UserTableQueryTimesNs: exporter.NewCountersWithMultiLabels("UserTableQueryTimesNs", "Total latency for each CallerID/table combination", []string{"TableName", "CallerID", "Type"}),
UserTransactionCount: exporter.NewCountersWithMultiLabels("UserTransactionCount", "transactions received for each CallerID", []string{"CallerID", "Conclusion"}),
@@ -98,6 +101,10 @@ func NewStats(exporter *servenv.Exporter) *Stats {
UserReservedTimesNs: exporter.NewCountersWithSingleLabel("UserReservedTimesNs", "Total reserved connection latency for each CallerID", "CallerID"),
QueryTimingsByTabletType: exporter.NewTimings("QueryTimingsByTabletType", "Query timings broken down by active tablet type", "TabletType"),
+
+ Unresolved: exporter.NewGaugesWithSingleLabel("UnresolvedTransaction", "Current unresolved transactions", "ManagerType"),
+ CommitPreparedFail: exporter.NewCountersWithSingleLabel("CommitPreparedFail", "failed prepared transactions commit", "FailureType"),
+ RedoPreparedFail: exporter.NewCountersWithSingleLabel("RedoPreparedFail", "failed prepared transactions on redo", "FailureType"),
}
stats.QPSRates = exporter.NewRates("QPS", stats.QueryTimings, 15*60/5, 5*time.Second)
return stats
diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go
index e3e951892b7..847de25eb02 100644
--- a/go/vt/vttablet/tabletserver/tabletserver.go
+++ b/go/vt/vttablet/tabletserver/tabletserver.go
@@ -176,7 +176,7 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c
tsv.se = schema.NewEngine(tsv)
tsv.hs = newHealthStreamer(tsv, alias, tsv.se)
tsv.rt = repltracker.NewReplTracker(tsv, alias)
- tsv.lagThrottler = throttle.NewThrottler(tsv, srvTopoServer, topoServer, alias.Cell, tsv.rt.HeartbeatWriter(), tabletTypeFunc)
+ tsv.lagThrottler = throttle.NewThrottler(tsv, srvTopoServer, topoServer, alias, tsv.rt.HeartbeatWriter(), tabletTypeFunc)
tsv.vstreamer = vstreamer.NewEngine(tsv, srvTopoServer, tsv.se, tsv.lagThrottler, alias.Cell)
tsv.tracker = schema.NewTracker(tsv, tsv.vstreamer, tsv.se)
tsv.watcher = NewBinlogWatcher(tsv, tsv.vstreamer, tsv.config)
@@ -186,7 +186,7 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c
tsv.messager = messager.NewEngine(tsv, tsv.se, tsv.vstreamer)
tsv.tableGC = gc.NewTableGC(tsv, topoServer, tsv.lagThrottler)
- tsv.onlineDDLExecutor = onlineddl.NewExecutor(tsv, alias, topoServer, tsv.lagThrottler, tabletTypeFunc, tsv.onlineDDLExecutorToggleTableBuffer, tsv.tableGC.RequestChecks)
+ tsv.onlineDDLExecutor = onlineddl.NewExecutor(tsv, alias, topoServer, tsv.lagThrottler, tabletTypeFunc, tsv.onlineDDLExecutorToggleTableBuffer, tsv.tableGC.RequestChecks, tsv.te.preparedPool.IsEmptyForTable)
tsv.sm = &stateManager{
statelessql: tsv.statelessql,
@@ -217,7 +217,9 @@ func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, c
tsv.exporter.NewGaugesFuncWithMultiLabels("TabletServerState", "Tablet server state labeled by state name", []string{"name"}, func() map[string]int64 {
return map[string]int64{tsv.sm.IsServingString(): 1}
})
- tsv.exporter.NewGaugeDurationFunc("QueryTimeout", "Tablet server query timeout", tsv.loadQueryTimeout)
+ tsv.exporter.NewGaugeDurationFunc("QueryTimeout", "Tablet server query timeout", func() time.Duration {
+ return time.Duration(tsv.QueryTimeout.Load())
+ })
tsv.registerHealthzHealthHandler()
tsv.registerDebugHealthHandler()
@@ -237,6 +239,28 @@ func (tsv *TabletServer) loadQueryTimeout() time.Duration {
return time.Duration(tsv.QueryTimeout.Load())
}
+func (tsv *TabletServer) loadQueryTimeoutWithTxAndOptions(txID int64, options *querypb.ExecuteOptions) time.Duration {
+ timeout := tsv.loadQueryTimeoutWithOptions(options)
+
+ if txID == 0 {
+ return timeout
+ }
+
+ // fetch the transaction timeout.
+ txTimeout := tsv.config.TxTimeoutForWorkload(querypb.ExecuteOptions_OLTP)
+
+ // Use the smaller of the two values (0 means infinity).
+ return smallerTimeout(timeout, txTimeout)
+}
+
+func (tsv *TabletServer) loadQueryTimeoutWithOptions(options *querypb.ExecuteOptions) time.Duration {
+ // returns the authoritative timeout if it is set.
+ if options != nil && options.Timeout != nil {
+ return time.Duration(options.GetAuthoritativeTimeout()) * time.Millisecond
+ }
+ return time.Duration(tsv.QueryTimeout.Load())
+}
+
// onlineDDLExecutorToggleTableBuffer is called by onlineDDLExecutor as a callback function. onlineDDLExecutor
// uses it to start/stop query buffering for a given table.
// It is onlineDDLExecutor's responsibility to make sure buffering is stopped after some definite amount of time.
@@ -476,11 +500,6 @@ func (tsv *TabletServer) TableGC() *gc.TableGC {
return tsv.tableGC
}
-// TwoPCEngineWait waits until the TwoPC engine has been opened, and the redo read
-func (tsv *TabletServer) TwoPCEngineWait() {
- tsv.te.twoPCReady.Wait()
-}
-
// SchemaEngine returns the SchemaEngine part of TabletServer.
func (tsv *TabletServer) SchemaEngine() *schema.Engine {
return tsv.se
@@ -491,10 +510,17 @@ func (tsv *TabletServer) Begin(ctx context.Context, target *querypb.Target, opti
return tsv.begin(ctx, target, nil, 0, nil, options)
}
-func (tsv *TabletServer) begin(ctx context.Context, target *querypb.Target, savepointQueries []string, reservedID int64, settings []string, options *querypb.ExecuteOptions) (state queryservice.TransactionState, err error) {
+func (tsv *TabletServer) begin(
+ ctx context.Context,
+ target *querypb.Target,
+ postBeginQueries []string,
+ reservedID int64,
+ settings []string,
+ options *querypb.ExecuteOptions,
+) (state queryservice.TransactionState, err error) {
state.TabletAlias = tsv.alias
err = tsv.execRequest(
- ctx, tsv.loadQueryTimeout(),
+ ctx, tsv.loadQueryTimeoutWithOptions(options),
"Begin", "begin", nil,
target, options, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
@@ -509,12 +535,43 @@ func (tsv *TabletServer) begin(ctx context.Context, target *querypb.Target, save
return err
}
}
- transactionID, beginSQL, sessionStateChanges, err := tsv.te.Begin(ctx, savepointQueries, reservedID, connSetting, options)
+ transactionID, beginSQL, sessionStateChanges, err := tsv.te.Begin(ctx, reservedID, connSetting, options)
state.TransactionID = transactionID
state.SessionStateChanges = sessionStateChanges
logStats.TransactionID = transactionID
logStats.ReservedID = reservedID
+ if err != nil {
+ return err
+ }
+
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ for _, query := range postBeginQueries {
+ plan, err := tsv.qe.GetPlan(ctx, logStats, query, true)
+ if err != nil {
+ return err
+ }
+
+ qre := &QueryExecutor{
+ ctx: ctx,
+ query: query,
+ connID: transactionID,
+ options: options,
+ plan: plan,
+ logStats: logStats,
+ tsv: tsv,
+ targetTabletType: targetType,
+ setting: connSetting,
+ }
+ _, err = qre.Execute()
+ if err != nil {
+ return err
+ }
+ }
+
// Record the actual statements that were executed in the logStats.
// If nothing was actually executed, don't count the operation in
// the tablet metrics, and clear out the logStats Method so that
@@ -533,7 +590,7 @@ func (tsv *TabletServer) begin(ctx context.Context, target *querypb.Target, save
return err
},
)
- return state, err
+ return
}
func (tsv *TabletServer) getPriorityFromOptions(options *querypb.ExecuteOptions) int {
@@ -646,7 +703,7 @@ func (tsv *TabletServer) Prepare(ctx context.Context, target *querypb.Target, tr
"Prepare", "prepare", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
return txe.Prepare(transactionID, dtid)
},
)
@@ -659,7 +716,7 @@ func (tsv *TabletServer) CommitPrepared(ctx context.Context, target *querypb.Tar
"CommitPrepared", "commit_prepared", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
if DebugTwoPc {
commitPreparedDelayForTest(tsv)
}
@@ -675,12 +732,32 @@ func (tsv *TabletServer) RollbackPrepared(ctx context.Context, target *querypb.T
"RollbackPrepared", "rollback_prepared", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
return txe.RollbackPrepared(dtid, originalID)
},
)
}
+// WaitForPreparedTwoPCTransactions waits for all the prepared transactions to complete.
+func (tsv *TabletServer) WaitForPreparedTwoPCTransactions(ctx context.Context) error {
+ if tsv.te.preparedPool.IsEmpty() {
+ return nil
+ }
+ ticker := time.NewTicker(100 * time.Millisecond)
+ defer ticker.Stop()
+ for {
+ select {
+ case <-ctx.Done():
+ // Return an error if we run out of time.
+ return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "Prepared transactions have not been resolved yet")
+ case <-ticker.C:
+ if tsv.te.preparedPool.IsEmpty() {
+ return nil
+ }
+ }
+ }
+}
+
// CreateTransaction creates the metadata for a 2PC transaction.
func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.Target, dtid string, participants []*querypb.Target) (err error) {
return tsv.execRequest(
@@ -688,7 +765,7 @@ func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.
"CreateTransaction", "create_transaction", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
return txe.CreateTransaction(dtid, participants)
},
)
@@ -696,16 +773,18 @@ func (tsv *TabletServer) CreateTransaction(ctx context.Context, target *querypb.
// StartCommit atomically commits the transaction along with the
// decision to commit the associated 2pc transaction.
-func (tsv *TabletServer) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (err error) {
- return tsv.execRequest(
+func (tsv *TabletServer) StartCommit(ctx context.Context, target *querypb.Target, transactionID int64, dtid string) (state querypb.StartCommitState, err error) {
+ err = tsv.execRequest(
ctx, tsv.loadQueryTimeout(),
"StartCommit", "start_commit", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
- return txe.StartCommit(transactionID, dtid)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
+ state, err = txe.StartCommit(transactionID, dtid)
+ return err
},
)
+ return state, err
}
// SetRollback transitions the 2pc transaction to the Rollback state.
@@ -716,7 +795,7 @@ func (tsv *TabletServer) SetRollback(ctx context.Context, target *querypb.Target
"SetRollback", "set_rollback", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
return txe.SetRollback(dtid, transactionID)
},
)
@@ -730,7 +809,7 @@ func (tsv *TabletServer) ConcludeTransaction(ctx context.Context, target *queryp
"ConcludeTransaction", "conclude_transaction", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
return txe.ConcludeTransaction(dtid)
},
)
@@ -743,7 +822,7 @@ func (tsv *TabletServer) ReadTransaction(ctx context.Context, target *querypb.Ta
"ReadTransaction", "read_transaction", nil,
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
metadata, err = txe.ReadTransaction(dtid)
return err
},
@@ -751,15 +830,30 @@ func (tsv *TabletServer) ReadTransaction(ctx context.Context, target *querypb.Ta
return metadata, err
}
+// GetTransactionInfo returns the data for the specified dtid.
+func (tsv *TabletServer) GetTransactionInfo(ctx context.Context, target *querypb.Target, dtid string) (resp *tabletmanagerdatapb.GetTransactionInfoResponse, err error) {
+ err = tsv.execRequest(
+ ctx, tsv.loadQueryTimeout(),
+ "GetTransactionInfo", "get_transaction_info", nil,
+ target, nil, true, /* allowOnShutdown */
+ func(ctx context.Context, logStats *tabletenv.LogStats) error {
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
+ resp, err = txe.GetTransactionInfo(dtid)
+ return err
+ },
+ )
+ return resp, err
+}
+
// UnresolvedTransactions returns the unresolved distributed transaction record.
-func (tsv *TabletServer) UnresolvedTransactions(ctx context.Context, target *querypb.Target) (transactions []*querypb.TransactionMetadata, err error) {
+func (tsv *TabletServer) UnresolvedTransactions(ctx context.Context, target *querypb.Target, abandonAgeSeconds int64) (transactions []*querypb.TransactionMetadata, err error) {
err = tsv.execRequest(
ctx, tsv.loadQueryTimeout(),
"UnresolvedTransactions", "unresolved_transaction", nil,
target, nil, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
- txe := NewDTExecutor(ctx, tsv.te, logStats)
- transactions, err = txe.UnresolvedTransactions()
+ txe := NewDTExecutor(ctx, logStats, tsv.te, tsv.qe, tsv.getShard)
+ transactions, err = txe.UnresolvedTransactions(time.Duration(abandonAgeSeconds) * time.Second)
return err
},
)
@@ -780,17 +874,8 @@ func (tsv *TabletServer) Execute(ctx context.Context, target *querypb.Target, sq
}
func (tsv *TabletServer) execute(ctx context.Context, target *querypb.Target, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, reservedID int64, settings []string, options *querypb.ExecuteOptions) (result *sqltypes.Result, err error) {
- allowOnShutdown := false
- timeout := tsv.loadQueryTimeout()
- if transactionID != 0 {
- allowOnShutdown = true
- // Execute calls happen for OLTP only, so we can directly fetch the
- // OLTP TX timeout.
- txTimeout := tsv.config.TxTimeoutForWorkload(querypb.ExecuteOptions_OLTP)
- // Use the smaller of the two values (0 means infinity).
- // TODO(sougou): Assign deadlines to each transaction and set query timeout accordingly.
- timeout = smallerTimeout(timeout, txTimeout)
- }
+ allowOnShutdown := transactionID != 0
+ timeout := tsv.loadQueryTimeoutWithTxAndOptions(transactionID, options)
err = tsv.execRequest(
ctx, timeout,
"Execute", sql, bindVariables,
@@ -948,7 +1033,7 @@ func (tsv *TabletServer) streamExecute(ctx context.Context, target *querypb.Targ
}
// BeginExecute combines Begin and Execute.
-func (tsv *TabletServer) BeginExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, reservedID int64, options *querypb.ExecuteOptions) (queryservice.TransactionState, *sqltypes.Result, error) {
+func (tsv *TabletServer) BeginExecute(ctx context.Context, target *querypb.Target, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, reservedID int64, options *querypb.ExecuteOptions) (queryservice.TransactionState, *sqltypes.Result, error) {
// Disable hot row protection in case of reserve connection.
if tsv.enableHotRowProtection && reservedID == 0 {
@@ -961,7 +1046,7 @@ func (tsv *TabletServer) BeginExecute(ctx context.Context, target *querypb.Targe
}
}
- state, err := tsv.begin(ctx, target, preQueries, reservedID, nil, options)
+ state, err := tsv.begin(ctx, target, postBeginQueries, reservedID, nil, options)
if err != nil {
return state, nil, err
}
@@ -974,14 +1059,14 @@ func (tsv *TabletServer) BeginExecute(ctx context.Context, target *querypb.Targe
func (tsv *TabletServer) BeginStreamExecute(
ctx context.Context,
target *querypb.Target,
- preQueries []string,
+ postBeginQueries []string,
sql string,
bindVariables map[string]*querypb.BindVariable,
reservedID int64,
options *querypb.ExecuteOptions,
callback func(*sqltypes.Result) error,
) (queryservice.TransactionState, error) {
- state, err := tsv.begin(ctx, target, preQueries, reservedID, nil, options)
+ state, err := tsv.begin(ctx, target, postBeginQueries, reservedID, nil, options)
if err != nil {
return state, err
}
@@ -1007,7 +1092,7 @@ func (tsv *TabletServer) beginWaitForSameRangeTransactions(ctx context.Context,
err := tsv.execRequest(
// Use (potentially longer) -queryserver-config-query-timeout and not
// -queryserver-config-txpool-timeout (defaults to 1s) to limit the waiting.
- ctx, tsv.loadQueryTimeout(),
+ ctx, tsv.loadQueryTimeoutWithOptions(options),
"", "waitForSameRangeTransactions", nil,
target, options, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
@@ -1171,7 +1256,7 @@ func (tsv *TabletServer) VStream(ctx context.Context, request *binlogdatapb.VStr
if err := tsv.sm.VerifyTarget(ctx, request.Target); err != nil {
return err
}
- return tsv.vstreamer.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send)
+ return tsv.vstreamer.Stream(ctx, request.Position, request.TableLastPKs, request.Filter, throttlerapp.VStreamerName, send, request.Options)
}
// VStreamRows streams rows from the specified starting point.
@@ -1187,7 +1272,7 @@ func (tsv *TabletServer) VStreamRows(ctx context.Context, request *binlogdatapb.
}
row = r.Rows[0]
}
- return tsv.vstreamer.StreamRows(ctx, request.Query, row, send)
+ return tsv.vstreamer.StreamRows(ctx, request.Query, row, send, request.Options)
}
// VStreamTables streams all tables.
@@ -1195,7 +1280,7 @@ func (tsv *TabletServer) VStreamTables(ctx context.Context, request *binlogdatap
if err := tsv.sm.VerifyTarget(ctx, request.Target); err != nil {
return err
}
- return tsv.vstreamer.StreamTables(ctx, send)
+ return tsv.vstreamer.StreamTables(ctx, send, request.Options)
}
// VStreamResults streams rows from the specified starting point.
@@ -1207,8 +1292,8 @@ func (tsv *TabletServer) VStreamResults(ctx context.Context, target *querypb.Tar
}
// ReserveBeginExecute implements the QueryService interface
-func (tsv *TabletServer) ReserveBeginExecute(ctx context.Context, target *querypb.Target, preQueries []string, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions) (state queryservice.ReservedTransactionState, result *sqltypes.Result, err error) {
- state, result, err = tsv.beginExecuteWithSettings(ctx, target, preQueries, postBeginQueries, sql, bindVariables, options)
+func (tsv *TabletServer) ReserveBeginExecute(ctx context.Context, target *querypb.Target, settings []string, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions) (state queryservice.ReservedTransactionState, result *sqltypes.Result, err error) {
+ state, result, err = tsv.beginExecuteWithSettings(ctx, target, settings, postBeginQueries, sql, bindVariables, options)
// If there is an error and the error message is about allowing query in reserved connection only,
// then we do not return an error from here and continue to use the reserved connection path.
// This is specially for get_lock function call from vtgate that needs a reserved connection.
@@ -1226,7 +1311,7 @@ func (tsv *TabletServer) ReserveBeginExecute(ctx context.Context, target *queryp
state.TabletAlias = tsv.alias
err = tsv.execRequest(
- ctx, tsv.loadQueryTimeout(),
+ ctx, tsv.loadQueryTimeoutWithOptions(options),
"ReserveBegin", "begin", bindVariables,
target, options, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
@@ -1236,12 +1321,35 @@ func (tsv *TabletServer) ReserveBeginExecute(ctx context.Context, target *queryp
return err
}
defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
- connID, sessionStateChanges, err = tsv.te.ReserveBegin(ctx, options, preQueries, postBeginQueries)
+ connID, sessionStateChanges, err = tsv.te.ReserveBegin(ctx, options, settings)
+ logStats.TransactionID = connID
+ logStats.ReservedID = connID
if err != nil {
return err
}
- logStats.TransactionID = connID
- logStats.ReservedID = connID
+
+ for _, query := range postBeginQueries {
+ plan, err := tsv.qe.GetPlan(ctx, logStats, query, true)
+ if err != nil {
+ return err
+ }
+
+ qre := &QueryExecutor{
+ ctx: ctx,
+ query: query,
+ connID: connID,
+ options: options,
+ plan: plan,
+ logStats: logStats,
+ tsv: tsv,
+ targetTabletType: targetType,
+ }
+ _, err = qre.Execute()
+ if err != nil {
+ return err
+ }
+ }
+
return nil
},
)
@@ -1262,13 +1370,13 @@ func (tsv *TabletServer) ReserveBeginStreamExecute(
ctx context.Context,
target *querypb.Target,
settings []string,
- savepointQueries []string,
+ postBeginQueries []string,
sql string,
bindVariables map[string]*querypb.BindVariable,
options *querypb.ExecuteOptions,
callback func(*sqltypes.Result) error,
) (state queryservice.ReservedTransactionState, err error) {
- txState, err := tsv.begin(ctx, target, savepointQueries, 0, settings, options)
+ txState, err := tsv.begin(ctx, target, postBeginQueries, 0, settings, options)
if err != nil {
return txToReserveState(txState), err
}
@@ -1278,9 +1386,9 @@ func (tsv *TabletServer) ReserveBeginStreamExecute(
}
// ReserveExecute implements the QueryService interface
-func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Target, preQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, options *querypb.ExecuteOptions) (state queryservice.ReservedState, result *sqltypes.Result, err error) {
+func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Target, settings []string, sql string, bindVariables map[string]*querypb.BindVariable, transactionID int64, options *querypb.ExecuteOptions) (state queryservice.ReservedState, result *sqltypes.Result, err error) {
- result, err = tsv.executeWithSettings(ctx, target, preQueries, sql, bindVariables, transactionID, options)
+ result, err = tsv.executeWithSettings(ctx, target, settings, sql, bindVariables, transactionID, options)
// If there is an error and the error message is about allowing query in reserved connection only,
// then we do not return an error from here and continue to use the reserved connection path.
// This is specially for get_lock function call from vtgate that needs a reserved connection.
@@ -1291,16 +1399,8 @@ func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Tar
// needs a reserved connection to execute the query.
state.TabletAlias = tsv.alias
- allowOnShutdown := false
- timeout := tsv.loadQueryTimeout()
- if transactionID != 0 {
- allowOnShutdown = true
- // ReserveExecute is for OLTP only, so we can directly fetch the OLTP
- // TX timeout.
- txTimeout := tsv.config.TxTimeoutForWorkload(querypb.ExecuteOptions_OLTP)
- // Use the smaller of the two values (0 means infinity).
- timeout = smallerTimeout(timeout, txTimeout)
- }
+ allowOnShutdown := transactionID != 0
+ timeout := tsv.loadQueryTimeoutWithTxAndOptions(transactionID, options)
err = tsv.execRequest(
ctx, timeout,
@@ -1313,7 +1413,7 @@ func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Tar
return err
}
defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
- state.ReservedID, err = tsv.te.Reserve(ctx, options, transactionID, preQueries)
+ state.ReservedID, err = tsv.te.Reserve(ctx, options, transactionID, settings)
if err != nil {
return err
}
@@ -1335,14 +1435,14 @@ func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Tar
func (tsv *TabletServer) ReserveStreamExecute(
ctx context.Context,
target *querypb.Target,
- preQueries []string,
+ settings []string,
sql string,
bindVariables map[string]*querypb.BindVariable,
transactionID int64,
options *querypb.ExecuteOptions,
callback func(*sqltypes.Result) error,
) (state queryservice.ReservedState, err error) {
- return state, tsv.streamExecute(ctx, target, sql, bindVariables, transactionID, 0, preQueries, options, callback)
+ return state, tsv.streamExecute(ctx, target, sql, bindVariables, transactionID, 0, settings, options, callback)
}
// Release implements the QueryService interface
@@ -1382,8 +1482,8 @@ func (tsv *TabletServer) executeWithSettings(ctx context.Context, target *queryp
return tsv.execute(ctx, target, sql, bindVariables, transactionID, 0, settings, options)
}
-func (tsv *TabletServer) beginExecuteWithSettings(ctx context.Context, target *querypb.Target, settings []string, savepointQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions) (queryservice.ReservedTransactionState, *sqltypes.Result, error) {
- txState, err := tsv.begin(ctx, target, savepointQueries, 0, settings, options)
+func (tsv *TabletServer) beginExecuteWithSettings(ctx context.Context, target *querypb.Target, settings []string, postBeginQueries []string, sql string, bindVariables map[string]*querypb.BindVariable, options *querypb.ExecuteOptions) (queryservice.ReservedTransactionState, *sqltypes.Result, error) {
+ txState, err := tsv.begin(ctx, target, postBeginQueries, 0, settings, options)
if err != nil {
return txToReserveState(txState), nil, err
}
@@ -1598,7 +1698,7 @@ func convertErrorCode(err error) vtrpcpb.Code {
sqlerror.ERTooLongString, sqlerror.ERDelayedInsertTableLocked, sqlerror.ERDupUnique, sqlerror.ERRequiresPrimaryKey, sqlerror.ERCantDoThisDuringAnTransaction, sqlerror.ERReadOnlyTransaction,
sqlerror.ERCannotAddForeign, sqlerror.ERNoReferencedRow, sqlerror.ERRowIsReferenced, sqlerror.ERCantUpdateWithReadLock, sqlerror.ERNoDefault, sqlerror.EROperandColumns,
sqlerror.ERSubqueryNo1Row, sqlerror.ERNonUpdateableTable, sqlerror.ERFeatureDisabled, sqlerror.ERDuplicatedValueInType, sqlerror.ERRowIsReferenced2,
- sqlerror.ErNoReferencedRow2, sqlerror.ERWarnDataOutOfRange:
+ sqlerror.ErNoReferencedRow2, sqlerror.ERWarnDataOutOfRange, sqlerror.ERInnodbIndexCorrupt:
errCode = vtrpcpb.Code_FAILED_PRECONDITION
case sqlerror.EROptionPreventsStatement:
errCode = vtrpcpb.Code_CLUSTER_EVENT
@@ -1623,7 +1723,7 @@ func convertErrorCode(err error) vtrpcpb.Code {
sqlerror.ERCantAggregate3Collations, sqlerror.ERCantAggregateNCollations, sqlerror.ERVariableIsNotStruct, sqlerror.ERUnknownCollation, sqlerror.ERWrongNameForIndex,
sqlerror.ERWrongNameForCatalog, sqlerror.ERBadFTColumn, sqlerror.ERTruncatedWrongValue, sqlerror.ERTooMuchAutoTimestampCols, sqlerror.ERInvalidOnUpdate, sqlerror.ERUnknownTimeZone,
sqlerror.ERInvalidCharacterString, sqlerror.ERIllegalReference, sqlerror.ERDerivedMustHaveAlias, sqlerror.ERTableNameNotAllowedHere, sqlerror.ERDataTooLong, sqlerror.ERDataOutOfRange,
- sqlerror.ERTruncatedWrongValueForField, sqlerror.ERIllegalValueForType:
+ sqlerror.ERTruncatedWrongValueForField, sqlerror.ERIllegalValueForType, sqlerror.ERWrongValue, sqlerror.ERWrongParamcountToNativeFct:
errCode = vtrpcpb.Code_INVALID_ARGUMENT
case sqlerror.ERSpecifiedAccessDenied:
errCode = vtrpcpb.Code_PERMISSION_DENIED
@@ -1692,6 +1792,17 @@ func (tsv *TabletServer) GetThrottlerStatus(ctx context.Context) *throttle.Throt
return r
}
+// RedoPreparedTransactions redoes the prepared transactions.
+func (tsv *TabletServer) RedoPreparedTransactions() {
+ tsv.te.RedoPreparedTransactions()
+}
+
+// SetTwoPCAllowed sets whether TwoPC is allowed or not. It also takes the reason of why it is being set.
+// The reason should be an enum value defined in the tabletserver.
+func (tsv *TabletServer) SetTwoPCAllowed(reason int, allowed bool) {
+ tsv.te.twopcAllowed[reason] = allowed
+}
+
// HandlePanic is part of the queryservice.QueryService interface
func (tsv *TabletServer) HandlePanic(err *error) {
if x := recover(); x != nil {
@@ -1771,7 +1882,7 @@ func (tsv *TabletServer) registerQueryListHandlers(queryLists []*QueryList) {
func (tsv *TabletServer) registerTwopczHandler() {
tsv.exporter.HandleFunc("/twopcz", func(w http.ResponseWriter, r *http.Request) {
ctx := tabletenv.LocalContext()
- txe := NewDTExecutor(ctx, tsv.te, tabletenv.NewLogStats(ctx, "twopcz"))
+ txe := NewDTExecutor(ctx, tabletenv.NewLogStats(ctx, "twopcz"), tsv.te, tsv.qe, tsv.getShard)
twopczHandler(txe, w, r)
})
}
@@ -1930,7 +2041,11 @@ func (tsv *TabletServer) StreamPoolSize() int {
// SetTxPoolSize changes the tx pool size to the specified value.
func (tsv *TabletServer) SetTxPoolSize(ctx context.Context, val int) error {
- return tsv.te.txPool.scp.conns.SetCapacity(ctx, int64(val))
+ // TxPool manages two pools, one for normal connections and one for CLIENT_FOUND_ROWS capability enabled on the connections.
+ if err := tsv.te.txPool.scp.conns.SetCapacity(ctx, int64(val)); err != nil {
+ return err
+ }
+ return tsv.te.txPool.scp.foundRowsPool.SetCapacity(ctx, int64(val))
}
// TxPoolSize returns the tx pool size.
@@ -2060,3 +2175,7 @@ func skipQueryPlanCache(options *querypb.ExecuteOptions) bool {
}
return options.SkipQueryPlanCache || options.HasCreatedTempTables
}
+
+func (tsv *TabletServer) getShard() string {
+ return tsv.sm.Target().Shard
+}
diff --git a/go/vt/vttablet/tabletserver/tabletserver_test.go b/go/vt/vttablet/tabletserver/tabletserver_test.go
index 7ffd201c0a4..621941224a9 100644
--- a/go/vt/vttablet/tabletserver/tabletserver_test.go
+++ b/go/vt/vttablet/tabletserver/tabletserver_test.go
@@ -36,6 +36,7 @@ import (
"vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/sidecardb"
"vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/test/utils"
@@ -152,13 +153,13 @@ func TestTabletServerPrimaryToReplica(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
// This is required because the test is verifying that we rollback transactions on changing serving type,
- // but that only happens immediately if the shut down grace period is not specified.
- tsv.te.shutdownGracePeriod = 0
- tsv.sm.shutdownGracePeriod = 0
- defer tsv.StopService()
- defer db.Close()
+ // but that only happens when we have a shutdown grace period, otherwise we wait for transactions to be resolved
+ // indefinitely.
+ tsv.te.shutdownGracePeriod = 1
+ tsv.sm.shutdownGracePeriod = 1
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
state1, err := tsv.Begin(ctx, &target, nil)
require.NoError(t, err)
@@ -197,17 +198,22 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
- tsv.SetServingType(topodatapb.TabletType_REPLICA, time.Time{}, true, "")
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+ // This is required because the test is verifying that we rollback transactions on changing serving type,
+ // but that only happens when we have a shutdown grace period, otherwise we wait for transactions to be resolved
+ // indefinitely.
+ tsv.te.shutdownGracePeriod = 1
+ tsv.sm.shutdownGracePeriod = 1
+ tsv.SetServingType(topodatapb.TabletType_PRIMARY, time.Time{}, false, "")
turnOnTxEngine := func() {
tsv.SetServingType(topodatapb.TabletType_PRIMARY, time.Time{}, true, "")
- tsv.TwoPCEngineWait()
}
turnOffTxEngine := func() {
- tsv.SetServingType(topodatapb.TabletType_REPLICA, time.Time{}, true, "")
+ // We can use a transition to PRIMARY non-serving or REPLICA serving to turn off the transaction engine.
+ // With primary serving, the shutdown of prepared transactions is synchronous, but for the latter its asynchronous.
+ tsv.SetServingType(topodatapb.TabletType_PRIMARY, time.Time{}, false, "")
}
tpc := tsv.te.twoPC
@@ -223,18 +229,22 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) {
{Type: sqltypes.Uint64},
{Type: sqltypes.Uint64},
{Type: sqltypes.VarBinary},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary(""),
sqltypes.NewVarBinary("update test_table set `name` = 2 where pk = 1 limit 10001"),
+ sqltypes.NULL,
}},
})
turnOnTxEngine()
assert.EqualValues(t, 1, len(tsv.te.preparedPool.conns), "len(tsv.te.preparedPool.conns)")
got := tsv.te.preparedPool.conns["dtid0"].TxProperties().Queries
- want := []string{"update test_table set `name` = 2 where pk = 1 limit 10001"}
+ want := []tx.Query{{
+ Sql: "update test_table set `name` = 2 where pk = 1 limit 10001",
+ Tables: []string{"test_table"}}}
utils.MustMatch(t, want, got, "Prepared queries")
turnOffTxEngine()
assert.Empty(t, tsv.te.preparedPool.conns, "tsv.te.preparedPool.conns")
@@ -247,30 +257,39 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) {
{Type: sqltypes.Uint64},
{Type: sqltypes.Uint64},
{Type: sqltypes.VarBinary},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("bogus"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary(""),
sqltypes.NewVarBinary("bogus"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("a:b:10"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary(""),
sqltypes.NewVarBinary("update test_table set `name` = 2 where pk = 1 limit 10001"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("a:b:20"),
sqltypes.NewInt64(RedoStateFailed),
sqltypes.NewVarBinary(""),
sqltypes.NewVarBinary("unused"),
+ sqltypes.TestValue(sqltypes.Text, "deadlock detected, transaction rolled back"),
}},
})
turnOnTxEngine()
assert.EqualValues(t, 1, len(tsv.te.preparedPool.conns), "len(tsv.te.preparedPool.conns)")
got = tsv.te.preparedPool.conns["a:b:10"].TxProperties().Queries
- want = []string{"update test_table set `name` = 2 where pk = 1 limit 10001"}
+ want = []tx.Query{{
+ Sql: "update test_table set `name` = 2 where pk = 1 limit 10001",
+ Tables: []string{"test_table"}}}
utils.MustMatch(t, want, got, "Prepared queries")
- wantFailed := map[string]error{"a:b:20": errPrepFailed}
+ wantFailed := map[string]error{
+ "bogus": errPrepFailed, // The query is rejected by database so added to failed list.
+ "a:b:20": errPrepFailed, // The DTID is already in failed state.
+ }
utils.MustMatch(t, tsv.te.preparedPool.reserved, wantFailed, fmt.Sprintf("Failed dtids: %v, want %v", tsv.te.preparedPool.reserved, wantFailed))
// Verify last id got adjusted.
assert.EqualValues(t, 20, tsv.te.txPool.scp.lastID.Load(), "tsv.te.txPool.lastID.Get()")
@@ -281,13 +300,12 @@ func TestTabletServerRedoLogIsKeptBetweenRestarts(t *testing.T) {
func TestTabletServerCreateTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
- db.AddQueryPattern(fmt.Sprintf("insert into _vt\\.dt_state\\(dtid, state, time_created\\) values \\('aa', %d,.*", int(querypb.TransactionState_PREPARE)), &sqltypes.Result{})
- db.AddQueryPattern("insert into _vt\\.dt_participant\\(dtid, id, keyspace, shard\\) values \\('aa', 1,.*", &sqltypes.Result{})
+ db.AddQueryPattern(fmt.Sprintf("insert into _vt\\.dt_state\\(dtid, state, time_created\\) values \\(_binary'aa', %d,.*", int(querypb.TransactionState_PREPARE)), &sqltypes.Result{})
+ db.AddQueryPattern("insert into _vt\\.dt_participant\\(dtid, id, keyspace, shard\\) values \\(_binary'aa', 1,.*", &sqltypes.Result{})
err := tsv.CreateTransaction(ctx, &target, "aa", []*querypb.Target{{
Keyspace: "t1",
Shard: "0",
@@ -298,32 +316,32 @@ func TestTabletServerCreateTransaction(t *testing.T) {
func TestTabletServerStartCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
- commitTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = 'aa' and state = %d", int(querypb.TransactionState_COMMIT), int(querypb.TransactionState_PREPARE))
+ commitTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = _binary'aa' and state = %d", int(querypb.TransactionState_COMMIT), int(querypb.TransactionState_PREPARE))
db.AddQuery(commitTransition, &sqltypes.Result{RowsAffected: 1})
txid := newTxForPrep(ctx, tsv)
- err := tsv.StartCommit(ctx, &target, txid, "aa")
+ state, err := tsv.StartCommit(ctx, &target, txid, "aa")
require.NoError(t, err)
+ assert.Equal(t, querypb.StartCommitState_Success, state, "StartCommit state")
db.AddQuery(commitTransition, &sqltypes.Result{})
txid = newTxForPrep(ctx, tsv)
- err = tsv.StartCommit(ctx, &target, txid, "aa")
+ state, err = tsv.StartCommit(ctx, &target, txid, "aa")
assert.EqualError(t, err, "could not transition to COMMIT: aa", "Prepare err")
+ assert.Equal(t, querypb.StartCommitState_Fail, state, "StartCommit state")
}
func TestTabletserverSetRollback(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
- rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = 'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
+ rollbackTransition := fmt.Sprintf("update _vt.dt_state set state = %d where dtid = _binary'aa' and state = %d", int(querypb.TransactionState_ROLLBACK), int(querypb.TransactionState_PREPARE))
db.AddQuery(rollbackTransition, &sqltypes.Result{RowsAffected: 1})
txid := newTxForPrep(ctx, tsv)
err := tsv.SetRollback(ctx, &target, "aa", txid)
@@ -338,12 +356,11 @@ func TestTabletserverSetRollback(t *testing.T) {
func TestTabletServerReadTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", &sqltypes.Result{})
got, err := tsv.ReadTransaction(ctx, &target, "aa")
require.NoError(t, err)
want := &querypb.TransactionMetadata{}
@@ -361,8 +378,8 @@ func TestTabletServerReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
- db.AddQuery("select keyspace, shard from _vt.dt_participant where dtid = 'aa'", &sqltypes.Result{
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
+ db.AddQuery("select keyspace, shard from _vt.dt_participant where dtid = _binary'aa'", &sqltypes.Result{
Fields: []*querypb.Field{
{Type: sqltypes.VarBinary},
{Type: sqltypes.VarBinary},
@@ -405,7 +422,7 @@ func TestTabletServerReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
want.State = querypb.TransactionState_COMMIT
got, err = tsv.ReadTransaction(ctx, &target, "aa")
require.NoError(t, err)
@@ -423,7 +440,7 @@ func TestTabletServerReadTransaction(t *testing.T) {
sqltypes.NewVarBinary("1"),
}},
}
- db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = 'aa'", txResult)
+ db.AddQuery("select dtid, state, time_created from _vt.dt_state where dtid = _binary'aa'", txResult)
want.State = querypb.TransactionState_ROLLBACK
got, err = tsv.ReadTransaction(ctx, &target, "aa")
require.NoError(t, err)
@@ -433,13 +450,12 @@ func TestTabletServerReadTransaction(t *testing.T) {
func TestTabletServerConcludeTransaction(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
- db.AddQuery("delete from _vt.dt_state where dtid = 'aa'", &sqltypes.Result{})
- db.AddQuery("delete from _vt.dt_participant where dtid = 'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.dt_state where dtid = _binary'aa'", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.dt_participant where dtid = _binary'aa'", &sqltypes.Result{})
err := tsv.ConcludeTransaction(ctx, &target, "aa")
require.NoError(t, err)
}
@@ -537,9 +553,8 @@ func TestTabletServerPrepare(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
state, err := tsv.Begin(ctx, &target, nil)
require.NoError(t, err)
@@ -554,9 +569,8 @@ func TestTabletServerCommitPrepared(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
state, err := tsv.Begin(ctx, &target, nil)
require.NoError(t, err)
@@ -671,6 +685,80 @@ func TestSmallerTimeout(t *testing.T) {
}
}
+func TestLoadQueryTimeout(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ db, tsv := setupTabletServerTest(t, ctx, "")
+ defer tsv.StopService()
+ defer db.Close()
+
+ testcases := []struct {
+ name string
+ txID int64
+ setOptions bool
+ optionTimeout int64
+
+ want time.Duration
+ }{{
+ name: "no options and no transaction",
+ want: 30 * time.Second,
+ }, {
+ name: "only transaction",
+ txID: 1234,
+ want: 30 * time.Second,
+ }, {
+ name: "only option - infinite time",
+ setOptions: true,
+ optionTimeout: 0,
+ want: 0 * time.Millisecond,
+ }, {
+ name: "only option - lower time",
+ setOptions: true,
+ optionTimeout: 3, // 3ms
+ want: 3 * time.Millisecond,
+ }, {
+ name: "only option - higher time",
+ setOptions: true,
+ optionTimeout: 40000, // 40s
+ want: 40 * time.Second,
+ }, {
+ name: "transaction and option - infinite time",
+ txID: 1234,
+ setOptions: true,
+ optionTimeout: 0,
+ want: 30 * time.Second,
+ }, {
+ name: "transaction and option - lower time",
+ txID: 1234,
+ setOptions: true,
+ optionTimeout: 3, // 3ms
+ want: 3 * time.Millisecond,
+ }, {
+ name: "transaction and option - higher time",
+ txID: 1234,
+ setOptions: true,
+ optionTimeout: 40000, // 40s
+ want: 30 * time.Second,
+ }}
+ for _, tcase := range testcases {
+ t.Run(tcase.name, func(t *testing.T) {
+ var options *querypb.ExecuteOptions
+ if tcase.setOptions {
+ options = &querypb.ExecuteOptions{
+ Timeout: &querypb.ExecuteOptions_AuthoritativeTimeout{AuthoritativeTimeout: tcase.optionTimeout},
+ }
+ }
+ var got time.Duration
+ if tcase.txID != 0 {
+ got = tsv.loadQueryTimeoutWithTxAndOptions(tcase.txID, options)
+ } else {
+ got = tsv.loadQueryTimeoutWithOptions(options)
+ }
+ assert.Equal(t, tcase.want, got)
+ })
+ }
+}
+
func TestTabletServerReserveConnection(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@@ -801,9 +889,8 @@ func TestTabletServerRollbackPrepared(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer tsv.StopService()
- defer db.Close()
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
state, err := tsv.Begin(ctx, &target, nil)
require.NoError(t, err)
@@ -1435,9 +1522,8 @@ func TestSerializeTransactionsSameRow_RequestCanceled(t *testing.T) {
func TestMessageStream(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
err := tsv.MessageStream(ctx, &target, "nomsg", func(qr *sqltypes.Result) error {
@@ -1463,9 +1549,8 @@ func TestMessageStream(t *testing.T) {
func TestCheckMySQLGauge(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, _, closer := newTestTxExecutor(t, ctx)
+ defer closer()
// Check that initially checkMySQLGauge has 0 value
assert.EqualValues(t, 0, tsv.checkMysqlGaugeFunc.Get())
@@ -1493,9 +1578,8 @@ func TestCheckMySQLGauge(t *testing.T) {
func TestMessageAck(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
ids := []*querypb.Value{{
@@ -1524,9 +1608,8 @@ func TestMessageAck(t *testing.T) {
func TestRescheduleMessages(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
_, err := tsv.messager.GetGenerator("nonmsg")
@@ -1550,9 +1633,8 @@ func TestRescheduleMessages(t *testing.T) {
func TestPurgeMessages(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
target := querypb.Target{TabletType: topodatapb.TabletType_PRIMARY}
_, err := tsv.messager.GetGenerator("nonmsg")
diff --git a/go/vt/vttablet/tabletserver/throttle/base/metric_cache.go b/go/vt/vttablet/tabletserver/throttle/base/metric_cache.go
index 8695cb83229..faad65ca79e 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/metric_cache.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/metric_cache.go
@@ -49,6 +49,7 @@ import (
"github.com/patrickmn/go-cache"
"vitess.io/vitess/go/stats"
+ "vitess.io/vitess/go/vt/vttablet/tmclient"
)
// MetricsQueryType indicates the type of metrics query on MySQL backend. See following.
@@ -142,13 +143,13 @@ func (metric *ThrottleMetric) WithError(err error) *ThrottleMetric {
// ReadThrottleMetrics returns a metric for the given probe. Either by explicit query
// or via SHOW REPLICA STATUS
-func ReadThrottleMetrics(ctx context.Context, probe *Probe, metricsFunc func(context.Context) ThrottleMetrics) ThrottleMetrics {
+func ReadThrottleMetrics(ctx context.Context, probe *Probe, tmClient tmclient.TabletManagerClient, metricsFunc func(context.Context, tmclient.TabletManagerClient) ThrottleMetrics) ThrottleMetrics {
if metrics := getCachedThrottleMetrics(probe); metrics != nil {
return metrics
}
started := time.Now()
- throttleMetrics := metricsFunc(ctx)
+ throttleMetrics := metricsFunc(ctx, tmClient)
go func(metrics ThrottleMetrics, started time.Time) {
stats.GetOrNewGauge("ThrottlerProbesLatency", "probes latency").Set(time.Since(started).Nanoseconds())
diff --git a/go/vt/vttablet/tabletserver/throttle/base/metric_name.go b/go/vt/vttablet/tabletserver/throttle/base/metric_name.go
index 98e1288fb23..43bd2d17a8c 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/metric_name.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/metric_name.go
@@ -60,11 +60,14 @@ func (names MetricNames) Unique() MetricNames {
}
const (
- DefaultMetricName MetricName = "default"
- LagMetricName MetricName = "lag"
- ThreadsRunningMetricName MetricName = "threads_running"
- CustomMetricName MetricName = "custom"
- LoadAvgMetricName MetricName = "loadavg"
+ DefaultMetricName MetricName = "default"
+ LagMetricName MetricName = "lag"
+ ThreadsRunningMetricName MetricName = "threads_running"
+ CustomMetricName MetricName = "custom"
+ LoadAvgMetricName MetricName = "loadavg"
+ HistoryListLengthMetricName MetricName = "history_list_length"
+ MysqldLoadAvgMetricName MetricName = "mysqld-loadavg"
+ MysqldDatadirUsedRatioMetricName MetricName = "mysqld-datadir-used-ratio"
)
func (metric MetricName) DefaultScope() Scope {
diff --git a/go/vt/vttablet/tabletserver/throttle/base/metric_name_test.go b/go/vt/vttablet/tabletserver/throttle/base/metric_name_test.go
index 9867ca18db3..c2e2b44b36f 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/metric_name_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/metric_name_test.go
@@ -21,6 +21,9 @@ import (
"testing"
"github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/textutil"
)
func TestAggregateName(t *testing.T) {
@@ -238,4 +241,27 @@ func TestKnownMetricNames(t *testing.T) {
assert.Contains(t, KnownMetricNames, LoadAvgMetricName)
assert.Contains(t, KnownMetricNames, CustomMetricName)
assert.Contains(t, KnownMetricNames, DefaultMetricName)
+ assert.Contains(t, KnownMetricNames, HistoryListLengthMetricName)
+ assert.Contains(t, KnownMetricNames, MysqldLoadAvgMetricName)
+ assert.Contains(t, KnownMetricNames, MysqldDatadirUsedRatioMetricName)
+}
+
+func TestKnownMetricNamesPascalCase(t *testing.T) {
+ expectCases := map[MetricName]string{
+ LagMetricName: "Lag",
+ ThreadsRunningMetricName: "ThreadsRunning",
+ LoadAvgMetricName: "Loadavg",
+ HistoryListLengthMetricName: "HistoryListLength",
+ CustomMetricName: "Custom",
+ DefaultMetricName: "Default",
+ MysqldLoadAvgMetricName: "MysqldLoadavg",
+ MysqldDatadirUsedRatioMetricName: "MysqldDatadirUsedRatio",
+ }
+ for _, metricName := range KnownMetricNames {
+ t.Run(metricName.String(), func(t *testing.T) {
+ expect, ok := expectCases[metricName]
+ require.True(t, ok)
+ assert.Equal(t, expect, textutil.PascalCase(metricName.String()))
+ })
+ }
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric.go
index 220dfa6bf60..88fbe2bdd13 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric.go
@@ -21,15 +21,24 @@ import (
"fmt"
"strconv"
+ "vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
+ "vitess.io/vitess/go/vt/vttablet/tmclient"
)
+type SelfMetricReadParams struct {
+ Throttler metricsPublisher
+ Conn *connpool.Conn
+ TmClient tmclient.TabletManagerClient
+ TabletInfo *topo.TabletInfo
+}
+
type SelfMetric interface {
Name() MetricName
DefaultScope() Scope
DefaultThreshold() float64
RequiresConn() bool
- Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric
+ Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric
}
var (
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_custom_query.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_custom_query.go
index 585e63ea285..88f789e5dcd 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric_custom_query.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_custom_query.go
@@ -18,8 +18,6 @@ package base
import (
"context"
-
- "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
)
var _ SelfMetric = registerSelfMetric(&CustomQuerySelfMetric{})
@@ -43,6 +41,6 @@ func (m *CustomQuerySelfMetric) RequiresConn() bool {
return true
}
-func (m *CustomQuerySelfMetric) Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric {
- return ReadSelfMySQLThrottleMetric(ctx, conn, throttler.GetCustomMetricsQuery())
+func (m *CustomQuerySelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ return ReadSelfMySQLThrottleMetric(ctx, params.Conn, params.Throttler.GetCustomMetricsQuery())
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_default.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_default.go
index 8bce295da7c..97309fa6ea9 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric_default.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_default.go
@@ -19,8 +19,6 @@ package base
import (
"context"
"fmt"
-
- "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
)
var _ SelfMetric = registerSelfMetric(&DefaultSelfMetric{})
@@ -44,7 +42,7 @@ func (m *DefaultSelfMetric) RequiresConn() bool {
return false
}
-func (m *DefaultSelfMetric) Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric {
+func (m *DefaultSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
return &ThrottleMetric{
Err: fmt.Errorf("unexpected direct call to DefaultSelfMetric.Read"),
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_innodb_history_list_length.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_innodb_history_list_length.go
new file mode 100644
index 00000000000..2696b1750ea
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_innodb_history_list_length.go
@@ -0,0 +1,68 @@
+/*
+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 base
+
+import (
+ "context"
+ "math"
+ "sync/atomic"
+ "time"
+)
+
+var (
+ historyListLengthQuery = "select count as history_len from information_schema.INNODB_METRICS where name = 'trx_rseg_history_len'"
+
+ cachedHistoryListLengthMetric atomic.Pointer[ThrottleMetric]
+ historyListLengthCacheDuration = 5 * time.Second
+ historyListLengthDefaultThreshold = math.Pow10(9)
+)
+
+var _ SelfMetric = registerSelfMetric(&HistoryListLengthSelfMetric{})
+
+type HistoryListLengthSelfMetric struct {
+}
+
+func (m *HistoryListLengthSelfMetric) Name() MetricName {
+ return HistoryListLengthMetricName
+}
+
+func (m *HistoryListLengthSelfMetric) DefaultScope() Scope {
+ return SelfScope
+}
+
+func (m *HistoryListLengthSelfMetric) DefaultThreshold() float64 {
+ return historyListLengthDefaultThreshold
+}
+
+func (m *HistoryListLengthSelfMetric) RequiresConn() bool {
+ return true
+}
+
+func (m *HistoryListLengthSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ // This function will be called sequentially, and therefore does not need strong mutex protection. Still, we use atomics
+ // to ensure correctness in case an external goroutine tries to read the metric concurrently.
+ metric := cachedHistoryListLengthMetric.Load()
+ if metric != nil {
+ return metric
+ }
+ metric = ReadSelfMySQLThrottleMetric(ctx, params.Conn, historyListLengthQuery)
+ cachedHistoryListLengthMetric.Store(metric)
+ time.AfterFunc(historyListLengthCacheDuration, func() {
+ cachedHistoryListLengthMetric.Store(nil)
+ })
+ return metric
+}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_lag.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_lag.go
index dc25ee5622a..3d0e4beebe1 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric_lag.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_lag.go
@@ -23,7 +23,6 @@ import (
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/vt/sqlparser"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
)
var (
@@ -65,6 +64,6 @@ func (m *LagSelfMetric) RequiresConn() bool {
return true
}
-func (m *LagSelfMetric) Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric {
- return ReadSelfMySQLThrottleMetric(ctx, conn, m.GetQuery())
+func (m *LagSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ return ReadSelfMySQLThrottleMetric(ctx, params.Conn, m.GetQuery())
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_loadavg.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_loadavg.go
index 40a2878421a..2d880169020 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric_loadavg.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_loadavg.go
@@ -18,20 +18,16 @@ package base
import (
"context"
- "fmt"
- "os"
"runtime"
- "strconv"
- "strings"
+ "sync/atomic"
+ "time"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
+ "vitess.io/vitess/go/osutil"
)
var (
- loadavgOnlyAvailableOnLinuxMetric = &ThrottleMetric{
- Scope: SelfScope,
- Err: fmt.Errorf("loadavg metric is only available on Linux"),
- }
+ cachedLoadAvgMetric atomic.Pointer[ThrottleMetric]
+ loadAvgCacheDuration = 1 * time.Second
)
var _ SelfMetric = registerSelfMetric(&LoadAvgSelfMetric{})
@@ -55,27 +51,26 @@ func (m *LoadAvgSelfMetric) RequiresConn() bool {
return false
}
-func (m *LoadAvgSelfMetric) Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric {
- if runtime.GOOS != "linux" {
- return loadavgOnlyAvailableOnLinuxMetric
+func (m *LoadAvgSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ // This function will be called sequentially, and therefore does not need strong mutex protection. Still, we use atomics
+ // to ensure correctness in case an external goroutine tries to read the metric concurrently.
+ metric := cachedLoadAvgMetric.Load()
+ if metric != nil {
+ return metric
}
- metric := &ThrottleMetric{
+ metric = &ThrottleMetric{
Scope: SelfScope,
}
- {
- content, err := os.ReadFile("/proc/loadavg")
- if err != nil {
- return metric.WithError(err)
- }
- fields := strings.Fields(string(content))
- if len(fields) == 0 {
- return metric.WithError(fmt.Errorf("unexpected /proc/loadavg content"))
- }
- loadAvg, err := strconv.ParseFloat(fields[0], 64)
- if err != nil {
- return metric.WithError(err)
- }
- metric.Value = loadAvg / float64(runtime.NumCPU())
+ val, err := osutil.LoadAvg()
+ if err != nil {
+ return metric.WithError(err)
}
+ metric.Value = val / float64(runtime.NumCPU())
+
+ cachedLoadAvgMetric.Store(metric)
+ time.AfterFunc(loadAvgCacheDuration, func() {
+ cachedLoadAvgMetric.Store(nil)
+ })
+
return metric
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld.go
new file mode 100644
index 00000000000..321837d86b4
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld.go
@@ -0,0 +1,156 @@
+/*
+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 base
+
+import (
+ "context"
+ "errors"
+ "fmt"
+ "sync/atomic"
+ "time"
+
+ "vitess.io/vitess/go/timer"
+
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
+)
+
+var (
+ mysqlHostMetricsRpcTimeout = 5 * time.Second
+ mysqlHostMetricsRateLimit = 10 * time.Second
+ mysqlHostMetricsRateLimiter atomic.Pointer[timer.RateLimiter]
+ lastMySQLHostMetricsResponse atomic.Pointer[tabletmanagerdatapb.MysqlHostMetricsResponse]
+)
+
+// getMysqlMetricsRateLimiter returns a rate limiter that is active until the given context is cancelled.
+// This function will be called sequentially, but nonetheless it offers _some_ concurrent safety. Namely,
+// that a created rate limiter is guaranteed to be cleaned up
+func getMysqlMetricsRateLimiter(ctx context.Context, rateLimit time.Duration) *timer.RateLimiter {
+ rateLimiter := mysqlHostMetricsRateLimiter.Load()
+ if rateLimiter == nil {
+ rateLimiter = timer.NewRateLimiter(rateLimit)
+ go func() {
+ defer mysqlHostMetricsRateLimiter.Store(nil)
+ defer rateLimiter.Stop()
+ <-ctx.Done()
+ }()
+ mysqlHostMetricsRateLimiter.Store(rateLimiter)
+ }
+ return rateLimiter
+}
+
+// readMysqlHostMetrics reads MySQL host metrics sporadically from the tablet manager (which in turn reads
+// them from mysql deamon). The metrics are then cached, whether successful or not.
+// This idea is that is is very wasteful to read these metrics for every single query. E.g. right now the throttler
+// can issue 4 reads per second, which is wasteful to go through two RPCs to get the disk space usage for example. Even the load
+// average on the MySQL server is not that susceptible to change.
+func readMysqlHostMetrics(ctx context.Context, params *SelfMetricReadParams) error {
+ if params.TmClient == nil {
+ return fmt.Errorf("tmClient is nil")
+ }
+ if params.TabletInfo == nil {
+ return fmt.Errorf("tabletInfo is nil")
+ }
+ rateLimiter := getMysqlMetricsRateLimiter(ctx, mysqlHostMetricsRateLimit)
+ err := rateLimiter.Do(func() error {
+ ctx, cancel := context.WithTimeout(ctx, mysqlHostMetricsRpcTimeout)
+ defer cancel()
+
+ resp, err := params.TmClient.MysqlHostMetrics(ctx, params.TabletInfo.Tablet, &tabletmanagerdatapb.MysqlHostMetricsRequest{})
+ if err != nil {
+ return err
+ }
+ lastMySQLHostMetricsResponse.Store(resp)
+ return nil
+ })
+ return err
+}
+
+// getMysqlHostMetric gets a metric from the last read MySQL host metrics. The metric will either be directly read from
+// tablet manager (which then reads it from the mysql deamon), or from the cache.
+func getMysqlHostMetric(ctx context.Context, params *SelfMetricReadParams, mysqlHostMetricName string) *ThrottleMetric {
+ metric := &ThrottleMetric{
+ Scope: SelfScope,
+ }
+ if err := readMysqlHostMetrics(ctx, params); err != nil {
+ return metric.WithError(err)
+ }
+ resp := lastMySQLHostMetricsResponse.Load()
+ if resp == nil {
+ return metric.WithError(ErrNoResultYet)
+ }
+ mysqlMetric := resp.HostMetrics.Metrics[mysqlHostMetricName]
+ if mysqlMetric == nil {
+ return metric.WithError(ErrNoSuchMetric)
+ }
+ metric.Value = mysqlMetric.Value
+ if mysqlMetric.Error != nil {
+ metric.Err = errors.New(mysqlMetric.Error.Message)
+ }
+ return metric
+}
+
+var _ SelfMetric = registerSelfMetric(&MysqldLoadAvgSelfMetric{})
+var _ SelfMetric = registerSelfMetric(&MysqldDatadirUsedRatioSelfMetric{})
+
+// MysqldLoadAvgSelfMetric stands for the load average per cpu, on the MySQL host.
+type MysqldLoadAvgSelfMetric struct {
+}
+
+func (m *MysqldLoadAvgSelfMetric) Name() MetricName {
+ return MysqldLoadAvgMetricName
+}
+
+func (m *MysqldLoadAvgSelfMetric) DefaultScope() Scope {
+ return SelfScope
+}
+
+func (m *MysqldLoadAvgSelfMetric) DefaultThreshold() float64 {
+ return 1.0
+}
+
+func (m *MysqldLoadAvgSelfMetric) RequiresConn() bool {
+ return false
+}
+
+func (m *MysqldLoadAvgSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ return getMysqlHostMetric(ctx, params, "loadavg")
+}
+
+// MysqldDatadirUsedRatioSelfMetric stands for the disk space usage of the mount where MySQL's datadir is located.
+// Range: 0.0 (empty) - 1.0 (full)
+type MysqldDatadirUsedRatioSelfMetric struct {
+}
+
+func (m *MysqldDatadirUsedRatioSelfMetric) Name() MetricName {
+ return MysqldDatadirUsedRatioMetricName
+}
+
+func (m *MysqldDatadirUsedRatioSelfMetric) DefaultScope() Scope {
+ return SelfScope
+}
+
+func (m *MysqldDatadirUsedRatioSelfMetric) DefaultThreshold() float64 {
+ return 0.98
+}
+
+func (m *MysqldDatadirUsedRatioSelfMetric) RequiresConn() bool {
+ return false
+}
+
+func (m *MysqldDatadirUsedRatioSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ return getMysqlHostMetric(ctx, params, "datadir-used-ratio")
+}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld_test.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld_test.go
new file mode 100644
index 00000000000..39d3f3f5ec2
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_mysqld_test.go
@@ -0,0 +1,72 @@
+/*
+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 base
+
+import (
+ "context"
+ "fmt"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/assert"
+)
+
+func TestGetMysqlMetricsRateLimiter(t *testing.T) {
+ rateLimit := 10 * time.Millisecond
+ for i := range 3 {
+ testName := fmt.Sprintf("iteration %d", i)
+ t.Run(testName, func(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ {
+ rateLimiter := mysqlHostMetricsRateLimiter.Load()
+ assert.Nil(t, rateLimiter)
+ }
+ rateLimiter := getMysqlMetricsRateLimiter(ctx, rateLimit)
+ assert.NotNil(t, rateLimiter)
+ for range 5 {
+ r := getMysqlMetricsRateLimiter(ctx, rateLimit)
+ // Returning the same rate limiter
+ assert.Equal(t, rateLimiter, r)
+ }
+ val := 0
+ incr := func() error {
+ val++
+ return nil
+ }
+ for range 10 {
+ rateLimiter.Do(incr)
+ time.Sleep(2 * rateLimit)
+ }
+ assert.EqualValues(t, 10, val)
+ cancel()
+ // There can be a race condition where the rate limiter still emits one final tick after the context is cancelled.
+ // So we wait enough time to ensure that tick is "wasted".
+ time.Sleep(2 * rateLimit)
+ // Now that the rate limited was stopped (we invoked `cancel()`), its `Do()` should not invoke the function anymore.
+ for range 7 {
+ rateLimiter.Do(incr)
+ time.Sleep(time.Millisecond)
+ }
+ assert.EqualValues(t, 10, val) // Same "10" value as before.
+ {
+ rateLimiter := mysqlHostMetricsRateLimiter.Load()
+ assert.Nil(t, rateLimiter)
+ }
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/self_metric_threads_running.go b/go/vt/vttablet/tabletserver/throttle/base/self_metric_threads_running.go
index 08f7d408d1c..cb59547a768 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/self_metric_threads_running.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/self_metric_threads_running.go
@@ -18,8 +18,6 @@ package base
import (
"context"
-
- "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
)
var (
@@ -47,6 +45,6 @@ func (m *ThreadsRunningSelfMetric) RequiresConn() bool {
return true
}
-func (m *ThreadsRunningSelfMetric) Read(ctx context.Context, throttler ThrottlerMetricsPublisher, conn *connpool.Conn) *ThrottleMetric {
- return ReadSelfMySQLThrottleMetric(ctx, conn, threadsRunningMetricQuery)
+func (m *ThreadsRunningSelfMetric) Read(ctx context.Context, params *SelfMetricReadParams) *ThrottleMetric {
+ return ReadSelfMySQLThrottleMetric(ctx, params.Conn, threadsRunningMetricQuery)
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/throttler_metrics_publisher.go b/go/vt/vttablet/tabletserver/throttle/base/throttler_metrics_publisher.go
index 1d2d4d0652c..10020af27e6 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/throttler_metrics_publisher.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/throttler_metrics_publisher.go
@@ -16,8 +16,8 @@ limitations under the License.
package base
-// ThrottlerMetricsPublisher is implemented by throttler.Throttler and is used by SelfMetric
+// metricsPublisher is implemented by throttler.Throttler and is used by SelfMetric
// implementations to query the throttler.
-type ThrottlerMetricsPublisher interface {
+type metricsPublisher interface {
GetCustomMetricsQuery() string
}
diff --git a/go/vt/vttablet/tabletserver/throttle/check.go b/go/vt/vttablet/tabletserver/throttle/check.go
index ccdfcb2ce23..d7f43d85e9d 100644
--- a/go/vt/vttablet/tabletserver/throttle/check.go
+++ b/go/vt/vttablet/tabletserver/throttle/check.go
@@ -188,9 +188,9 @@ func (check *ThrottlerCheck) Check(ctx context.Context, appName string, scope ba
// Out of abundance of caution, we will protect against such a scenario.
return
}
- stats.GetOrNewCounter(fmt.Sprintf("ThrottlerCheck%s%sTotal", textutil.SingleWordCamel(metricScope.String()), textutil.SingleWordCamel(metricName.String())), "").Add(1)
+ stats.GetOrNewCounter(fmt.Sprintf("ThrottlerCheck%s%sTotal", textutil.PascalCase(metricScope.String()), textutil.PascalCase(metricName.String())), "").Add(1)
if !metricCheckResult.IsOK() {
- stats.GetOrNewCounter(fmt.Sprintf("ThrottlerCheck%s%sError", textutil.SingleWordCamel(metricScope.String()), textutil.SingleWordCamel(metricName.String())), "").Add(1)
+ stats.GetOrNewCounter(fmt.Sprintf("ThrottlerCheck%s%sError", textutil.PascalCase(metricScope.String()), textutil.PascalCase(metricName.String())), "").Add(1)
}
}(metricCheckResult)
}
@@ -249,7 +249,7 @@ func (check *ThrottlerCheck) localCheck(ctx context.Context, aggregatedMetricNam
check.throttler.markMetricHealthy(aggregatedMetricName)
}
if timeSinceHealthy, found := check.throttler.timeSinceMetricHealthy(aggregatedMetricName); found {
- go stats.GetOrNewGauge(fmt.Sprintf("ThrottlerCheck%sSecondsSinceHealthy", textutil.SingleWordCamel(scope.String())), fmt.Sprintf("seconds since last healthy check for %v", scope)).Set(int64(timeSinceHealthy.Seconds()))
+ go stats.GetOrNewGauge(fmt.Sprintf("ThrottlerCheck%sSecondsSinceHealthy", textutil.PascalCase(scope.String())), fmt.Sprintf("seconds since last healthy check for %v", scope)).Set(int64(timeSinceHealthy.Seconds()))
}
return checkResult
@@ -261,7 +261,7 @@ func (check *ThrottlerCheck) reportAggregated(aggregatedMetricName string, metri
return
}
if value, err := metricResult.Get(); err == nil {
- stats.GetOrNewGaugeFloat64(fmt.Sprintf("ThrottlerAggregated%s%s", textutil.SingleWordCamel(scope.String()), textutil.SingleWordCamel(metricName.String())), fmt.Sprintf("aggregated value for %v", scope)).Set(value)
+ stats.GetOrNewGaugeFloat64(fmt.Sprintf("ThrottlerAggregated%s%s", textutil.PascalCase(scope.String()), textutil.PascalCase(metricName.String())), fmt.Sprintf("aggregated value for %v", scope)).Set(value)
}
}
diff --git a/go/vt/vttablet/tabletserver/throttle/throttler.go b/go/vt/vttablet/tabletserver/throttle/throttler.go
index 5cd56460713..839ba9d43b8 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttler.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttler.go
@@ -95,7 +95,6 @@ const (
DefaultThrottleRatio = 1.0
defaultReplicationLagQuery = "select unix_timestamp(now(6))-max(ts/1000000000) as replication_lag from %s.heartbeat"
- threadsRunningQuery = "show global status like 'threads_running'"
inventoryPrefix = "inventory/"
throttlerConfigPrefix = "config/"
@@ -134,9 +133,10 @@ type throttlerTopoService interface {
// Throttler is the main entity in the throttling mechanism. This service runs, probes, collects data,
// aggregates, reads inventory, provides information, etc.
type Throttler struct {
- keyspace string
- shard string
- cell string
+ keyspace string
+ shard string
+ tabletAlias *topodatapb.TabletAlias
+ tabletInfo atomic.Pointer[topo.TabletInfo]
check *ThrottlerCheck
isEnabled atomic.Bool
@@ -159,7 +159,6 @@ type Throttler struct {
srvTopoServer srvtopo.Server
heartbeatWriter heartbeat.HeartbeatWriter
overrideTmClient tmclient.TabletManagerClient
- tabletAlias string
recentCheckRateLimiter *timer.RateLimiter
recentCheckDormantDiff int64
@@ -191,7 +190,7 @@ type Throttler struct {
cancelEnableContext context.CancelFunc
throttledAppsMutex sync.Mutex
- readSelfThrottleMetrics func(context.Context) base.ThrottleMetrics // overwritten by unit test
+ readSelfThrottleMetrics func(context.Context, tmclient.TabletManagerClient) base.ThrottleMetrics // overwritten by unit test
}
// ThrottlerStatus published some status values from the throttler
@@ -219,9 +218,9 @@ type ThrottlerStatus struct {
}
// NewThrottler creates a Throttler
-func NewThrottler(env tabletenv.Env, srvTopoServer srvtopo.Server, ts *topo.Server, cell string, heartbeatWriter heartbeat.HeartbeatWriter, tabletTypeFunc func() topodatapb.TabletType) *Throttler {
+func NewThrottler(env tabletenv.Env, srvTopoServer srvtopo.Server, ts *topo.Server, tabletAlias *topodatapb.TabletAlias, heartbeatWriter heartbeat.HeartbeatWriter, tabletTypeFunc func() topodatapb.TabletType) *Throttler {
throttler := &Throttler{
- cell: cell,
+ tabletAlias: tabletAlias,
env: env,
tabletTypeFunc: tabletTypeFunc,
srvTopoServer: srvTopoServer,
@@ -263,12 +262,20 @@ func NewThrottler(env tabletenv.Env, srvTopoServer srvtopo.Server, ts *topo.Serv
}
throttler.StoreMetricsThreshold(base.RegisteredSelfMetrics[base.LagMetricName].DefaultThreshold())
- throttler.readSelfThrottleMetrics = func(ctx context.Context) base.ThrottleMetrics {
- return throttler.readSelfThrottleMetricsInternal(ctx)
+ throttler.readSelfThrottleMetrics = func(ctx context.Context, tmClient tmclient.TabletManagerClient) base.ThrottleMetrics {
+ return throttler.readSelfThrottleMetricsInternal(ctx, tmClient)
}
return throttler
}
+// tabletAliasString returns tablet alias as string
+func (throttler *Throttler) tabletAliasString() string {
+ if throttler.tabletAlias == nil {
+ return ""
+ }
+ return topoproto.TabletAliasString(throttler.tabletAlias)
+}
+
func (throttler *Throttler) StoreMetricsThreshold(threshold float64) {
throttler.MetricsThreshold.Store(math.Float64bits(threshold))
}
@@ -331,7 +338,16 @@ func (throttler *Throttler) initConfig() {
// readThrottlerConfig proactively reads the throttler's config from SrvKeyspace in local topo
func (throttler *Throttler) readThrottlerConfig(ctx context.Context) (*topodatapb.ThrottlerConfig, error) {
- srvks, err := throttler.ts.GetSrvKeyspace(ctx, throttler.cell, throttler.keyspace)
+ // since we're reading from topo, let's seize this opportunity to read table info as well
+ if throttler.tabletInfo.Load() == nil {
+ if ti, err := throttler.ts.GetTablet(ctx, throttler.tabletAlias); err == nil {
+ throttler.tabletInfo.Store(ti)
+ } else {
+ log.Errorf("Throttler: error reading tablet info: %v", err)
+ }
+ }
+
+ srvks, err := throttler.ts.GetSrvKeyspace(ctx, throttler.tabletAlias.Cell, throttler.keyspace)
if err != nil {
return nil, err
}
@@ -578,7 +594,7 @@ func (throttler *Throttler) retryReadAndApplyThrottlerConfig(ctx context.Context
go watchSrvKeyspaceOnce.Do(func() {
// We start watching SrvKeyspace only after we know it's been created. Now is that time!
// We watch using the given ctx, which is cancelled when the throttler is Close()d.
- throttler.srvTopoServer.WatchSrvKeyspace(ctx, throttler.cell, throttler.keyspace, throttler.WatchSrvKeyspaceCallback)
+ throttler.srvTopoServer.WatchSrvKeyspace(ctx, throttler.tabletAlias.Cell, throttler.keyspace, throttler.WatchSrvKeyspaceCallback)
})
return
}
@@ -797,7 +813,7 @@ func (throttler *Throttler) Operate(ctx context.Context, wg *sync.WaitGroup) {
if throttler.IsOpen() {
// frequent
// Always collect self metrics:
- throttler.collectSelfMetrics(ctx)
+ throttler.collectSelfMetrics(ctx, tmClient)
if !throttler.isDormant() {
throttler.collectShardMetrics(ctx, tmClient)
}
@@ -862,7 +878,7 @@ func (throttler *Throttler) Operate(ctx context.Context, wg *sync.WaitGroup) {
}()
}
-func (throttler *Throttler) generateTabletProbeFunction(scope base.Scope, tmClient tmclient.TabletManagerClient, probe *base.Probe) (probeFunc func(context.Context) base.ThrottleMetrics) {
+func (throttler *Throttler) generateTabletProbeFunction(scope base.Scope, probe *base.Probe) (probeFunc func(context.Context, tmclient.TabletManagerClient) base.ThrottleMetrics) {
metricsWithError := func(err error) base.ThrottleMetrics {
metrics := base.ThrottleMetrics{}
for _, metricName := range base.KnownMetricNames {
@@ -875,7 +891,7 @@ func (throttler *Throttler) generateTabletProbeFunction(scope base.Scope, tmClie
}
return metrics
}
- return func(ctx context.Context) base.ThrottleMetrics {
+ return func(ctx context.Context, tmClient tmclient.TabletManagerClient) base.ThrottleMetrics {
// Some reasonable timeout, to ensure we release connections even if they're hanging (otherwise grpc-go keeps polling those connections forever)
ctx, cancel := context.WithTimeout(ctx, 4*activeCollectInterval)
defer cancel()
@@ -933,7 +949,7 @@ func (throttler *Throttler) generateTabletProbeFunction(scope base.Scope, tmClie
// readSelfThrottleMetricsInternal rreads all registsred self metrics on this tablet (or backend MySQL server).
// This is the actual place where metrics are read, to be later aggregated and/or propagated to other tablets.
-func (throttler *Throttler) readSelfThrottleMetricsInternal(ctx context.Context) base.ThrottleMetrics {
+func (throttler *Throttler) readSelfThrottleMetricsInternal(ctx context.Context, tmClient tmclient.TabletManagerClient) base.ThrottleMetrics {
result := make(base.ThrottleMetrics, len(base.RegisteredSelfMetrics))
writeMetric := func(metric *base.ThrottleMetric) {
select {
@@ -943,15 +959,20 @@ func (throttler *Throttler) readSelfThrottleMetricsInternal(ctx context.Context)
}
}
readMetric := func(selfMetric base.SelfMetric) *base.ThrottleMetric {
- if !selfMetric.RequiresConn() {
- return selfMetric.Read(ctx, throttler, nil)
+ params := &base.SelfMetricReadParams{
+ Throttler: throttler,
+ TmClient: tmClient,
+ TabletInfo: throttler.tabletInfo.Load(),
}
- conn, err := throttler.pool.Get(ctx, nil)
- if err != nil {
- return &base.ThrottleMetric{Err: err}
+ if selfMetric.RequiresConn() {
+ conn, err := throttler.pool.Get(ctx, nil)
+ if err != nil {
+ return &base.ThrottleMetric{Err: err}
+ }
+ defer conn.Recycle()
+ params.Conn = conn.Conn
}
- defer conn.Recycle()
- return selfMetric.Read(ctx, throttler, conn.Conn)
+ return selfMetric.Read(ctx, params)
}
for metricName, selfMetric := range base.RegisteredSelfMetrics {
if metricName == base.DefaultMetricName {
@@ -959,16 +980,17 @@ func (throttler *Throttler) readSelfThrottleMetricsInternal(ctx context.Context)
}
metric := readMetric(selfMetric)
metric.Name = metricName
- metric.Alias = throttler.tabletAlias
+ metric.Alias = throttler.tabletAliasString()
go writeMetric(metric)
result[metricName] = metric
}
+
return result
}
-func (throttler *Throttler) collectSelfMetrics(ctx context.Context) {
- probe := throttler.inventory.ClustersProbes[throttler.tabletAlias]
+func (throttler *Throttler) collectSelfMetrics(ctx context.Context, tmClient tmclient.TabletManagerClient) {
+ probe := throttler.inventory.ClustersProbes[throttler.tabletAliasString()]
if probe == nil {
// probe not created yet
return
@@ -982,7 +1004,7 @@ func (throttler *Throttler) collectSelfMetrics(ctx context.Context) {
defer atomic.StoreInt64(&probe.QueryInProgress, 0)
// Throttler is probing its own tablet's metrics:
- _ = base.ReadThrottleMetrics(ctx, probe, throttler.readSelfThrottleMetrics)
+ _ = base.ReadThrottleMetrics(ctx, probe, tmClient, throttler.readSelfThrottleMetrics)
}()
}
@@ -990,7 +1012,7 @@ func (throttler *Throttler) collectShardMetrics(ctx context.Context, tmClient tm
// probes is known not to change. It can be *replaced*, but not changed.
// so it's safe to iterate it
for _, probe := range throttler.inventory.ClustersProbes {
- if probe.Alias == throttler.tabletAlias {
+ if probe.Alias == throttler.tabletAliasString() {
// We skip collecting our own metrics
continue
}
@@ -1003,9 +1025,9 @@ func (throttler *Throttler) collectShardMetrics(ctx context.Context, tmClient tm
defer atomic.StoreInt64(&probe.QueryInProgress, 0)
// Throttler probing other tablets:
- throttleMetricFunc := throttler.generateTabletProbeFunction(base.ShardScope, tmClient, probe)
+ throttleMetricFunc := throttler.generateTabletProbeFunction(base.ShardScope, probe)
- throttleMetrics := base.ReadThrottleMetrics(ctx, probe, throttleMetricFunc)
+ throttleMetrics := base.ReadThrottleMetrics(ctx, probe, tmClient, throttleMetricFunc)
for _, metric := range throttleMetrics {
select {
case <-ctx.Done():
@@ -1077,7 +1099,7 @@ func (throttler *Throttler) refreshInventory(ctx context.Context) error {
TabletProbes: base.NewProbes(),
}
// self tablet
- addProbe(throttler.tabletAlias, nil, base.SelfScope, &clusterSettingsCopy, clusterProbes.TabletProbes)
+ addProbe(throttler.tabletAliasString(), nil, base.SelfScope, &clusterSettingsCopy, clusterProbes.TabletProbes)
if !throttler.isLeader.Load() {
// This tablet may have used to be the primary, but it isn't now. It may have a recollection
// of previous clusters it used to probe. It may have recollection of specific probes for such clusters.
@@ -1168,7 +1190,7 @@ func (throttler *Throttler) aggregateMetrics() error {
// is to be stored as "default"
continue
}
- selfResultsMap, shardResultsMap := throttler.inventory.TabletMetrics.Split(throttler.tabletAlias)
+ selfResultsMap, shardResultsMap := throttler.inventory.TabletMetrics.Split(throttler.tabletAliasString())
aggregateTabletsMetrics(base.SelfScope, metricName, selfResultsMap)
aggregateTabletsMetrics(base.ShardScope, metricName, shardResultsMap)
}
diff --git a/go/vt/vttablet/tabletserver/throttle/throttler_test.go b/go/vt/vttablet/tabletserver/throttle/throttler_test.go
index 41036620a60..352e641fa35 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttler_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttler_test.go
@@ -71,6 +71,24 @@ var (
Value: 2.718,
Err: nil,
},
+ base.HistoryListLengthMetricName: &base.ThrottleMetric{
+ Scope: base.SelfScope,
+ Alias: "",
+ Value: 5,
+ Err: nil,
+ },
+ base.MysqldLoadAvgMetricName: &base.ThrottleMetric{
+ Scope: base.SelfScope,
+ Alias: "",
+ Value: 0.3311,
+ Err: nil,
+ },
+ base.MysqldDatadirUsedRatioMetricName: &base.ThrottleMetric{
+ Scope: base.SelfScope,
+ Alias: "",
+ Value: 0.85,
+ Err: nil,
+ },
}
replicaMetrics = map[string]*MetricResult{
base.LagMetricName.String(): {
@@ -93,6 +111,21 @@ var (
ResponseCode: tabletmanagerdatapb.CheckThrottlerResponseCode_OK,
Value: 5.1,
},
+ base.HistoryListLengthMetricName.String(): {
+ StatusCode: http.StatusOK,
+ ResponseCode: tabletmanagerdatapb.CheckThrottlerResponseCode_OK,
+ Value: 6,
+ },
+ base.MysqldLoadAvgMetricName.String(): {
+ StatusCode: http.StatusOK,
+ ResponseCode: tabletmanagerdatapb.CheckThrottlerResponseCode_OK,
+ Value: 0.2211,
+ },
+ base.MysqldDatadirUsedRatioMetricName.String(): {
+ StatusCode: http.StatusOK,
+ ResponseCode: tabletmanagerdatapb.CheckThrottlerResponseCode_OK,
+ Value: 0.87,
+ },
}
nonPrimaryTabletType atomic.Int32
)
@@ -283,7 +316,7 @@ func newTestThrottler() *Throttler {
throttler.recentCheckDormantDiff = int64(throttler.dormantPeriod / recentCheckRateLimiterInterval)
throttler.recentCheckDiff = int64(3 * time.Second / recentCheckRateLimiterInterval)
- throttler.readSelfThrottleMetrics = func(ctx context.Context) base.ThrottleMetrics {
+ throttler.readSelfThrottleMetrics = func(ctx context.Context, tmClient tmclient.TabletManagerClient) base.ThrottleMetrics {
for _, metric := range selfMetrics {
go func() {
select {
@@ -1660,7 +1693,7 @@ func TestProbesPostDisable(t *testing.T) {
localTabletFound := 0
for _, probe := range probes {
require.NotNil(t, probe)
- if probe.Alias == throttler.tabletAlias {
+ if probe.Alias == throttler.tabletAliasString() {
localTabletFound++
} else {
assert.NotEmpty(t, probe.Alias)
@@ -1827,10 +1860,13 @@ func TestChecks(t *testing.T) {
assert.Equal(t, testAppName.String(), checkResult.AppName)
assert.Equal(t, len(base.KnownMetricNames), len(checkResult.Metrics))
- assert.EqualValues(t, 0.3, checkResult.Metrics[base.LagMetricName.String()].Value) // self lag value, because flags.Scope is set
- assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // self value, because flags.Scope is set
- assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // self value, because flags.Scope is set
- assert.EqualValues(t, 2.718, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 0.3, checkResult.Metrics[base.LagMetricName.String()].Value) // self lag value, because flags.Scope is set
+ assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 2.718, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 5, checkResult.Metrics[base.HistoryListLengthMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 0.3311, checkResult.Metrics[base.MysqldLoadAvgMetricName.String()].Value) // self value, because flags.Scope is set
+ assert.EqualValues(t, 0.85, checkResult.Metrics[base.MysqldDatadirUsedRatioMetricName.String()].Value) // self value, because flags.Scope is set
for _, metric := range checkResult.Metrics {
assert.EqualValues(t, base.SelfScope.String(), metric.Scope)
}
@@ -1886,10 +1922,13 @@ func TestChecks(t *testing.T) {
assert.Equal(t, testAppName.String(), checkResult.AppName)
assert.Equal(t, len(base.KnownMetricNames), len(checkResult.Metrics))
- assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, because flags.Scope is set
- assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // shard value, because flags.Scope is set
- assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // shard value, because flags.Scope is set
- assert.EqualValues(t, 5.1, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, because flags.Scope is set
+ assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 5.1, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 6, checkResult.Metrics[base.HistoryListLengthMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 0.3311, checkResult.Metrics[base.MysqldLoadAvgMetricName.String()].Value) // shard value, because flags.Scope is set
+ assert.EqualValues(t, 0.87, checkResult.Metrics[base.MysqldDatadirUsedRatioMetricName.String()].Value) // shard value, because flags.Scope is set
for _, metric := range checkResult.Metrics {
assert.EqualValues(t, base.ShardScope.String(), metric.Scope)
}
@@ -1918,14 +1957,19 @@ func TestChecks(t *testing.T) {
assert.ErrorIs(t, checkResult.Error, base.ErrThresholdExceeded)
assert.Equal(t, len(base.KnownMetricNames), len(checkResult.Metrics))
- assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, because "shard" is the default scope for lag
- assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // self value, because "self" is the default scope for threads_running
- assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // self value, because "self" is the default scope for custom
- assert.EqualValues(t, 2.718, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // self value, because "self" is the default scope for loadavg
+ assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, because "shard" is the default scope for lag
+ assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // self value, because "self" is the default scope for threads_running
+ assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // self value, because "self" is the default scope for custom
+ assert.EqualValues(t, 2.718, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // self value, because "self" is the default scope for loadavg
+ assert.EqualValues(t, 5, checkResult.Metrics[base.HistoryListLengthMetricName.String()].Value) // self value, because "self" is the default scope for loadavg
+ assert.EqualValues(t, 0.3311, checkResult.Metrics[base.MysqldLoadAvgMetricName.String()].Value) // self value, because "self" is the default scope for loadavg
+ assert.EqualValues(t, 0.85, checkResult.Metrics[base.MysqldDatadirUsedRatioMetricName.String()].Value) // self value, because "self" is the default scope for loadavg
assert.EqualValues(t, base.ShardScope.String(), checkResult.Metrics[base.LagMetricName.String()].Scope)
assert.EqualValues(t, base.SelfScope.String(), checkResult.Metrics[base.ThreadsRunningMetricName.String()].Scope)
assert.EqualValues(t, base.SelfScope.String(), checkResult.Metrics[base.CustomMetricName.String()].Scope)
assert.EqualValues(t, base.SelfScope.String(), checkResult.Metrics[base.LoadAvgMetricName.String()].Scope)
+ assert.EqualValues(t, base.SelfScope.String(), checkResult.Metrics[base.MysqldLoadAvgMetricName.String()].Scope)
+ assert.EqualValues(t, base.SelfScope.String(), checkResult.Metrics[base.MysqldDatadirUsedRatioMetricName.String()].Scope)
})
})
t.Run("checks, defined scope masks explicit scope metrics", func(t *testing.T) {
@@ -1939,6 +1983,9 @@ func TestChecks(t *testing.T) {
base.MetricName("self/threads_running"),
base.MetricName("custom"),
base.MetricName("shard/loadavg"),
+ base.MetricName("shard/mysqld-loadavg"),
+ base.MetricName("self/history_list_length"),
+ base.MetricName("self/mysqld-datadir-used-ratio"),
base.MetricName("default"),
}
checkResult := throttler.Check(ctx, testAppName.String(), metricNames, flags)
@@ -1950,10 +1997,13 @@ func TestChecks(t *testing.T) {
assert.ErrorIs(t, checkResult.Error, base.ErrThresholdExceeded)
assert.Equal(t, len(metricNames), len(checkResult.Metrics))
- assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, even though scope name is in metric name
- assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // shard value, even though scope name is in metric name
- assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // shard value because flags.Scope is set
- assert.EqualValues(t, 5.1, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // shard value, not because scope name is in metric name but because flags.Scope is set
+ assert.EqualValues(t, 0.9, checkResult.Metrics[base.LagMetricName.String()].Value) // shard lag value, even though scope name is in metric name
+ assert.EqualValues(t, 26, checkResult.Metrics[base.ThreadsRunningMetricName.String()].Value) // shard value, even though scope name is in metric name
+ assert.EqualValues(t, 17, checkResult.Metrics[base.CustomMetricName.String()].Value) // shard value because flags.Scope is set
+ assert.EqualValues(t, 5.1, checkResult.Metrics[base.LoadAvgMetricName.String()].Value) // shard value, not because scope name is in metric name but because flags.Scope is set
+ assert.EqualValues(t, 6, checkResult.Metrics[base.HistoryListLengthMetricName.String()].Value) // shard value, even though scope name is in metric name
+ assert.EqualValues(t, 0.3311, checkResult.Metrics[base.MysqldLoadAvgMetricName.String()].Value) // shard value, not because scope name is in metric name but because flags.Scope is set
+ assert.EqualValues(t, 0.87, checkResult.Metrics[base.MysqldDatadirUsedRatioMetricName.String()].Value) // shard value, even though scope name is in metric name
for _, metric := range checkResult.Metrics {
assert.EqualValues(t, base.ShardScope.String(), metric.Scope)
}
@@ -2222,8 +2272,13 @@ func TestReplica(t *testing.T) {
base.DefaultMetricName:
assert.Error(t, metricResult.Error, "metricName=%v, value=%v, threshold=%v", metricName, metricResult.Value, metricResult.Threshold)
assert.ErrorIs(t, metricResult.Error, base.ErrThresholdExceeded)
- case base.ThreadsRunningMetricName:
+ case base.ThreadsRunningMetricName,
+ base.HistoryListLengthMetricName,
+ base.MysqldLoadAvgMetricName,
+ base.MysqldDatadirUsedRatioMetricName:
assert.NoError(t, metricResult.Error, "metricName=%v, value=%v, threshold=%v", metricName, metricResult.Value, metricResult.Threshold)
+ default:
+ assert.Fail(t, "unexpected metric", "name=%v", metricName)
}
}
})
diff --git a/go/vt/vttablet/tabletserver/twopc.go b/go/vt/vttablet/tabletserver/twopc.go
index 0bdf4ac0c91..e1212ebe627 100644
--- a/go/vt/vttablet/tabletserver/twopc.go
+++ b/go/vt/vttablet/tabletserver/twopc.go
@@ -27,6 +27,7 @@ import (
"vitess.io/vitess/go/vt/dbconnpool"
"vitess.io/vitess/go/vt/log"
querypb "vitess.io/vitess/go/vt/proto/query"
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
@@ -47,7 +48,7 @@ const (
// DTStateRollback represents the ROLLBACK state for dt_state.
DTStateRollback = querypb.TransactionState_ROLLBACK
- readAllRedo = `select t.dtid, t.state, t.time_created, s.statement
+ readAllRedo = `select t.dtid, t.state, t.time_created, s.statement, t.message
from %s.redo_state t
join %s.redo_statement s on t.dtid = s.dtid
order by t.dtid, s.id`
@@ -61,7 +62,7 @@ const (
// Resolving COMMIT first is crucial because we need to address transactions where a commit decision has already been made but remains unresolved.
// For transactions with a commit decision, applications are already aware of the outcome and are waiting for the resolution.
// By addressing these first, we ensure atomic commits and improve user experience. For other transactions, the decision is typically to rollback.
- readUnresolvedTransactions = `select t.dtid, t.state, p.keyspace, p.shard
+ readUnresolvedTransactions = `select t.dtid, t.state, t.time_created, p.keyspace, p.shard
from %s.dt_state t
join %s.dt_participant p on t.dtid = p.dtid
where time_created < %a
@@ -75,7 +76,9 @@ type TwoPC struct {
readPool *connpool.Pool
insertRedoTx *sqlparser.ParsedQuery
+ readRedoTx *sqlparser.ParsedQuery
insertRedoStmt *sqlparser.ParsedQuery
+ readRedoStmts *sqlparser.ParsedQuery
updateRedoTx *sqlparser.ParsedQuery
deleteRedoTx *sqlparser.ParsedQuery
deleteRedoStmt *sqlparser.ParsedQuery
@@ -105,12 +108,18 @@ func (tpc *TwoPC) initializeQueries() {
tpc.insertRedoTx = sqlparser.BuildParsedQuery(
"insert into %s.redo_state(dtid, state, time_created) values (%a, %a, %a)",
dbname, ":dtid", ":state", ":time_created")
+ tpc.readRedoTx = sqlparser.BuildParsedQuery(
+ "select state, time_created, message from %s.redo_state where dtid = %a",
+ dbname, ":dtid")
tpc.insertRedoStmt = sqlparser.BuildParsedQuery(
"insert into %s.redo_statement(dtid, id, statement) values %a",
dbname, ":vals")
+ tpc.readRedoStmts = sqlparser.BuildParsedQuery(
+ "select statement from %s.redo_statement where dtid = %a order by id",
+ dbname, ":dtid")
tpc.updateRedoTx = sqlparser.BuildParsedQuery(
- "update %s.redo_state set state = %a where dtid = %a",
- dbname, ":state", ":dtid")
+ "update %s.redo_state set state = %a, message = %a where dtid = %a",
+ dbname, ":state", ":message", ":dtid")
tpc.deleteRedoTx = sqlparser.BuildParsedQuery(
"delete from %s.redo_state where dtid = %a",
dbname, ":dtid")
@@ -150,6 +159,18 @@ func (tpc *TwoPC) initializeQueries() {
dbname, ":time_created")
}
+// getStateString gets the redo state of the transaction as a string.
+func getStateString(st int) string {
+ switch st {
+ case RedoStateFailed:
+ return "FAILED"
+ case RedoStatePrepared:
+ return "PREPARED"
+ default:
+ return "UNKNOWN"
+ }
+}
+
// Open starts the TwoPC service.
func (tpc *TwoPC) Open(dbconfigs *dbconfigs.DBConfigs) error {
conn, err := dbconnpool.NewDBConnection(context.TODO(), dbconfigs.DbaWithDB())
@@ -169,9 +190,9 @@ func (tpc *TwoPC) Close() {
}
// SaveRedo saves the statements in the redo log using the supplied connection.
-func (tpc *TwoPC) SaveRedo(ctx context.Context, conn *StatefulConnection, dtid string, queries []string) error {
+func (tpc *TwoPC) SaveRedo(ctx context.Context, conn *StatefulConnection, dtid string, queries []tx.Query) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
"state": sqltypes.Int64BindVariable(RedoStatePrepared),
"time_created": sqltypes.Int64BindVariable(time.Now().UnixNano()),
}
@@ -185,7 +206,7 @@ func (tpc *TwoPC) SaveRedo(ctx context.Context, conn *StatefulConnection, dtid s
rows[i] = []sqltypes.Value{
sqltypes.NewVarBinary(dtid),
sqltypes.NewInt64(int64(i + 1)),
- sqltypes.NewVarBinary(query),
+ sqltypes.NewVarBinary(query.Sql),
}
}
extras := map[string]sqlparser.Encodable{
@@ -200,10 +221,11 @@ func (tpc *TwoPC) SaveRedo(ctx context.Context, conn *StatefulConnection, dtid s
}
// UpdateRedo changes the state of the redo log for the dtid.
-func (tpc *TwoPC) UpdateRedo(ctx context.Context, conn *StatefulConnection, dtid string, state int) error {
+func (tpc *TwoPC) UpdateRedo(ctx context.Context, conn *StatefulConnection, dtid string, state int, message string) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
- "state": sqltypes.Int64BindVariable(int64(state)),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
+ "state": sqltypes.Int64BindVariable(int64(state)),
+ "message": sqltypes.StringBindVariable(message),
}
_, err := tpc.exec(ctx, conn, tpc.updateRedoTx, bindVars)
return err
@@ -212,7 +234,7 @@ func (tpc *TwoPC) UpdateRedo(ctx context.Context, conn *StatefulConnection, dtid
// DeleteRedo deletes the redo log for the dtid.
func (tpc *TwoPC) DeleteRedo(ctx context.Context, conn *StatefulConnection, dtid string) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
}
_, err := tpc.exec(ctx, conn, tpc.deleteRedoTx, bindVars)
if err != nil {
@@ -244,8 +266,9 @@ func (tpc *TwoPC) ReadAllRedo(ctx context.Context) (prepared, failed []*tx.Prepa
// which is harmless.
tm, _ := row[2].ToCastInt64()
curTx = &tx.PreparedTx{
- Dtid: dtid,
- Time: time.Unix(0, tm),
+ Dtid: dtid,
+ Time: time.Unix(0, tm),
+ Message: row[4].ToString(),
}
st, err := row[1].ToCastInt64()
if err != nil {
@@ -289,7 +312,7 @@ func (tpc *TwoPC) CountUnresolvedRedo(ctx context.Context, unresolvedTime time.T
// CreateTransaction saves the metadata of a 2pc transaction as Prepared.
func (tpc *TwoPC) CreateTransaction(ctx context.Context, conn *StatefulConnection, dtid string, participants []*querypb.Target) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
"state": sqltypes.Int64BindVariable(int64(DTStatePrepare)),
"cur_time": sqltypes.Int64BindVariable(time.Now().UnixNano()),
}
@@ -322,7 +345,7 @@ func (tpc *TwoPC) CreateTransaction(ctx context.Context, conn *StatefulConnectio
// If the transaction is not a in the Prepare state, an error is returned.
func (tpc *TwoPC) Transition(ctx context.Context, conn *StatefulConnection, dtid string, state querypb.TransactionState) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
"state": sqltypes.Int64BindVariable(int64(state)),
"prepare": sqltypes.Int64BindVariable(int64(querypb.TransactionState_PREPARE)),
}
@@ -339,7 +362,7 @@ func (tpc *TwoPC) Transition(ctx context.Context, conn *StatefulConnection, dtid
// DeleteTransaction deletes the metadata for the specified transaction.
func (tpc *TwoPC) DeleteTransaction(ctx context.Context, conn *StatefulConnection, dtid string) error {
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
}
_, err := tpc.exec(ctx, conn, tpc.deleteTransaction, bindVars)
if err != nil {
@@ -359,7 +382,7 @@ func (tpc *TwoPC) ReadTransaction(ctx context.Context, dtid string) (*querypb.Tr
result := &querypb.TransactionMetadata{}
bindVars := map[string]*querypb.BindVariable{
- "dtid": sqltypes.StringBindVariable(dtid),
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
}
qr, err := tpc.read(ctx, conn.Conn, tpc.readTransaction, bindVars)
if err != nil {
@@ -374,7 +397,7 @@ func (tpc *TwoPC) ReadTransaction(ctx context.Context, dtid string) (*querypb.Tr
return nil, vterrors.Wrapf(err, "error parsing state for dtid %s", dtid)
}
result.State = querypb.TransactionState(st)
- if result.State < querypb.TransactionState_PREPARE || result.State > querypb.TransactionState_COMMIT {
+ if result.State < DTStatePrepare || result.State > DTStateCommit {
return nil, fmt.Errorf("unexpected state for dtid %s: %v", dtid, result.State)
}
// A failure in time parsing will show up as a very old time,
@@ -398,6 +421,47 @@ func (tpc *TwoPC) ReadTransaction(ctx context.Context, dtid string) (*querypb.Tr
return result, nil
}
+// GetTransactionInfo returns the data for the transaction.
+func (tpc *TwoPC) GetTransactionInfo(ctx context.Context, dtid string) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ conn, err := tpc.readPool.Get(ctx, nil)
+ if err != nil {
+ return nil, err
+ }
+ defer conn.Recycle()
+
+ result := &tabletmanagerdatapb.GetTransactionInfoResponse{}
+ bindVars := map[string]*querypb.BindVariable{
+ "dtid": sqltypes.BytesBindVariable([]byte(dtid)),
+ }
+ qr, err := tpc.read(ctx, conn.Conn, tpc.readRedoTx, bindVars)
+ if err != nil {
+ return nil, err
+ }
+ if len(qr.Rows) == 0 {
+ return result, nil
+ }
+ state, err := qr.Rows[0][0].ToInt()
+ if err != nil {
+ return nil, err
+ }
+ result.State = getStateString(state)
+ // A failure in time parsing will show up as a very old time,
+ // which is harmless.
+ tm, _ := qr.Rows[0][1].ToCastInt64()
+ result.TimeCreated = tm
+ result.Message = qr.Rows[0][2].ToString()
+
+ qr, err = tpc.read(ctx, conn.Conn, tpc.readRedoStmts, bindVars)
+ if err != nil {
+ return nil, err
+ }
+ result.Statements = make([]string, len(qr.Rows))
+ for idx, row := range qr.Rows {
+ result.Statements[idx] = row[0].ToString()
+ }
+ return result, nil
+}
+
// ReadAllTransactions returns info about all distributed transactions.
func (tpc *TwoPC) ReadAllTransactions(ctx context.Context) ([]*tx.DistributedTx, error) {
conn, err := tpc.readPool.Get(ctx, nil)
@@ -427,7 +491,7 @@ func (tpc *TwoPC) ReadAllTransactions(ctx context.Context) ([]*tx.DistributedTx,
log.Errorf("Error parsing state for dtid %s: %v.", dtid, err)
}
protostate := querypb.TransactionState(st)
- if protostate < querypb.TransactionState_PREPARE || protostate > querypb.TransactionState_COMMIT {
+ if protostate < DTStatePrepare || protostate > DTStateCommit {
log.Errorf("Unexpected state for dtid %s: %v.", dtid, protostate)
}
curTx = &tx.DistributedTx{
@@ -464,10 +528,10 @@ func (tpc *TwoPC) read(ctx context.Context, conn *connpool.Conn, pq *sqlparser.P
// UnresolvedTransactions returns the list of unresolved transactions
// the list from database is retrieved as
-// dtid | state | keyspace | shard
-// 1 | PREPARE | ks | 40-80
-// 1 | PREPARE | ks | 80-c0
-// 2 | COMMIT | ks | -40
+// dtid | state | time_created | keyspace | shard
+// 1 | PREPARE | 1726748387 | ks | 40-80
+// 1 | PREPARE | 1726748387 | ks | 80-c0
+// 2 | COMMIT | 1726748387 | ks | -40
// Here there are 2 dtids with 2 participants for dtid:1 and 1 participant for dtid:2.
func (tpc *TwoPC) UnresolvedTransactions(ctx context.Context, abandonTime time.Time) ([]*querypb.TransactionMetadata, error) {
conn, err := tpc.readPool.Get(ctx, nil)
@@ -506,17 +570,19 @@ func (tpc *TwoPC) UnresolvedTransactions(ctx context.Context, abandonTime time.T
// Extract the transaction state and initialize a new TransactionMetadata
stateID, _ := row[1].ToInt()
+ timeCreated, _ := row[2].ToCastInt64()
currentTx = &querypb.TransactionMetadata{
Dtid: dtid,
State: querypb.TransactionState(stateID),
+ TimeCreated: timeCreated,
Participants: []*querypb.Target{},
}
}
// Add the current participant (keyspace and shard) to the transaction
currentTx.Participants = append(currentTx.Participants, &querypb.Target{
- Keyspace: row[2].ToString(),
- Shard: row[3].ToString(),
+ Keyspace: row[3].ToString(),
+ Shard: row[4].ToString(),
TabletType: topodatapb.TabletType_PRIMARY,
})
}
diff --git a/go/vt/vttablet/tabletserver/twopc_test.go b/go/vt/vttablet/tabletserver/twopc_test.go
index cc9e987664c..0dc10266e8c 100644
--- a/go/vt/vttablet/tabletserver/twopc_test.go
+++ b/go/vt/vttablet/tabletserver/twopc_test.go
@@ -36,9 +36,9 @@ func TestReadAllRedo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Reuse code from tx_executor_test.
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+
tpc := tsv.te.twoPC
conn, err := tsv.qe.conns.Get(ctx, nil)
@@ -66,12 +66,14 @@ func TestReadAllRedo(t *testing.T) {
{Type: sqltypes.Int64},
{Type: sqltypes.Int64},
{Type: sqltypes.VarChar},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt01"),
+ sqltypes.NULL,
}},
})
prepared, failed, err = tpc.ReadAllRedo(ctx)
@@ -96,17 +98,20 @@ func TestReadAllRedo(t *testing.T) {
{Type: sqltypes.Int64},
{Type: sqltypes.Int64},
{Type: sqltypes.VarChar},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt01"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt02"),
+ sqltypes.NULL,
}},
})
prepared, failed, err = tpc.ReadAllRedo(ctx)
@@ -131,22 +136,26 @@ func TestReadAllRedo(t *testing.T) {
{Type: sqltypes.Int64},
{Type: sqltypes.Int64},
{Type: sqltypes.VarChar},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt01"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt02"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("dtid1"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt11"),
+ sqltypes.NULL,
}},
})
prepared, failed, err = tpc.ReadAllRedo(ctx)
@@ -175,37 +184,44 @@ func TestReadAllRedo(t *testing.T) {
{Type: sqltypes.Int64},
{Type: sqltypes.Int64},
{Type: sqltypes.VarChar},
+ {Type: sqltypes.Text},
},
Rows: [][]sqltypes.Value{{
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt01"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("dtid0"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt02"),
+ sqltypes.NULL,
}, {
sqltypes.NewVarBinary("dtid1"),
- sqltypes.NewVarBinary("Failed"),
+ sqltypes.NewInt64(RedoStateFailed),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt11"),
+ sqltypes.TestValue(sqltypes.Text, "error1"),
}, {
sqltypes.NewVarBinary("dtid2"),
- sqltypes.NewVarBinary("Failed"),
+ sqltypes.NewInt64(RedoStateFailed),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt21"),
+ sqltypes.TestValue(sqltypes.Text, "error2"),
}, {
sqltypes.NewVarBinary("dtid2"),
- sqltypes.NewVarBinary("Failed"),
+ sqltypes.NewInt64(RedoStateFailed),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt22"),
+ sqltypes.TestValue(sqltypes.Text, "error2"),
}, {
sqltypes.NewVarBinary("dtid3"),
sqltypes.NewInt64(RedoStatePrepared),
sqltypes.NewVarBinary("1"),
sqltypes.NewVarBinary("stmt31"),
+ sqltypes.NULL,
}},
})
prepared, failed, err = tpc.ReadAllRedo(ctx)
@@ -221,29 +237,27 @@ func TestReadAllRedo(t *testing.T) {
Queries: []string{"stmt31"},
Time: time.Unix(0, 1),
}}
- if !reflect.DeepEqual(prepared, want) {
- t.Errorf("ReadAllRedo: %s, want %s", jsonStr(prepared), jsonStr(want))
- }
+ utils.MustMatch(t, want, prepared)
wantFailed := []*tx.PreparedTx{{
Dtid: "dtid1",
Queries: []string{"stmt11"},
Time: time.Unix(0, 1),
+ Message: "error1",
}, {
Dtid: "dtid2",
Queries: []string{"stmt21", "stmt22"},
Time: time.Unix(0, 1),
+ Message: "error2",
}}
- if !reflect.DeepEqual(failed, wantFailed) {
- t.Errorf("ReadAllRedo failed): %s, want %s", jsonStr(failed), jsonStr(wantFailed))
- }
+ utils.MustMatch(t, wantFailed, failed)
}
func TestReadAllTransactions(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
+
tpc := tsv.te.twoPC
conn, err := tsv.qe.conns.Get(ctx, nil)
@@ -404,9 +418,8 @@ func jsonStr(v any) string {
func TestUnresolvedTransactions(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- _, tsv, db := newTestTxExecutor(t, ctx)
- defer db.Close()
- defer tsv.StopService()
+ _, tsv, db, closer := newTestTxExecutor(t, ctx)
+ defer closer()
conn, err := tsv.qe.conns.Get(ctx, nil)
require.NoError(t, err)
@@ -422,13 +435,14 @@ func TestUnresolvedTransactions(t *testing.T) {
}, {
name: "one unresolved transaction",
unresolvedTx: sqltypes.MakeTestResult(
- sqltypes.MakeTestFields("dtid|state|keyspace|shard",
- "VARBINARY|INT64|VARCHAR|VARCHAR"),
- "dtid0|1|ks01|shard01",
- "dtid0|1|ks01|shard02"),
+ sqltypes.MakeTestFields("dtid|state|time_created|keyspace|shard",
+ "VARBINARY|INT64|INT64|VARCHAR|VARCHAR"),
+ "dtid0|1|2|ks01|shard01",
+ "dtid0|1|2|ks01|shard02"),
expectedTx: []*querypb.TransactionMetadata{{
- Dtid: "dtid0",
- State: querypb.TransactionState_PREPARE,
+ Dtid: "dtid0",
+ State: querypb.TransactionState_PREPARE,
+ TimeCreated: 2,
Participants: []*querypb.Target{
{Keyspace: "ks01", Shard: "shard01", TabletType: topodatapb.TabletType_PRIMARY},
{Keyspace: "ks01", Shard: "shard02", TabletType: topodatapb.TabletType_PRIMARY},
@@ -436,21 +450,23 @@ func TestUnresolvedTransactions(t *testing.T) {
}, {
name: "two unresolved transaction",
unresolvedTx: sqltypes.MakeTestResult(
- sqltypes.MakeTestFields("dtid|state|keyspace|shard",
- "VARBINARY|INT64|VARCHAR|VARCHAR"),
- "dtid0|3|ks01|shard01",
- "dtid0|3|ks01|shard02",
- "dtid1|2|ks02|shard03",
- "dtid1|2|ks01|shard02"),
+ sqltypes.MakeTestFields("dtid|state|time_created|keyspace|shard",
+ "VARBINARY|INT64|INT64|VARCHAR|VARCHAR"),
+ "dtid0|3|1|ks01|shard01",
+ "dtid0|3|1|ks01|shard02",
+ "dtid1|2|2|ks02|shard03",
+ "dtid1|2|2|ks01|shard02"),
expectedTx: []*querypb.TransactionMetadata{{
- Dtid: "dtid0",
- State: querypb.TransactionState_COMMIT,
+ Dtid: "dtid0",
+ State: querypb.TransactionState_COMMIT,
+ TimeCreated: 1,
Participants: []*querypb.Target{
{Keyspace: "ks01", Shard: "shard01", TabletType: topodatapb.TabletType_PRIMARY},
{Keyspace: "ks01", Shard: "shard02", TabletType: topodatapb.TabletType_PRIMARY},
}}, {
- Dtid: "dtid1",
- State: querypb.TransactionState_ROLLBACK,
+ Dtid: "dtid1",
+ TimeCreated: 2,
+ State: querypb.TransactionState_ROLLBACK,
Participants: []*querypb.Target{
{Keyspace: "ks02", Shard: "shard03", TabletType: topodatapb.TabletType_PRIMARY},
{Keyspace: "ks01", Shard: "shard02", TabletType: topodatapb.TabletType_PRIMARY},
diff --git a/go/vt/vttablet/tabletserver/tx/api.go b/go/vt/vttablet/tabletserver/tx/api.go
index a392e530ffa..a7bc4389b89 100644
--- a/go/vt/vttablet/tabletserver/tx/api.go
+++ b/go/vt/vttablet/tabletserver/tx/api.go
@@ -21,45 +21,53 @@ import (
"strings"
"time"
+ "vitess.io/vitess/go/slice"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
)
type (
// ConnID as type int64
ConnID = int64
-
- //DTID as type string
+ // DTID as type string
DTID = string
-
- //EngineStateMachine is used to control the state the transactional engine -
- //whether new connections and/or transactions are allowed or not.
+ // EngineStateMachine is used to control the state the transactional engine -
+ // whether new connections and/or transactions are allowed or not.
EngineStateMachine interface {
Init() error
AcceptReadWrite() error
AcceptReadOnly() error
StopGently()
}
-
// ReleaseReason as type int
ReleaseReason int
-
- //Properties contains all information that is related to the currently running
- //transaction on the connection
+ // Properties contains all information that is related to the currently running
+ // transaction on the connection
Properties struct {
EffectiveCaller *vtrpcpb.CallerID
ImmediateCaller *querypb.VTGateCallerID
StartTime time.Time
EndTime time.Time
- Queries []string
+ Queries []Query
Autocommit bool
Conclusion string
LogToFile bool
Stats *servenv.TimingsWrapper
}
+
+ // Query contains the query and involved tables executed inside transaction.
+ // A savepoint is represented by having only the Savepoint field set.
+ // This is used to rollback to a specific savepoint.
+ // The query log on commit, does not need to store the savepoint.
+ Query struct {
+ Savepoint string
+ Sql string
+ Tables []string
+ }
)
const (
@@ -114,12 +122,57 @@ var txNames = map[ReleaseReason]string{
ConnRenewFail: "renewFail",
}
-// RecordQuery records the query against this transaction.
-func (p *Properties) RecordQuery(query string) {
+// RecordQueryDetail records the query and tables against this transaction.
+func (p *Properties) RecordQueryDetail(query string, tables []string) {
if p == nil {
return
}
- p.Queries = append(p.Queries, query)
+ p.Queries = append(p.Queries, Query{
+ Sql: query,
+ Tables: tables,
+ })
+}
+
+// RecordQueryDetail records the query and tables against this transaction.
+func (p *Properties) RecordSavePointDetail(savepoint string) {
+ if p == nil {
+ return
+ }
+ p.Queries = append(p.Queries, Query{
+ Savepoint: savepoint,
+ })
+}
+
+func (p *Properties) RollbackToSavepoint(savepoint string) error {
+ if p == nil {
+ return nil
+ }
+ for i, query := range p.Queries {
+ if query.Savepoint == savepoint {
+ p.Queries = p.Queries[:i]
+ return nil
+ }
+ }
+
+ return vterrors.VT13001(fmt.Sprintf("savepoint %s not found", savepoint))
+}
+
+// RecordQuery records the query and extract tables against this transaction.
+func (p *Properties) RecordQuery(query string, parser *sqlparser.Parser) {
+ if p == nil {
+ return
+ }
+ stmt, err := parser.Parse(query)
+ if err != nil {
+ // This should neven happen, but if it does,
+ // we would not be able to block cut-overs on this query.
+ return
+ }
+ tables := sqlparser.ExtractAllTables(stmt)
+ p.Queries = append(p.Queries, Query{
+ Sql: query,
+ Tables: tables,
+ })
}
// InTransaction returns true as soon as this struct is not nil
@@ -134,10 +187,11 @@ func (p *Properties) String(sanitize bool, parser *sqlparser.Parser) string {
printQueries := func() string {
sb := strings.Builder{}
for _, query := range p.Queries {
+ sql := query.Sql
if sanitize {
- query, _ = parser.RedactSQLQuery(query)
+ sql, _ = parser.RedactSQLQuery(sql)
}
- sb.WriteString(query)
+ sb.WriteString(sql)
sb.WriteString(";")
}
return sb.String()
@@ -154,3 +208,12 @@ func (p *Properties) String(sanitize bool, parser *sqlparser.Parser) string {
printQueries(),
)
}
+
+func (p *Properties) GetQueries() []Query {
+ if p == nil {
+ return nil
+ }
+ return slice.Filter(p.Queries, func(q Query) bool {
+ return q.Sql != ""
+ })
+}
diff --git a/go/vt/vttablet/tabletserver/tx/api_test.go b/go/vt/vttablet/tabletserver/tx/api_test.go
new file mode 100644
index 00000000000..cefb04c0391
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/tx/api_test.go
@@ -0,0 +1,62 @@
+/*
+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 tx
+
+import (
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/test/utils"
+)
+
+/*
+ TestRollbackToSavePointQueryDetails tests the rollback to savepoint query details
+
+s1
+q1
+s2
+r1
+q2
+q3
+s3
+s4
+q4
+q5
+r2 -- error
+r4
+*/
+func TestRollbackToSavePointQueryDetails(t *testing.T) {
+ p := &Properties{}
+ p.RecordSavePointDetail("s1")
+ p.RecordQueryDetail("select 1", nil)
+ p.RecordSavePointDetail("s2")
+ require.NoError(t, p.RollbackToSavepoint("s1"))
+ p.RecordQueryDetail("select 2", nil)
+ p.RecordQueryDetail("select 3", nil)
+ p.RecordSavePointDetail("s3")
+ p.RecordSavePointDetail("s4")
+ p.RecordQueryDetail("select 4", nil)
+ p.RecordQueryDetail("select 5", nil)
+ require.ErrorContains(t, p.RollbackToSavepoint("s2"), "savepoint s2 not found")
+ require.NoError(t, p.RollbackToSavepoint("s4"))
+
+ utils.MustMatch(t, p.GetQueries(), []Query{
+ {Sql: "select 2"},
+ {Sql: "select 3"},
+ })
+}
diff --git a/go/vt/vttablet/tabletserver/tx/twopc.go b/go/vt/vttablet/tabletserver/tx/twopc.go
index 56cfbd1a51f..6412fc53b4d 100644
--- a/go/vt/vttablet/tabletserver/tx/twopc.go
+++ b/go/vt/vttablet/tabletserver/tx/twopc.go
@@ -36,4 +36,5 @@ type PreparedTx struct {
Dtid string
Queries []string
Time time.Time
+ Message string
}
diff --git a/go/vt/vttablet/tabletserver/tx_engine.go b/go/vt/vttablet/tabletserver/tx_engine.go
index 33e22e321bc..d0ca4ec498c 100644
--- a/go/vt/vttablet/tabletserver/tx_engine.go
+++ b/go/vt/vttablet/tabletserver/tx_engine.go
@@ -22,15 +22,16 @@ import (
"sync"
"time"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/pools/smartconnpool"
"vitess.io/vitess/go/timer"
"vitess.io/vitess/go/trace"
- "vitess.io/vitess/go/vt/concurrency"
"vitess.io/vitess/go/vt/dtids"
"vitess.io/vitess/go/vt/log"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/servenv"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -75,9 +76,14 @@ type TxEngine struct {
// transition while creating new transactions
beginRequests sync.WaitGroup
- twopcEnabled bool
+ // twopcEnabled is the flag value of whether the user has enabled twopc or not.
+ twopcEnabled bool
+ // twopcAllowed is whether it is safe to allow two pc transactions or not.
+ // There are multiple reasons to disallow TwoPC:
+ // 1. If the primary tablet doesn't run with semi-sync we set this to false, and disallow any prepared calls.
+ // 2. TabletControls have been set in the tablet record, and Query service is going to be disabled.
+ twopcAllowed []bool
shutdownGracePeriod time.Duration
- coordinatorAddress string
abandonAge time.Duration
ticks *timer.Timer
@@ -87,10 +93,17 @@ type TxEngine struct {
txPool *TxPool
preparedPool *TxPreparedPool
twoPC *TwoPC
- twoPCReady sync.WaitGroup
dxNotify func()
}
+// TwoPC can be disallowed for various reasons. These are the reasons we keep track off
+// when deciding if new prepared transactions should be allowed or not.
+const (
+ TwoPCAllowed_SemiSync = iota
+ TwoPCAllowed_TabletControls
+ TwoPCAllowed_Len
+)
+
// NewTxEngine creates a new TxEngine.
func NewTxEngine(env tabletenv.Env, dxNotifier func()) *TxEngine {
config := env.Config()
@@ -101,14 +114,21 @@ func NewTxEngine(env tabletenv.Env, dxNotifier func()) *TxEngine {
}
limiter := txlimiter.New(env)
te.txPool = NewTxPool(env, limiter)
- te.twopcEnabled = config.TwoPCEnable
- if te.twopcEnabled {
- if config.TwoPCAbandonAge <= 0 {
- log.Error("2PC abandon age not specified: Disabling 2PC")
- te.twopcEnabled = false
- }
+ // We initially allow twoPC (handles vttablet restarts).
+ // We will disallow them for a few reasons -
+ // 1. When a new tablet is promoted if semi-sync is turned off.
+ // 2. TabletControls have been set by a Resharding workflow.
+ te.twopcAllowed = make([]bool, TwoPCAllowed_Len)
+ for idx := range te.twopcAllowed {
+ te.twopcAllowed[idx] = true
+ }
+ te.twopcEnabled = true
+ if config.TwoPCAbandonAge <= 0 {
+ log.Error("2PC abandon age not specified: Disabling 2PC")
+ te.twopcEnabled = false
}
- te.abandonAge = config.TwoPCAbandonAge.Get()
+
+ te.abandonAge = config.TwoPCAbandonAge
te.ticks = timer.NewTimer(te.abandonAge / 2)
// Set the prepared pool capacity to something lower than
@@ -116,7 +136,7 @@ func NewTxEngine(env tabletenv.Env, dxNotifier func()) *TxEngine {
// perform metadata state change operations. Without this,
// the system can deadlock if all connections get moved to
// the TxPreparedPool.
- te.preparedPool = NewTxPreparedPool(config.TxPool.Size - 2)
+ te.preparedPool = NewTxPreparedPool(config.TxPool.Size-2, te.twopcEnabled)
readPool := connpool.NewPool(env, "TxReadPool", tabletenv.ConnPoolConfig{
Size: 3,
IdleTimeout: env.Config().TxPool.IdleTimeout,
@@ -128,9 +148,6 @@ func NewTxEngine(env tabletenv.Env, dxNotifier func()) *TxEngine {
}
// AcceptReadWrite will start accepting all transactions.
-// If transitioning from RO mode, transactions are rolled
-// back before accepting new transactions. This is to allow
-// for 2PC state to be correctly initialized.
func (te *TxEngine) AcceptReadWrite() {
te.transition(AcceptingReadAndWrite)
}
@@ -149,37 +166,70 @@ func (te *TxEngine) transition(state txEngineState) {
}
log.Infof("TxEngine transition: %v", state)
- switch te.state {
- case AcceptingReadOnly, AcceptingReadAndWrite:
+
+ // When we are transitioning from read write state, we should close all transactions.
+ if te.state == AcceptingReadAndWrite {
te.shutdownLocked()
- case NotServing:
- // No special action.
}
te.state = state
+ if te.twopcEnabled && te.state == AcceptingReadAndWrite {
+ // If the prepared pool is not open, then we need to redo the prepared transactions
+ // before we open the transaction engine to accept new writes.
+ // This check is required because during a Promotion, we would have already setup the prepared pool
+ // and redid the prepared transactions when we turn super_read_only off. So we don't need to do it again.
+ if !te.preparedPool.IsOpen() {
+ // We need to redo prepared transactions here to handle vttablet restarts.
+ // If MySQL continues to work fine, then we won't end up redoing the prepared transactions as part of any RPC call
+ // since VTOrc won't call `UndoDemotePrimary`. We need to do them as part of this transition.
+ te.redoPreparedTransactionsLocked()
+ }
+ te.startTransactionWatcher()
+ }
te.txPool.Open(te.env.Config().DB.AppWithDB(), te.env.Config().DB.DbaWithDB(), te.env.Config().DB.AppDebugWithDB())
+}
- if te.twopcEnabled && te.state == AcceptingReadAndWrite {
- // Set the preparedPool to start accepting connections.
- te.preparedPool.shutdown = false
- // If there are errors, we choose to raise an alert and
- // continue anyway. Serving traffic is considered more important
- // than blocking everything for the sake of a few transactions.
- // We do this async; so we do not end up blocking writes on
- // failover for our setup tasks if using semi-sync replication.
- te.twoPCReady.Add(1)
- go func() {
- defer te.twoPCReady.Done()
- if err := te.twoPC.Open(te.env.Config().DB); err != nil {
- te.env.Stats().InternalErrors.Add("TwopcOpen", 1)
- log.Errorf("Could not open TwoPC engine: %v", err)
- }
- if err := te.prepareFromRedo(); err != nil {
- te.env.Stats().InternalErrors.Add("TwopcResurrection", 1)
- log.Errorf("Could not prepare transactions: %v", err)
- }
- te.startTransactionWatcher()
- }()
+// RedoPreparedTransactions acquires the state lock and calls redoPreparedTransactionsLocked.
+func (te *TxEngine) RedoPreparedTransactions() {
+ if te.twopcEnabled {
+ te.stateLock.Lock()
+ defer te.stateLock.Unlock()
+ te.redoPreparedTransactionsLocked()
+ }
+}
+
+// redoPreparedTransactionsLocked redoes the prepared transactions.
+// If there are errors, we choose to raise an alert and
+// continue anyway. Serving traffic is considered more important
+// than blocking everything for the sake of a few transactions.
+// We do this async; so we do not end up blocking writes on
+// failover for our setup tasks if using semi-sync replication.
+func (te *TxEngine) redoPreparedTransactionsLocked() {
+ oldState := te.state
+ // We shutdown to ensure no other writes are in progress.
+ te.shutdownLocked()
+ defer func() {
+ te.state = oldState
+ }()
+
+ if err := te.twoPC.Open(te.env.Config().DB); err != nil {
+ te.env.Stats().InternalErrors.Add("TwopcOpen", 1)
+ log.Errorf("Could not open TwoPC engine: %v", err)
+ return
+ }
+
+ // We should only open the prepared pool and the transaction pool if the opening of twoPC pool is successful.
+ // We use the prepared pool being open to know if we need to redo the prepared transactions.
+ // So if we open the prepared pool and then opening of twoPC fails, we will never end up opening the twoPC pool at all!
+ // This is why opening prepared pool after the twoPC pool is crucial for correctness.
+ te.preparedPool.Open()
+ // We have to defer opening the transaction pool because we call shutdown in the beginning that closes it.
+ // We want to open the transaction pool after the prepareFromRedo has run. Also, we want this to run even if that fails.
+ defer te.txPool.Open(te.env.Config().DB.AppWithDB(), te.env.Config().DB.DbaWithDB(), te.env.Config().DB.AppDebugWithDB())
+
+ if err := te.prepareFromRedo(); err != nil {
+ te.env.Stats().InternalErrors.Add("TwopcResurrection", 1)
+ log.Errorf("Could not prepare transactions: %v", err)
}
}
@@ -219,7 +269,7 @@ func (te *TxEngine) isTxPoolAvailable(addToWaitGroup func(int)) error {
// statement(s) used to execute the begin (if any).
//
// Subsequent statements can access the connection through the transaction id.
-func (te *TxEngine) Begin(ctx context.Context, savepointQueries []string, reservedID int64, setting *smartconnpool.Setting, options *querypb.ExecuteOptions) (int64, string, string, error) {
+func (te *TxEngine) Begin(ctx context.Context, reservedID int64, setting *smartconnpool.Setting, options *querypb.ExecuteOptions) (int64, string, string, error) {
span, ctx := trace.NewSpan(ctx, "TxEngine.Begin")
defer span.Finish()
@@ -234,7 +284,7 @@ func (te *TxEngine) Begin(ctx context.Context, savepointQueries []string, reserv
}
defer te.beginRequests.Done()
- conn, beginSQL, sessionStateChanges, err := te.txPool.Begin(ctx, options, te.state == AcceptingReadOnly, reservedID, savepointQueries, setting)
+ conn, beginSQL, sessionStateChanges, err := te.txPool.Begin(ctx, options, te.state == AcceptingReadOnly, reservedID, setting)
if err != nil {
return 0, "", "", err
}
@@ -306,11 +356,6 @@ func (te *TxEngine) shutdownLocked() {
te.stateLock.Lock()
log.Infof("TxEngine - state lock acquired again")
- // Shut down functions are idempotent.
- // No need to check if 2pc is enabled.
- log.Infof("TxEngine - stop watchdog")
- te.stopTransactionWatcher()
-
poolEmpty := make(chan bool)
rollbackDone := make(chan bool)
// This goroutine decides if transactions have to be
@@ -333,13 +378,6 @@ func (te *TxEngine) shutdownLocked() {
// connections.
te.txPool.scp.ShutdownNonTx()
if te.shutdownGracePeriod <= 0 {
- // No grace period was specified. Wait indefinitely for transactions to be concluded.
- // TODO(sougou): invoking rollbackPrepared is incorrect here. Prepared statements should
- // actually be rolled back last. But this will cause the shutdown to hang because the
- // tx pool will never become empty, because the prepared pool is holding on to connections
- // from the tx pool. But we plan to deprecate this approach to 2PC. So, this
- // should eventually be deleted.
- te.rollbackPrepared()
log.Info("No grace period specified: performing normal wait.")
return
}
@@ -354,6 +392,9 @@ func (te *TxEngine) shutdownLocked() {
log.Info("Transactions completed before grace period: shutting down.")
}
}()
+ // It is important to note, that we aren't rolling back prepared transactions here.
+ // That is happneing in the same place where we are killing queries. This will block
+ // until either all prepared transactions get resolved or rollbacked.
log.Infof("TxEngine - waiting for empty txPool")
te.txPool.WaitForEmpty()
// If the goroutine is still running, signal that it can exit.
@@ -362,10 +403,19 @@ func (te *TxEngine) shutdownLocked() {
log.Infof("TxEngine - making sure the goroutine has returned")
<-rollbackDone
+ // We stop the transaction watcher so late, because if the user isn't running
+ // with any shutdown grace period, we still want the watcher to run while we are waiting
+ // for resolving transactions.
+ log.Infof("TxEngine - stop transaction watcher")
+ te.stopTransactionWatcher()
+
+ // Mark the prepared pool closed.
log.Infof("TxEngine - closing the txPool")
te.txPool.Close()
log.Infof("TxEngine - closing twoPC")
te.twoPC.Close()
+ log.Infof("TxEngine - closing the prepared pool")
+ te.preparedPool.Close()
log.Infof("TxEngine - finished shutdownLocked")
}
@@ -375,74 +425,164 @@ func (te *TxEngine) shutdownLocked() {
// to ensure there are no future collisions.
func (te *TxEngine) prepareFromRedo() error {
ctx := tabletenv.LocalContext()
- var allErr concurrency.AllErrorRecorder
- prepared, failed, err := te.twoPC.ReadAllRedo(ctx)
- if err != nil {
- return err
+
+ prepared, failed, readErr := te.twoPC.ReadAllRedo(ctx)
+ if readErr != nil {
+ return readErr
}
- maxid := int64(0)
-outer:
+ var (
+ maxID = int64(0)
+ preparedCounter = 0
+ failedCounter = len(failed)
+ allErrs []error
+ )
+
+ // While going through the prepared transaction.
+ // We will extract the transaction ID from the dtid and
+ // update the last transaction ID to max value to avoid any collision with the new transactions.
+
for _, preparedTx := range prepared {
- txid, err := dtids.TransactionID(preparedTx.Dtid)
- if err != nil {
- log.Errorf("Error extracting transaction ID from dtid: %v", err)
- }
- if txid > maxid {
- maxid = txid
+ txID, _ := dtids.TransactionID(preparedTx.Dtid)
+ if txID > maxID {
+ maxID = txID
}
- conn, _, _, err := te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+
+ prepFailed, err := te.prepareTx(ctx, preparedTx)
if err != nil {
- allErr.RecordError(err)
- continue
- }
- for _, stmt := range preparedTx.Queries {
- conn.TxProperties().RecordQuery(stmt)
- _, err := conn.Exec(ctx, stmt, 1, false)
- if err != nil {
- allErr.RecordError(err)
- te.txPool.RollbackAndRelease(ctx, conn)
- continue outer
+ allErrs = append(allErrs, vterrors.Wrapf(err, "dtid - %v", preparedTx.Dtid))
+ if prepFailed {
+ failedCounter++
+ te.env.Stats().RedoPreparedFail.Add("NonRetryable", 1)
+ } else {
+ te.env.Stats().RedoPreparedFail.Add("Retryable", 1)
}
- }
- // We should not use the external Prepare because
- // we don't want to write again to the redo log.
- err = te.preparedPool.Put(conn, preparedTx.Dtid)
- if err != nil {
- allErr.RecordError(err)
- continue
+ } else {
+ preparedCounter++
}
}
+
for _, preparedTx := range failed {
- txid, err := dtids.TransactionID(preparedTx.Dtid)
+ txID, _ := dtids.TransactionID(preparedTx.Dtid)
+ if txID > maxID {
+ maxID = txID
+ }
+ te.preparedPool.SetFailed(preparedTx.Dtid)
+ }
+
+ te.txPool.AdjustLastID(maxID)
+ log.Infof("TwoPC: Prepared %d transactions, and registered %d failures.", preparedCounter, failedCounter)
+ return vterrors.Aggregate(allErrs)
+}
+
+func (te *TxEngine) prepareTx(ctx context.Context, preparedTx *tx.PreparedTx) (failed bool, err error) {
+ defer func() {
if err != nil {
- log.Errorf("Error extracting transaction ID from dtid: %v", err)
+ failed = te.checkErrorAndMarkFailed(ctx, preparedTx.Dtid, err, "TwopcPrepareRedo")
}
- if txid > maxid {
- maxid = txid
+ }()
+
+ // We need to check whether the first query is a SET query or not.
+ // If it is then we need to run it before we begin the transaction because
+ // some connection settings can't be modified after a transaction has started
+ // For example -
+ // mysql> begin;
+ // Query OK, 0 rows affected (0.00 sec)
+ // mysql> set @@transaction_isolation="read-committed";
+ // ERROR 1568 (25001): Transaction characteristics can't be changed while a transaction is in progress.
+ var settingsQuery string
+ firstQuery := preparedTx.Queries[0]
+ if sqlparser.Preview(firstQuery) == sqlparser.StmtSet {
+ settingsQuery = firstQuery
+ preparedTx.Queries = preparedTx.Queries[1:]
+ }
+
+ // We need to redo the prepared transactions using a dba user because MySQL might still be in read only mode.
+ var conn *StatefulConnection
+ if conn, err = te.beginNewDbaConnection(ctx, settingsQuery); err != nil {
+ return
+ }
+
+ for _, stmt := range preparedTx.Queries {
+ conn.TxProperties().RecordQuery(stmt, te.env.Environment().Parser())
+ if _, err = conn.Exec(ctx, stmt, 1, false); err != nil {
+ te.txPool.RollbackAndRelease(ctx, conn)
+ return
}
- te.preparedPool.SetFailed(preparedTx.Dtid)
}
- te.txPool.AdjustLastID(maxid)
- log.Infof("TwoPC: Prepared %d transactions, and registered %d failures.", len(prepared), len(failed))
- return allErr.Error()
+ // We should not use the external Prepare because
+ // we don't want to write again to the redo log.
+ err = te.preparedPool.Put(conn, preparedTx.Dtid)
+ return
}
-// shutdownTransactions rolls back all open transactions
-// including the prepared ones.
-// This is used for transitioning from a primary to a non-primary
-// serving type.
+// checkErrorAndMarkFailed check that the error is retryable or non-retryable error.
+// If it is a non-retryable error than it marks the dtid as failed in the prepared pool,
+// increments the InternalErrors counter, and also changes the state of the transaction in the redo log as failed.
+func (te *TxEngine) checkErrorAndMarkFailed(ctx context.Context, dtid string, receivedErr error, metricName string) (fail bool) {
+ state := RedoStateFailed
+ if isRetryableError(receivedErr) {
+ log.Infof("retryable error for dtid: %s", dtid)
+ state = RedoStatePrepared
+ } else {
+ fail = true
+ te.env.Stats().InternalErrors.Add(metricName, 1)
+ te.preparedPool.SetFailed(dtid)
+ }
+
+ // Update the state of the transaction in the redo log.
+ // Retryable Error: Update the message with error message.
+ // Non-retryable Error: Along with message, update the state as RedoStateFailed.
+ conn, _, _, err := te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
+ if err != nil {
+ log.Errorf("markFailed: Begin failed for dtid %s: %v", dtid, err)
+ return
+ }
+ defer te.txPool.RollbackAndRelease(ctx, conn)
+
+ if err = te.twoPC.UpdateRedo(ctx, conn, dtid, state, receivedErr.Error()); err != nil {
+ log.Errorf("markFailed: UpdateRedo failed for dtid %s: %v", dtid, err)
+ return
+ }
+
+ if _, err = te.txPool.Commit(ctx, conn); err != nil {
+ log.Errorf("markFailed: Commit failed for dtid %s: %v", dtid, err)
+ }
+ return
+}
+
+func isRetryableError(err error) bool {
+ switch vterrors.Code(err) {
+ case vtrpcpb.Code_OK,
+ vtrpcpb.Code_DEADLINE_EXCEEDED,
+ vtrpcpb.Code_CANCELED,
+ vtrpcpb.Code_UNAVAILABLE:
+ return true
+ case vtrpcpb.Code_UNKNOWN:
+ // If the error is unknown, convert to SQL Error.
+ sqlErr := sqlerror.NewSQLErrorFromError(err)
+ // Connection errors are retryable
+ return sqlerror.IsConnErr(sqlErr)
+ default:
+ return false
+ }
+}
+
+// shutdownTransactions rolls back all open transactions that are idol.
+// These are transactions that are open but no write is executing on them right now.
+// By definition, prepared transactions aren't part of them since these are transactions on which
+// the user has issued a commit command. These transactions are rollbacked elsewhere when we kill all writes.
+// This is used for transitioning from a primary to a non-primary serving type.
func (te *TxEngine) shutdownTransactions() {
- te.rollbackPrepared()
ctx := tabletenv.LocalContext()
- // The order of rollbacks is currently not material because
- // we don't allow new statements or commits during
- // this function. In case of any such change, this will
- // have to be revisited.
te.txPool.Shutdown(ctx)
}
-func (te *TxEngine) rollbackPrepared() {
+// RollbackPrepared rollbacks all the prepared transactions.
+// This should only be called after we are certain no other writes are in progress.
+// If there were some other conflicting write in progress that hadn't been killed, then it could potentially go through
+// and cause data corruption since we won't be able to prepare the transaction again.
+func (te *TxEngine) RollbackPrepared() {
ctx := tabletenv.LocalContext()
for _, conn := range te.preparedPool.FetchAllForRollback() {
te.txPool.Rollback(ctx, conn)
@@ -457,14 +597,13 @@ func (te *TxEngine) startTransactionWatcher() {
ctx, cancel := context.WithTimeout(tabletenv.LocalContext(), te.abandonAge/4)
defer cancel()
- // Raise alerts on prepares that have been unresolved for too long.
- // Use 5x abandonAge to give opportunity for transaction coordinator to resolve these redo logs.
- count, err := te.twoPC.CountUnresolvedRedo(ctx, time.Now().Add(-te.abandonAge*5))
+ // Track unresolved redo logs.
+ count, err := te.twoPC.CountUnresolvedRedo(ctx, time.Now().Add(-te.abandonAge))
if err != nil {
te.env.Stats().InternalErrors.Add("RedoWatcherFail", 1)
log.Errorf("Error reading prepared transactions: %v", err)
}
- te.env.Stats().Unresolved.Set("Prepares", count)
+ te.env.Stats().Unresolved.Set("ResourceManager", count)
// Notify lingering distributed transactions.
count, err = te.twoPC.CountUnresolvedTransaction(ctx, time.Now().Add(-te.abandonAge))
@@ -473,6 +612,7 @@ func (te *TxEngine) startTransactionWatcher() {
log.Errorf("Error reading unresolved transactions: %v", err)
return
}
+ te.env.Stats().Unresolved.Set("MetadataManager", count)
if count > 0 {
te.dxNotify()
}
@@ -485,7 +625,7 @@ func (te *TxEngine) stopTransactionWatcher() {
}
// ReserveBegin creates a reserved connection, and in it opens a transaction
-func (te *TxEngine) ReserveBegin(ctx context.Context, options *querypb.ExecuteOptions, preQueries []string, savepointQueries []string) (int64, string, error) {
+func (te *TxEngine) ReserveBegin(ctx context.Context, options *querypb.ExecuteOptions, preQueries []string) (int64, string, error) {
span, ctx := trace.NewSpan(ctx, "TxEngine.ReserveBegin")
defer span.Finish()
err := te.isTxPoolAvailable(te.beginRequests.Add)
@@ -499,7 +639,7 @@ func (te *TxEngine) ReserveBegin(ctx context.Context, options *querypb.ExecuteOp
return 0, "", err
}
defer conn.UnlockUpdateTime()
- _, sessionStateChanges, err := te.txPool.begin(ctx, options, te.state == AcceptingReadOnly, conn, savepointQueries)
+ _, sessionStateChanges, err := te.txPool.begin(ctx, options, te.state == AcceptingReadOnly, conn)
if err != nil {
conn.Close()
conn.Release(tx.ConnInitFail)
@@ -581,3 +721,39 @@ func (te *TxEngine) Release(connID int64) error {
return nil
}
+
+// beginNewDbaConnection gets a new dba connection and starts a transaction in it.
+// This should only be used to redo prepared transactions. All the other writes should use the normal pool.
+func (te *TxEngine) beginNewDbaConnection(ctx context.Context, settingsQuery string) (*StatefulConnection, error) {
+ dbConn, err := connpool.NewConn(ctx, te.env.Config().DB.DbaWithDB(), nil, nil, te.env)
+ if err != nil {
+ return nil, err
+ }
+
+ // If we have a settings query that we need to apply, we do that before starting the transaction.
+ if settingsQuery != "" {
+ if _, err = dbConn.ExecOnce(ctx, settingsQuery, 1, false); err != nil {
+ return nil, err
+ }
+ }
+
+ sc := &StatefulConnection{
+ dbConn: &connpool.PooledConn{
+ Conn: dbConn,
+ },
+ env: te.env,
+ }
+
+ _, _, err = te.txPool.begin(ctx, nil, false, sc)
+ return sc, err
+}
+
+// IsTwoPCAllowed checks if TwoPC is allowed.
+func (te *TxEngine) IsTwoPCAllowed() bool {
+ for _, allowed := range te.twopcAllowed {
+ if !allowed {
+ return false
+ }
+ }
+ return true
+}
diff --git a/go/vt/vttablet/tabletserver/tx_engine_test.go b/go/vt/vttablet/tabletserver/tx_engine_test.go
index 95057d754fb..cba7bf86e8f 100644
--- a/go/vt/vttablet/tabletserver/tx_engine_test.go
+++ b/go/vt/vttablet/tabletserver/tx_engine_test.go
@@ -25,7 +25,10 @@ import (
"testing"
"time"
+ "vitess.io/vitess/go/mysql/sqlerror"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
"github.com/stretchr/testify/assert"
@@ -59,11 +62,11 @@ func TestTxEngineClose(t *testing.T) {
// Normal close with timeout wait.
te.AcceptReadWrite()
- c, beginSQL, _, err := te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, beginSQL, _, err := te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
require.Equal(t, "begin", beginSQL)
c.Unlock()
- c, beginSQL, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, beginSQL, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
require.Equal(t, "begin", beginSQL)
c.Unlock()
@@ -75,7 +78,7 @@ func TestTxEngineClose(t *testing.T) {
// Immediate close.
te.AcceptReadOnly()
- c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
if err != nil {
t.Fatal(err)
}
@@ -87,7 +90,7 @@ func TestTxEngineClose(t *testing.T) {
// Normal close with short grace period.
te.shutdownGracePeriod = 25 * time.Millisecond
te.AcceptReadWrite()
- c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
c.Unlock()
start = time.Now()
@@ -98,7 +101,7 @@ func TestTxEngineClose(t *testing.T) {
// Normal close with short grace period, but pool gets empty early.
te.shutdownGracePeriod = 25 * time.Millisecond
te.AcceptReadWrite()
- c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
c.Unlock()
go func() {
@@ -114,7 +117,7 @@ func TestTxEngineClose(t *testing.T) {
// Immediate close, but connection is in use.
te.AcceptReadOnly()
- c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ c, _, _, err = te.txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
go func() {
time.Sleep(100 * time.Millisecond)
@@ -135,7 +138,7 @@ func TestTxEngineClose(t *testing.T) {
te.AcceptReadWrite()
_, err = te.Reserve(ctx, &querypb.ExecuteOptions{}, 0, nil)
require.NoError(t, err)
- _, _, err = te.ReserveBegin(ctx, &querypb.ExecuteOptions{}, nil, nil)
+ _, _, err = te.ReserveBegin(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
start = time.Now()
te.Close()
@@ -156,11 +159,11 @@ func TestTxEngineBegin(t *testing.T) {
for _, exec := range []func() (int64, string, error){
func() (int64, string, error) {
- tx, _, schemaStateChanges, err := te.Begin(ctx, nil, 0, nil, &querypb.ExecuteOptions{})
+ tx, _, schemaStateChanges, err := te.Begin(ctx, 0, nil, &querypb.ExecuteOptions{})
return tx, schemaStateChanges, err
},
func() (int64, string, error) {
- return te.ReserveBegin(ctx, &querypb.ExecuteOptions{}, nil, nil)
+ return te.ReserveBegin(ctx, &querypb.ExecuteOptions{}, nil)
},
} {
te.AcceptReadOnly()
@@ -201,7 +204,7 @@ func TestTxEngineRenewFails(t *testing.T) {
te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"), nil)
te.AcceptReadOnly()
options := &querypb.ExecuteOptions{}
- connID, _, err := te.ReserveBegin(ctx, options, nil, nil)
+ connID, _, err := te.ReserveBegin(ctx, options, nil)
require.NoError(t, err)
conn, err := te.txPool.GetAndLock(connID, "for test")
@@ -556,7 +559,7 @@ func startTx(te *TxEngine, writeTransaction bool) error {
} else {
options.TransactionIsolation = querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY
}
- _, _, _, err := te.Begin(context.Background(), nil, 0, nil, options)
+ _, _, _, err := te.Begin(context.Background(), 0, nil, options)
return err
}
@@ -574,7 +577,7 @@ func TestTxEngineFailReserve(t *testing.T) {
_, err := te.Reserve(ctx, options, 0, nil)
assert.EqualError(t, err, "tx engine can't accept new connections in state NotServing")
- _, _, err = te.ReserveBegin(ctx, options, nil, nil)
+ _, _, err = te.ReserveBegin(ctx, options, nil)
assert.EqualError(t, err, "tx engine can't accept new connections in state NotServing")
te.AcceptReadOnly()
@@ -583,14 +586,14 @@ func TestTxEngineFailReserve(t *testing.T) {
_, err = te.Reserve(ctx, options, 0, []string{"dummy_query"})
assert.EqualError(t, err, "unknown error: failed executing dummy_query (errno 1105) (sqlstate HY000) during query: dummy_query")
- _, _, err = te.ReserveBegin(ctx, options, []string{"dummy_query"}, nil)
+ _, _, err = te.ReserveBegin(ctx, options, []string{"dummy_query"})
assert.EqualError(t, err, "unknown error: failed executing dummy_query (errno 1105) (sqlstate HY000) during query: dummy_query")
nonExistingID := int64(42)
_, err = te.Reserve(ctx, options, nonExistingID, nil)
assert.EqualError(t, err, "transaction 42: not found (potential transaction timeout)")
- txID, _, _, err := te.Begin(ctx, nil, 0, nil, options)
+ txID, _, _, err := te.Begin(ctx, 0, nil, options)
require.NoError(t, err)
conn, err := te.txPool.GetAndLock(txID, "for test")
require.NoError(t, err)
@@ -603,3 +606,203 @@ func TestTxEngineFailReserve(t *testing.T) {
require.Error(t, err)
assert.Zero(t, connID)
}
+
+func TestCheckReceivedError(t *testing.T) {
+ db := setUpQueryExecutorTest(t)
+ defer db.Close()
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest")
+ env.Config().TwoPCAbandonAge = 5 * time.Second
+ te := NewTxEngine(env, nil)
+ te.AcceptReadWrite()
+
+ tcases := []struct {
+ receivedErr error
+ retryable bool
+ expQuery string
+ }{{
+ receivedErr: vterrors.New(vtrpcpb.Code_DEADLINE_EXCEEDED, "deadline exceeded"),
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = 'deadline exceeded' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "invalid argument"),
+ retryable: false,
+ expQuery: `update _vt.redo_state set state = 0, message = 'invalid argument' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: sqlerror.NewSQLError(sqlerror.ERLockDeadlock, sqlerror.SSLockDeadlock, "Deadlock found when trying to get lock; try restarting transaction"),
+ retryable: false,
+ expQuery: `update _vt.redo_state set state = 0, message = 'Deadlock found when trying to get lock; try restarting transaction (errno 1213) (sqlstate 40001)' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: context.DeadlineExceeded,
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = 'context deadline exceeded' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: context.Canceled,
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = 'context canceled' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: sqlerror.NewSQLError(sqlerror.CRServerLost, sqlerror.SSUnknownSQLState, "Lost connection to MySQL server during query"),
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = 'Lost connection to MySQL server during query (errno 2013) (sqlstate HY000)' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: sqlerror.NewSQLError(sqlerror.CRMalformedPacket, sqlerror.SSUnknownSQLState, "Malformed packet"),
+ retryable: false,
+ expQuery: `update _vt.redo_state set state = 0, message = 'Malformed packet (errno 2027) (sqlstate HY000)' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: sqlerror.NewSQLError(sqlerror.CRServerGone, sqlerror.SSUnknownSQLState, "Server has gone away"),
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = 'Server has gone away (errno 2006) (sqlstate HY000)' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: vterrors.New(vtrpcpb.Code_ABORTED, "Row count exceeded"),
+ retryable: false,
+ expQuery: `update _vt.redo_state set state = 0, message = 'Row count exceeded' where dtid = _binary'aa'`,
+ }, {
+ receivedErr: errors.New("(errno 2013) (sqlstate HY000) lost connection"),
+ retryable: true,
+ expQuery: `update _vt.redo_state set state = 1, message = '(errno 2013) (sqlstate HY000) lost connection' where dtid = _binary'aa'`,
+ }}
+
+ for _, tc := range tcases {
+ t.Run(tc.receivedErr.Error(), func(t *testing.T) {
+ if tc.expQuery != "" {
+ db.AddQuery(tc.expQuery, &sqltypes.Result{})
+ }
+ nonRetryable := te.checkErrorAndMarkFailed(context.Background(), "aa", tc.receivedErr, "")
+ require.NotEqual(t, tc.retryable, nonRetryable)
+ if !tc.retryable {
+ require.Equal(t, errPrepFailed, te.preparedPool.reserved["aa"])
+ }
+ delete(te.preparedPool.reserved, "aa")
+ })
+ }
+}
+
+func TestIsTwoPCAllowed(t *testing.T) {
+ testcases := []struct {
+ semiSyncAllowed bool
+ tabletControllsAllowed bool
+ wantAllowed bool
+ }{
+ {
+ semiSyncAllowed: true,
+ tabletControllsAllowed: true,
+ wantAllowed: true,
+ },
+ {
+ semiSyncAllowed: false,
+ tabletControllsAllowed: true,
+ wantAllowed: false,
+ },
+ {
+ semiSyncAllowed: true,
+ tabletControllsAllowed: false,
+ wantAllowed: false,
+ },
+ {
+ semiSyncAllowed: false,
+ tabletControllsAllowed: false,
+ wantAllowed: false,
+ },
+ }
+
+ for _, tt := range testcases {
+ t.Run(fmt.Sprintf("SemiSyncAllowed - %v, TabletControlsAllowed - %v", tt.semiSyncAllowed, tt.tabletControllsAllowed), func(t *testing.T) {
+ te := &TxEngine{
+ twopcAllowed: []bool{true, true},
+ }
+ tsv := TabletServer{
+ te: te,
+ }
+ tsv.SetTwoPCAllowed(TwoPCAllowed_SemiSync, tt.semiSyncAllowed)
+ tsv.SetTwoPCAllowed(TwoPCAllowed_TabletControls, tt.tabletControllsAllowed)
+ require.Equal(t, tt.wantAllowed, te.IsTwoPCAllowed())
+ })
+ }
+}
+
+// TestPrepareTx tests prepareTx function in transaction engine.
+func TestPrepareTx(t *testing.T) {
+ testcases := []struct {
+ name string
+ preparedTx *tx.PreparedTx
+ requireFailure bool
+ errWanted string
+ queryLogWanted string
+ }{
+ {
+ name: "Success",
+ preparedTx: &tx.PreparedTx{
+ Queries: []string{
+ "insert into vitess_test (intval) values(40)",
+ "set @@time_zone='+10:30'",
+ "insert into vitess_test (intval) values(20)",
+ },
+ },
+ requireFailure: false,
+ errWanted: "",
+ queryLogWanted: "use `fakesqldb`;begin;insert into vitess_test (intval) values(40);set @@time_zone='+10:30';insert into vitess_test (intval) values(20)",
+ },
+ {
+ name: "Unretryable failure during query",
+ preparedTx: &tx.PreparedTx{
+ Queries: []string{
+ "insert into vitess_test (intval) values(40)",
+ "failing query",
+ "insert into vitess_test (intval) values(20)",
+ },
+ },
+ requireFailure: true,
+ errWanted: "(errno 1105) (sqlstate HY000)",
+ },
+ {
+ name: "Retryable failure during query",
+ preparedTx: &tx.PreparedTx{
+ Queries: []string{
+ "insert into vitess_test (intval) values(40)",
+ "retryable query",
+ "insert into vitess_test (intval) values(20)",
+ },
+ },
+ requireFailure: false,
+ errWanted: "Retryable error (errno 2002) (sqlstate HY000)",
+ },
+ {
+ name: "Success - Settings query in the beginning",
+ preparedTx: &tx.PreparedTx{
+ Queries: []string{
+ "set @@time_zone='+10:30'",
+ "insert into vitess_test (intval) values(40)",
+ "insert into vitess_test (intval) values(20)",
+ },
+ },
+ requireFailure: false,
+ errWanted: "",
+ queryLogWanted: "use `fakesqldb`;set @@time_zone='+10:30';begin;insert into vitess_test (intval) values(40);insert into vitess_test (intval) values(20)",
+ },
+ }
+ for _, tt := range testcases {
+ t.Run(tt.name, func(t *testing.T) {
+ db := setUpQueryExecutorTest(t)
+ defer db.Close()
+ db.AddQueryPattern(".*", &sqltypes.Result{})
+ db.AddRejectedQuery("failing query", assert.AnError)
+ db.AddRejectedQuery("retryable query", sqlerror.NewSQLError(sqlerror.CRConnectionError, "", "Retryable error"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ cfg.TwoPCAbandonAge = 200 * time.Second
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"), nil)
+ te.AcceptReadWrite()
+ db.ResetQueryLog()
+ failed, err := te.prepareTx(context.Background(), tt.preparedTx)
+ require.EqualValues(t, tt.requireFailure, failed)
+ if tt.errWanted != "" {
+ require.ErrorContains(t, err, tt.errWanted)
+ return
+ }
+ require.NoError(t, err)
+ require.EqualValues(t, 1, len(te.preparedPool.conns))
+ require.EqualValues(t, tt.queryLogWanted, db.QueryLog())
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go
index 52f356e0cca..302a3d41050 100644
--- a/go/vt/vttablet/tabletserver/tx_pool.go
+++ b/go/vt/vttablet/tabletserver/tx_pool.go
@@ -40,9 +40,8 @@ import (
)
const (
- txLogInterval = 1 * time.Minute
- beginWithCSRO = "start transaction with consistent snapshot, read only"
- trackGtidQuery = "set session session_track_gtids = START_GTID"
+ txLogInterval = 1 * time.Minute
+ beginWithCSRO = "start transaction with consistent snapshot, read only"
)
var txIsolations = map[querypb.ExecuteOptions_TransactionIsolation]string{
@@ -230,7 +229,7 @@ func (tp *TxPool) Rollback(ctx context.Context, txConn *StatefulConnection) erro
// the statements (if any) executed to initiate the transaction. In autocommit
// mode the statement will be "".
// The connection returned is locked for the callee and its responsibility is to unlock the connection.
-func (tp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions, readOnly bool, reservedID int64, savepointQueries []string, setting *smartconnpool.Setting) (*StatefulConnection, string, string, error) {
+func (tp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions, readOnly bool, reservedID int64, setting *smartconnpool.Setting) (*StatefulConnection, string, string, error) {
span, ctx := trace.NewSpan(ctx, "TxPool.Begin")
defer span.Finish()
@@ -262,25 +261,28 @@ func (tp *TxPool) Begin(ctx context.Context, options *querypb.ExecuteOptions, re
if err != nil {
return nil, "", "", err
}
- sql, sessionStateChanges, err := tp.begin(ctx, options, readOnly, conn, savepointQueries)
+ sql, sessionStateChanges, err := tp.begin(ctx, options, readOnly, conn)
if err != nil {
conn.Close()
conn.Release(tx.ConnInitFail)
return nil, "", "", err
}
+ // If we have applied any settings on the connection, then we need to record the query
+ // in case we need to redo the transaction because of a failure.
+ if setting != nil {
+ conn.TxProperties().RecordQueryDetail(setting.ApplyQuery(), nil)
+ }
return conn, sql, sessionStateChanges, nil
}
-func (tp *TxPool) begin(ctx context.Context, options *querypb.ExecuteOptions, readOnly bool, conn *StatefulConnection, savepointQueries []string) (string, string, error) {
+func (tp *TxPool) begin(ctx context.Context, options *querypb.ExecuteOptions, readOnly bool, conn *StatefulConnection) (string, string, error) {
immediateCaller := callerid.ImmediateCallerIDFromContext(ctx)
effectiveCaller := callerid.EffectiveCallerIDFromContext(ctx)
- beginQueries, autocommit, sessionStateChanges, err := createTransaction(ctx, options, conn, readOnly, savepointQueries)
+ beginQueries, autocommit, sessionStateChanges, err := createTransaction(ctx, options, conn, readOnly)
if err != nil {
return "", "", err
}
-
conn.txProps = tp.NewTxProps(immediateCaller, effectiveCaller, autocommit)
-
return beginQueries, sessionStateChanges, nil
}
@@ -306,7 +308,6 @@ func createTransaction(
options *querypb.ExecuteOptions,
conn *StatefulConnection,
readOnly bool,
- savepointQueries []string,
) (beginQueries string, autocommitTransaction bool, sessionStateChanges string, err error) {
switch options.GetTransactionIsolation() {
case querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY:
@@ -344,12 +345,6 @@ func createTransaction(
default:
return "", false, "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "[BUG] don't know how to open a transaction of this type: %v", options.GetTransactionIsolation())
}
-
- for _, savepoint := range savepointQueries {
- if _, err = conn.Exec(ctx, savepoint, 1, false); err != nil {
- return "", false, "", err
- }
- }
return
}
@@ -398,16 +393,6 @@ func createStartTxStmt(options *querypb.ExecuteOptions, readOnly bool) (string,
}
func handleConsistentSnapshotCase(ctx context.Context, conn *StatefulConnection) (beginSQL string, sessionStateChanges string, err error) {
- _, err = conn.execWithRetry(ctx, trackGtidQuery, 1, false)
- // We allow this to fail since this is a custom MySQL extension, but we return
- // then if this query was executed or not.
- //
- // Callers also can know because the sessionStateChanges will be empty for a snapshot
- // transaction and get GTID information in another (less efficient) way.
- if err == nil {
- beginSQL = trackGtidQuery + "; "
- }
-
isolationLevel := txIsolations[querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY]
execSQL, err := setIsolationLevel(ctx, conn, isolationLevel)
diff --git a/go/vt/vttablet/tabletserver/tx_pool_test.go b/go/vt/vttablet/tabletserver/tx_pool_test.go
index aa2d5b69e89..22810d4c422 100644
--- a/go/vt/vttablet/tabletserver/tx_pool_test.go
+++ b/go/vt/vttablet/tabletserver/tx_pool_test.go
@@ -48,7 +48,7 @@ func TestTxPoolExecuteCommit(t *testing.T) {
sql := "select 'this is a query'"
// begin a transaction and then return the connection
- conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
id := conn.ReservedID()
@@ -83,7 +83,7 @@ func TestTxPoolExecuteRollback(t *testing.T) {
db, txPool, _, closer := setup(t)
defer closer()
- conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
defer conn.Release(tx.TxRollback)
@@ -104,7 +104,7 @@ func TestTxPoolExecuteRollbackOnClosedConn(t *testing.T) {
db, txPool, _, closer := setup(t)
defer closer()
- conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
defer conn.Release(tx.TxRollback)
@@ -125,9 +125,9 @@ func TestTxPoolRollbackNonBusy(t *testing.T) {
defer closer()
// start two transactions, and mark one of them as unused
- conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
- conn2, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn2, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
conn2.Unlock() // this marks conn2 as NonBusy
@@ -154,7 +154,7 @@ func TestTxPoolTransactionIsolation(t *testing.T) {
db, txPool, _, closer := setup(t)
defer closer()
- c2, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_READ_COMMITTED}, false, 0, nil, nil)
+ c2, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_READ_COMMITTED}, false, 0, nil)
require.NoError(t, err)
c2.Release(tx.TxClose)
@@ -172,7 +172,7 @@ func TestTxPoolAutocommit(t *testing.T) {
// to mysql.
// This test is meaningful because if txPool.Begin were to send a BEGIN statement to the connection, it will fatal
// because is not in the list of expected queries (i.e db.AddQuery hasn't been called).
- conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_AUTOCOMMIT}, false, 0, nil, nil)
+ conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{TransactionIsolation: querypb.ExecuteOptions_AUTOCOMMIT}, false, 0, nil)
require.NoError(t, err)
// run a query to see it in the query log
@@ -204,7 +204,7 @@ func TestTxPoolBeginWithPoolConnectionError_Errno2006_Transient(t *testing.T) {
err := db.WaitForClose(2 * time.Second)
require.NoError(t, err)
- txConn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ txConn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err, "Begin should have succeeded after the retry in DBConn.Exec()")
txConn.Release(tx.TxCommit)
}
@@ -225,7 +225,7 @@ func primeTxPoolWithConnection(t *testing.T, ctx context.Context) (*fakesqldb.DB
// reused by subsequent transactions.
db.AddQuery("begin", &sqltypes.Result{})
db.AddQuery("rollback", &sqltypes.Result{})
- txConn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ txConn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
txConn.Release(tx.TxCommit)
@@ -248,7 +248,7 @@ func TestTxPoolBeginWithError(t *testing.T) {
}
ctxWithCallerID := callerid.NewContext(ctx, ef, im)
- _, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ _, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil)
require.Error(t, err)
require.Contains(t, err.Error(), "error: rejected")
require.Equal(t, vtrpcpb.Code_UNKNOWN, vterrors.Code(err), "wrong error code for Begin error")
@@ -270,19 +270,6 @@ func TestTxPoolBeginWithError(t *testing.T) {
}, limiter.Actions())
}
-func TestTxPoolBeginWithPreQueryError(t *testing.T) {
- ctx, cancel := context.WithCancel(context.Background())
- defer cancel()
-
- db, txPool, _, closer := setup(t)
- defer closer()
- db.AddRejectedQuery("pre_query", errRejected)
- _, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, []string{"pre_query"}, nil)
- require.Error(t, err)
- require.Contains(t, err.Error(), "error: rejected")
- require.Equal(t, vtrpcpb.Code_UNKNOWN, vterrors.Code(err), "wrong error code for Begin error")
-}
-
func TestTxPoolCancelledContextError(t *testing.T) {
// given
db, txPool, _, closer := setup(t)
@@ -291,7 +278,7 @@ func TestTxPoolCancelledContextError(t *testing.T) {
cancel()
// when
- _, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ _, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
// then
require.Error(t, err)
@@ -312,12 +299,12 @@ func TestTxPoolWaitTimeoutError(t *testing.T) {
defer closer()
// lock the only connection in the pool.
- conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
defer conn.Unlock()
// try locking one more connection.
- _, _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ _, _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
// then
require.Error(t, err)
@@ -337,7 +324,7 @@ func TestTxPoolRollbackFailIsPassedThrough(t *testing.T) {
defer closer()
db.AddRejectedQuery("rollback", errRejected)
- conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn1, _, _, err := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
_, err = conn1.Exec(ctx, sql, 1, true)
@@ -357,7 +344,7 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
db, txPool, _, _ := setup(t)
defer db.Close()
- conn1, _, _, _ := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn1, _, _, _ := txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
id := conn1.ReservedID()
conn1.Unlock()
txPool.Close()
@@ -365,7 +352,7 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
assertErrorMatch := func(id int64, reason string) {
conn, err := txPool.GetAndLock(id, "for query")
if err == nil { //
- conn.Releasef("fail")
+ conn.ReleaseString("fail")
t.Errorf("expected to get an error")
return
}
@@ -380,12 +367,12 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
params := dbconfigs.New(db.ConnParams())
txPool.Open(params, params, params)
- conn1, _, _, _ = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn1, _, _, _ = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
id = conn1.ReservedID()
_, err := txPool.Commit(ctx, conn1)
require.NoError(t, err)
- conn1.Releasef("transaction committed")
+ conn1.ReleaseString("transaction committed")
assertErrorMatch(id, "transaction committed")
@@ -396,7 +383,7 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
txPool.Open(params, params, params)
defer txPool.Close()
- conn1, _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn1, _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err, "unable to start transaction: %v", err)
conn1.Unlock()
id = conn1.ReservedID()
@@ -412,7 +399,7 @@ func TestTxPoolCloseKillsStrayTransactions(t *testing.T) {
startingStray := txPool.env.Stats().InternalErrors.Counts()["StrayTransactions"]
// Start stray transaction.
- conn, _, _, err := txPool.Begin(context.Background(), &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(context.Background(), &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
conn.Unlock()
@@ -443,7 +430,7 @@ func TestTxTimeoutKillsTransactions(t *testing.T) {
ctxWithCallerID := callerid.NewContext(ctx, ef, im)
// Start transaction.
- conn, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
conn.Unlock()
@@ -491,7 +478,7 @@ func TestTxTimeoutDoesNotKillShortLivedTransactions(t *testing.T) {
ctxWithCallerID := callerid.NewContext(ctx, ef, im)
// Start transaction.
- conn, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
conn.Unlock()
@@ -526,7 +513,7 @@ func TestTxTimeoutKillsOlapTransactions(t *testing.T) {
// Start transaction.
conn, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{
Workload: querypb.ExecuteOptions_OLAP,
- }, false, 0, nil, nil)
+ }, false, 0, nil)
require.NoError(t, err)
conn.Unlock()
@@ -561,11 +548,11 @@ func TestTxTimeoutNotEnforcedForZeroLengthTimeouts(t *testing.T) {
ctxWithCallerID := callerid.NewContext(ctx, ef, im)
// Start transactions.
- conn0, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil, nil)
+ conn0, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, 0, nil)
require.NoError(t, err)
conn1, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{
Workload: querypb.ExecuteOptions_OLAP,
- }, false, 0, nil, nil)
+ }, false, 0, nil)
require.NoError(t, err)
conn0.Unlock()
conn1.Unlock()
@@ -606,7 +593,7 @@ func TestTxTimeoutReservedConn(t *testing.T) {
// Start OLAP transaction and return it to pool right away.
conn0, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{
Workload: querypb.ExecuteOptions_OLAP,
- }, false, 0, nil, nil)
+ }, false, 0, nil)
require.NoError(t, err)
// Taint the connection.
conn0.Taint(ctxWithCallerID, nil)
@@ -648,14 +635,14 @@ func TestTxTimeoutReusedReservedConn(t *testing.T) {
// Start OLAP transaction and return it to pool right away.
conn0, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{
Workload: querypb.ExecuteOptions_OLAP,
- }, false, 0, nil, nil)
+ }, false, 0, nil)
require.NoError(t, err)
// Taint the connection.
conn0.Taint(ctxWithCallerID, nil)
conn0.Unlock()
// Reuse underlying connection in an OLTP transaction.
- conn1, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, conn0.ReservedID(), nil, nil)
+ conn1, _, _, err := txPool.Begin(ctxWithCallerID, &querypb.ExecuteOptions{}, false, conn0.ReservedID(), nil)
require.NoError(t, err)
require.Equal(t, conn1.ReservedID(), conn0.ReservedID())
conn1.Unlock()
@@ -714,11 +701,11 @@ func TestTxPoolBeginStatements(t *testing.T) {
expBeginSQL: "set transaction isolation level serializable; start transaction read only",
}, {
txIsolationLevel: querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY,
- expBeginSQL: "set session session_track_gtids = START_GTID; set transaction isolation level repeatable read; start transaction with consistent snapshot, read only",
+ expBeginSQL: "set transaction isolation level repeatable read; start transaction with consistent snapshot, read only",
}, {
txIsolationLevel: querypb.ExecuteOptions_CONSISTENT_SNAPSHOT_READ_ONLY,
readOnly: true,
- expBeginSQL: "set session session_track_gtids = START_GTID; set transaction isolation level repeatable read; start transaction with consistent snapshot, read only",
+ expBeginSQL: "set transaction isolation level repeatable read; start transaction with consistent snapshot, read only",
}, {
txIsolationLevel: querypb.ExecuteOptions_AUTOCOMMIT,
expBeginSQL: "",
@@ -786,7 +773,7 @@ func TestTxPoolBeginStatements(t *testing.T) {
TransactionIsolation: tc.txIsolationLevel,
TransactionAccessMode: tc.txAccessModes,
}
- conn, beginSQL, _, err := txPool.Begin(ctx, options, tc.readOnly, 0, nil, nil)
+ conn, beginSQL, _, err := txPool.Begin(ctx, options, tc.readOnly, 0, nil)
if tc.expErr != "" {
require.Error(t, err)
require.Contains(t, err.Error(), tc.expErr)
diff --git a/go/vt/vttablet/tabletserver/tx_prep_pool.go b/go/vt/vttablet/tabletserver/tx_prep_pool.go
index d5376172856..8e766062a92 100644
--- a/go/vt/vttablet/tabletserver/tx_prep_pool.go
+++ b/go/vt/vttablet/tabletserver/tx_prep_pool.go
@@ -36,31 +36,55 @@ type TxPreparedPool struct {
mu sync.Mutex
conns map[string]*StatefulConnection
reserved map[string]error
- // shutdown tells if the prepared pool has been drained and shutdown.
- shutdown bool
+ // open tells if the prepared pool is open for accepting transactions.
+ open bool
capacity int
+ // twoPCEnabled is set to true if 2PC is enabled.
+ twoPCEnabled bool
}
// NewTxPreparedPool creates a new TxPreparedPool.
-func NewTxPreparedPool(capacity int) *TxPreparedPool {
+func NewTxPreparedPool(capacity int, twoPCEnabled bool) *TxPreparedPool {
if capacity < 0 {
// If capacity is 0 all prepares will fail.
capacity = 0
}
return &TxPreparedPool{
- conns: make(map[string]*StatefulConnection, capacity),
- reserved: make(map[string]error),
- capacity: capacity,
+ conns: make(map[string]*StatefulConnection, capacity),
+ reserved: make(map[string]error),
+ capacity: capacity,
+ twoPCEnabled: twoPCEnabled,
}
}
+// Open marks the prepared pool open for use.
+func (pp *TxPreparedPool) Open() {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = true
+}
+
+// Close marks the prepared pool closed.
+func (pp *TxPreparedPool) Close() {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = false
+}
+
+// IsOpen checks if the prepared pool is open for use.
+func (pp *TxPreparedPool) IsOpen() bool {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ return pp.open
+}
+
// Put adds the connection to the pool. It returns an error
// if the pool is full or on duplicate key.
func (pp *TxPreparedPool) Put(c *StatefulConnection, dtid string) error {
pp.mu.Lock()
defer pp.mu.Unlock()
// If the pool is shutdown, we don't accept new prepared transactions.
- if pp.shutdown {
+ if !pp.open {
return vterrors.VT09025("pool is shutdown")
}
if _, ok := pp.reserved[dtid]; ok {
@@ -105,7 +129,7 @@ func (pp *TxPreparedPool) FetchForCommit(dtid string) (*StatefulConnection, erro
defer pp.mu.Unlock()
// If the pool is shutdown, we don't have any connections to return.
// That however doesn't mean this transaction was committed, it could very well have been rollbacked.
- if pp.shutdown {
+ if !pp.open {
return nil, vterrors.VT09025("pool is shutdown")
}
if err, ok := pp.reserved[dtid]; ok {
@@ -139,7 +163,7 @@ func (pp *TxPreparedPool) Forget(dtid string) {
func (pp *TxPreparedPool) FetchAllForRollback() []*StatefulConnection {
pp.mu.Lock()
defer pp.mu.Unlock()
- pp.shutdown = true
+ pp.open = false
conns := make([]*StatefulConnection, 0, len(pp.conns))
for _, c := range pp.conns {
conns = append(conns, c)
@@ -148,3 +172,40 @@ func (pp *TxPreparedPool) FetchAllForRollback() []*StatefulConnection {
pp.reserved = make(map[string]error)
return conns
}
+
+// IsEmptyForTable returns true if no prepared transactions are found for the table.
+func (pp *TxPreparedPool) IsEmptyForTable(tableName string) bool {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ if !pp.twoPCEnabled {
+ return true
+ }
+ // If the pool is shutdown, we do not know the correct state of prepared transactions.
+ if !pp.open {
+ return false
+ }
+ for _, connection := range pp.conns {
+ for _, query := range connection.txProps.Queries {
+ for _, table := range query.Tables {
+ if table == tableName {
+ return false
+ }
+ }
+ }
+ }
+ return true
+}
+
+// IsEmpty returns true if the pool is empty.
+func (pp *TxPreparedPool) IsEmpty() bool {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ if !pp.twoPCEnabled {
+ return true
+ }
+ // If the pool is shutdown, we do not know the correct state of prepared transactions.
+ if !pp.open {
+ return false
+ }
+ return len(pp.conns) == 0
+}
diff --git a/go/vt/vttablet/tabletserver/tx_prep_pool_test.go b/go/vt/vttablet/tabletserver/tx_prep_pool_test.go
index 42e2b800e0e..e8c889990f0 100644
--- a/go/vt/vttablet/tabletserver/tx_prep_pool_test.go
+++ b/go/vt/vttablet/tabletserver/tx_prep_pool_test.go
@@ -21,16 +21,18 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
)
func TestEmptyPrep(t *testing.T) {
- pp := NewTxPreparedPool(0)
+ pp := createAndOpenPreparedPool(0)
err := pp.Put(nil, "aa")
require.ErrorContains(t, err, "prepared transactions exceeded limit: 0")
}
func TestPrepPut(t *testing.T) {
- pp := NewTxPreparedPool(2)
+ pp := createAndOpenPreparedPool(2)
err := pp.Put(nil, "aa")
require.NoError(t, err)
err = pp.Put(nil, "bb")
@@ -50,7 +52,7 @@ func TestPrepPut(t *testing.T) {
}
func TestPrepFetchForRollback(t *testing.T) {
- pp := NewTxPreparedPool(2)
+ pp := createAndOpenPreparedPool(2)
conn := &StatefulConnection{}
pp.Put(conn, "aa")
got := pp.FetchForRollback("bb")
@@ -68,7 +70,7 @@ func TestPrepFetchForRollback(t *testing.T) {
}
func TestPrepFetchForCommit(t *testing.T) {
- pp := NewTxPreparedPool(2)
+ pp := createAndOpenPreparedPool(2)
conn := &StatefulConnection{}
got, err := pp.FetchForCommit("aa")
require.NoError(t, err)
@@ -97,7 +99,7 @@ func TestPrepFetchForCommit(t *testing.T) {
}
func TestPrepFetchAll(t *testing.T) {
- pp := NewTxPreparedPool(2)
+ pp := createAndOpenPreparedPool(2)
conn1 := &StatefulConnection{}
conn2 := &StatefulConnection{}
pp.Put(conn1, "aa")
@@ -108,3 +110,143 @@ func TestPrepFetchAll(t *testing.T) {
_, err := pp.FetchForCommit("aa")
require.ErrorContains(t, err, "pool is shutdown")
}
+
+// createAndOpenPreparedPool creates a new transaction prepared pool and opens it.
+// Used as a helper function for testing.
+func createAndOpenPreparedPool(capacity int) *TxPreparedPool {
+ pp := NewTxPreparedPool(capacity, true)
+ pp.Open()
+ return pp
+}
+
+func TestTxPreparedPoolIsEmptyForTable(t *testing.T) {
+ tests := []struct {
+ name string
+ setupFunc func(pp *TxPreparedPool)
+ wantIsEmpty bool
+ }{
+ {
+ name: "Closed prepared pool",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = false
+ },
+ wantIsEmpty: false,
+ },
+ {
+ name: "Two PC Disabled",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.twoPCEnabled = false
+ },
+ wantIsEmpty: true,
+ },
+ {
+ name: "No prepared transactions",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = true
+ },
+ wantIsEmpty: true,
+ },
+ {
+ name: "Prepared transactions for table t1",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ pp.open = true
+ pp.mu.Unlock()
+ pp.Put(&StatefulConnection{
+ txProps: &tx.Properties{
+ Queries: []tx.Query{
+ {
+ Tables: []string{"t1", "t2"},
+ },
+ },
+ },
+ }, "dtid1")
+ },
+ wantIsEmpty: false,
+ },
+ {
+ name: "Prepared transactions for other tables",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ pp.open = true
+ pp.mu.Unlock()
+ pp.Put(&StatefulConnection{
+ txProps: &tx.Properties{
+ Queries: []tx.Query{
+ {
+ Tables: []string{"t3", "t2"},
+ },
+ },
+ },
+ }, "dtid1")
+ },
+ wantIsEmpty: true,
+ },
+ }
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ pp := NewTxPreparedPool(1, true)
+ tt.setupFunc(pp)
+ assert.Equalf(t, tt.wantIsEmpty, pp.IsEmptyForTable("t1"), "IsEmptyForTable()")
+ })
+ }
+}
+
+func TestTxPreparedPoolIsEmpty(t *testing.T) {
+ tests := []struct {
+ name string
+ setupFunc func(pp *TxPreparedPool)
+ wantIsEmpty bool
+ }{
+ {
+ name: "Closed prepared pool",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = false
+ },
+ wantIsEmpty: false,
+ },
+ {
+ name: "Two PC Disabled",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.twoPCEnabled = false
+ },
+ wantIsEmpty: true,
+ },
+ {
+ name: "No prepared transactions",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ defer pp.mu.Unlock()
+ pp.open = true
+ },
+ wantIsEmpty: true,
+ },
+ {
+ name: "Prepared transactions exist",
+ setupFunc: func(pp *TxPreparedPool) {
+ pp.mu.Lock()
+ pp.open = true
+ pp.mu.Unlock()
+ pp.Put(&StatefulConnection{}, "dtid1")
+ },
+ wantIsEmpty: false,
+ },
+ }
+ for _, tt := range tests {
+ t.Run(tt.name, func(t *testing.T) {
+ pp := NewTxPreparedPool(1, true)
+ tt.setupFunc(pp)
+ assert.Equalf(t, tt.wantIsEmpty, pp.IsEmpty(), "IsEmpty()")
+ })
+ }
+}
diff --git a/go/vt/vttablet/tabletserver/txlogz_test.go b/go/vt/vttablet/tabletserver/txlogz_test.go
index 319669a0023..8faec74d07b 100644
--- a/go/vt/vttablet/tabletserver/txlogz_test.go
+++ b/go/vt/vttablet/tabletserver/txlogz_test.go
@@ -60,7 +60,7 @@ func testHandler(req *http.Request, t *testing.T) {
ImmediateCaller: callerid.NewImmediateCallerID("immediate-caller"),
StartTime: time.Now(),
Conclusion: "unknown",
- Queries: []string{"select * from test"},
+ Queries: []tx.Query{{Sql: "select * from test"}},
},
}
txConn.txProps.EndTime = txConn.txProps.StartTime
diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
index 41f075e3802..bd0d1cc5f52 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
@@ -40,7 +40,7 @@ import (
// These vars store the functions used to create the topo server, healthcheck,
// and go/vt/throttler. These are provided here so that they can be overridden
// in tests to generate mocks.
-type healthCheckFactoryFunc func(topoServer *topo.Server, cell string, cellsToWatch []string) discovery.HealthCheck
+type healthCheckFactoryFunc func(ctx context.Context, topoServer *topo.Server, cell, keyspace, shard string, cellsToWatch []string) (discovery.HealthCheck, error)
type throttlerFactoryFunc func(name, unit string, threadCount int, maxRate int64, maxReplicationLagConfig throttler.MaxReplicationLagModuleConfig) (throttler.Throttler, error)
var (
@@ -49,8 +49,13 @@ var (
)
func resetTxThrottlerFactories() {
- healthCheckFactory = func(topoServer *topo.Server, cell string, cellsToWatch []string) discovery.HealthCheck {
- return discovery.NewHealthCheck(context.Background(), discovery.DefaultHealthCheckRetryDelay, discovery.DefaultHealthCheckTimeout, topoServer, cell, strings.Join(cellsToWatch, ","))
+ healthCheckFactory = func(ctx context.Context, topoServer *topo.Server, cell, keyspace, shard string, cellsToWatch []string) (discovery.HealthCheck, error) {
+ // discovery.NewFilterByShard expects a single-shard filter to be in "keyspace|shard" format.
+ filter, err := discovery.NewFilterByShard([]string{keyspace + "|" + shard})
+ if err != nil {
+ return nil, err
+ }
+ return discovery.NewHealthCheck(ctx, discovery.DefaultHealthCheckRetryDelay, discovery.DefaultHealthCheckTimeout, topoServer, cell, strings.Join(cellsToWatch, ","), filter), nil
}
throttlerFactory = func(name, unit string, threadCount int, maxRate int64, maxReplicationLagConfig throttler.MaxReplicationLagModuleConfig) (throttler.Throttler, error) {
return throttler.NewThrottlerFromConfig(name, unit, threadCount, maxRate, maxReplicationLagConfig, time.Now)
@@ -150,6 +155,9 @@ type txThrottlerStateImpl struct {
throttleMu sync.Mutex
throttler throttler.Throttler
+ ctx context.Context
+ cancel context.CancelFunc
+
cellsFromTopo bool
healthCheck discovery.HealthCheck
healthCheckCancel context.CancelFunc
@@ -297,7 +305,10 @@ func newTxThrottlerState(txThrottler *txThrottler, config *tabletenv.TabletConfi
cellsFromTopo = true
}
- return &txThrottlerStateImpl{
+ ctx, cancel := context.WithCancel(context.Background())
+ state := &txThrottlerStateImpl{
+ ctx: ctx,
+ cancel: cancel,
config: config,
cellsFromTopo: cellsFromTopo,
healthCheckCells: healthCheckCells,
@@ -305,40 +316,59 @@ func newTxThrottlerState(txThrottler *txThrottler, config *tabletenv.TabletConfi
target: target,
throttler: t,
txThrottler: txThrottler,
- }, nil
+ }
+
+ // get cells from topo if none defined in tabletenv config
+ if len(state.healthCheckCells) == 0 {
+ cellsCtx, cellsCancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
+ defer cellsCancel()
+ state.healthCheckCells = fetchKnownCells(cellsCtx, txThrottler.topoServer, target)
+ state.cellsFromTopo = true
+ }
+
+ return state, nil
}
-func (ts *txThrottlerStateImpl) initHealthCheck() {
- ts.healthCheck = healthCheckFactory(ts.txThrottler.topoServer, ts.target.Cell, ts.healthCheckCells)
+
+func (ts *txThrottlerStateImpl) initHealthCheck(topoServer *topo.Server, target *querypb.Target) (err error) {
+ ts.healthCheck, err = healthCheckFactory(ts.ctx, topoServer, target.Cell, target.Keyspace, target.Shard, ts.healthCheckCells)
+ if err != nil {
+ return err
+ }
+
ts.healthCheckChan = ts.healthCheck.Subscribe()
ts.healthCheck.RegisterStats()
+
+ return nil
}
func (ts *txThrottlerStateImpl) closeHealthCheck() {
if ts.healthCheck == nil {
return
}
- ts.healthCheckCancel()
- ts.wg.Wait()
+
+ ts.cancel()
ts.healthCheck.Close()
ts.healthCheck = nil
ts.maxLag = 0
}
-func (ts *txThrottlerStateImpl) updateHealthCheckCells(ctx context.Context) {
- fetchCtx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout)
+func (ts *txThrottlerStateImpl) updateHealthCheckCells(topoServer *topo.Server, target *querypb.Target) error {
+ fetchCtx, cancel := context.WithTimeout(ts.ctx, topo.RemoteOperationTimeout)
defer cancel()
- knownCells := fetchKnownCells(fetchCtx, ts.txThrottler.topoServer, ts.target)
+ knownCells := fetchKnownCells(fetchCtx, topoServer, target)
if !slices.Equal(knownCells, ts.healthCheckCells) {
log.Info("txThrottler: restarting healthcheck stream due to topology cells update")
ts.healthCheckCells = knownCells
ts.closeHealthCheck()
- ts.initHealthCheck()
+ return ts.initHealthCheck(topoServer, target)
}
+
+ return nil
}
-func (ts *txThrottlerStateImpl) healthCheckProcessor(ctx context.Context) {
+func (ts *txThrottlerStateImpl) healthChecksProcessor(topoServer *topo.Server, target *querypb.Target) {
defer ts.wg.Done()
var cellsUpdateTicks <-chan time.Time
if ts.cellsFromTopo {
@@ -348,10 +378,12 @@ func (ts *txThrottlerStateImpl) healthCheckProcessor(ctx context.Context) {
}
for {
select {
- case <-ctx.Done():
+ case <-ts.ctx.Done():
return
case <-cellsUpdateTicks:
- ts.updateHealthCheckCells(ctx)
+ if err := ts.updateHealthCheckCells(topoServer, target); err != nil {
+ log.Errorf("txThrottler: failed to update cell list: %+v", err)
+ }
case th := <-ts.healthCheckChan:
ts.StatsUpdate(th)
}
@@ -364,7 +396,7 @@ func (ts *txThrottlerStateImpl) makePrimary() {
ctx, ts.healthCheckCancel = context.WithCancel(context.Background())
ts.wg.Add(1)
- go ts.healthCheckProcessor(ctx)
+ go ts.healthChecksProcessor(ts.txThrottler.topoServer, ts.target)
ts.wg.Add(1)
go ts.updateMaxLag(ctx)
diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
index 3e46e79c70b..514ef7a1c14 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
@@ -75,8 +75,8 @@ func TestEnabledThrottler(t *testing.T) {
hcCall2.After(hcCall1)
hcCall3 := mockHealthCheck.EXPECT().Close()
hcCall3.After(hcCall2)
- healthCheckFactory = func(topoServer *topo.Server, cell string, cellsToWatch []string) discovery.HealthCheck {
- return mockHealthCheck
+ healthCheckFactory = func(ctx context.Context, topoServer *topo.Server, cell, keyspace, shard string, cellsToWatch []string) (discovery.HealthCheck, error) {
+ return mockHealthCheck, nil
}
mockThrottler := NewMockThrottler(mockCtrl)
diff --git a/go/vt/vttablet/tabletserver/vstreamer/copy.go b/go/vt/vttablet/tabletserver/vstreamer/copy.go
index 2f54ea235fe..1e1d432956d 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/copy.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/copy.go
@@ -74,7 +74,7 @@ func (uvs *uvstreamer) catchup(ctx context.Context) error {
errch := make(chan error, 1)
go func() {
startPos := replication.EncodePosition(uvs.pos)
- vs := newVStreamer(ctx, uvs.cp, uvs.se, startPos, "", uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send2, "catchup", uvs.vse)
+ vs := newVStreamer(ctx, uvs.cp, uvs.se, startPos, "", uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send2, "catchup", uvs.vse, nil)
uvs.setVs(vs)
errch <- vs.Stream()
uvs.setVs(nil)
@@ -306,7 +306,7 @@ func (uvs *uvstreamer) copyTable(ctx context.Context, tableName string) error {
uvs.setCopyState(tableName, qrLastPK)
log.V(2).Infof("NewLastPK: %v", qrLastPK)
return nil
- })
+ }, nil)
if err != nil {
uvs.vse.errorCounts.Add("StreamRows", 1)
return err
@@ -333,7 +333,7 @@ func (uvs *uvstreamer) fastForward(stopPos string) error {
}()
log.Infof("starting fastForward from %s upto pos %s", replication.EncodePosition(uvs.pos), stopPos)
uvs.stopPos, _ = replication.DecodePosition(stopPos)
- vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, replication.EncodePosition(uvs.pos), "", uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send2, "fastforward", uvs.vse)
+ vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, replication.EncodePosition(uvs.pos), "", uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send2, "fastforward", uvs.vse, nil)
uvs.setVs(vs)
return vs.Stream()
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go
index 3dc1f5f9a92..eba6e736a21 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/engine.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go
@@ -39,6 +39,7 @@ import (
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/vindexes"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle"
@@ -233,7 +234,10 @@ func (vse *Engine) validateBinlogRowImage(ctx context.Context, db dbconfigs.Conn
// Stream starts a new stream.
// This streams events from the binary logs
-func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) error {
+func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binlogdatapb.TableLastPK,
+ filter *binlogdatapb.Filter, throttlerApp throttlerapp.Name,
+ send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) error {
+
if err := vse.validateBinlogRowImage(ctx, vse.se.GetDBConnector()); err != nil {
return err
}
@@ -250,7 +254,8 @@ func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binl
}
vse.mu.Lock()
defer vse.mu.Unlock()
- streamer := newUVStreamer(ctx, vse, vse.env.Config().DB.FilteredWithDB(), vse.se, startPos, tablePKs, filter, vse.lvschema, throttlerApp, send)
+ streamer := newUVStreamer(ctx, vse, vse.env.Config().DB.FilteredWithDB(), vse.se, startPos, tablePKs,
+ filter, vse.lvschema, throttlerApp, send, options)
idx := vse.streamIdx
vse.streamers[idx] = streamer
vse.streamIdx++
@@ -277,7 +282,8 @@ func (vse *Engine) Stream(ctx context.Context, startPos string, tablePKs []*binl
// StreamRows streams rows.
// This streams the table data rows (so we can copy the table data snapshot)
-func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltypes.Value, send func(*binlogdatapb.VStreamRowsResponse) error) error {
+func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltypes.Value,
+ send func(*binlogdatapb.VStreamRowsResponse) error, options *binlogdatapb.VStreamOptions) error {
// Ensure vschema is initialized and the watcher is started.
// Starting of the watcher has to be delayed till the first call to Stream
// because this overhead should be incurred only if someone uses this feature.
@@ -292,7 +298,8 @@ func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltyp
vse.mu.Lock()
defer vse.mu.Unlock()
- rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema, send, vse, RowStreamerModeSingleTable, nil)
+ rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema,
+ send, vse, RowStreamerModeSingleTable, nil, options)
idx := vse.streamIdx
vse.rowStreamers[idx] = rowStreamer
vse.streamIdx++
@@ -318,7 +325,9 @@ func (vse *Engine) StreamRows(ctx context.Context, query string, lastpk []sqltyp
}
// StreamTables streams all tables.
-func (vse *Engine) StreamTables(ctx context.Context, send func(*binlogdatapb.VStreamTablesResponse) error) error {
+func (vse *Engine) StreamTables(ctx context.Context,
+ send func(*binlogdatapb.VStreamTablesResponse) error, options *binlogdatapb.VStreamOptions) error {
+
// Ensure vschema is initialized and the watcher is started.
// Starting of the watcher is delayed till the first call to StreamTables
// so that this overhead is incurred only if someone uses this feature.
@@ -333,7 +342,7 @@ func (vse *Engine) StreamTables(ctx context.Context, send func(*binlogdatapb.VSt
vse.mu.Lock()
defer vse.mu.Unlock()
- tableStreamer := newTableStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, vse.lvschema, send, vse)
+ tableStreamer := newTableStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, vse.lvschema, send, vse, options)
idx := vse.streamIdx
vse.tableStreamers[idx] = tableStreamer
vse.streamIdx++
@@ -359,7 +368,9 @@ func (vse *Engine) StreamTables(ctx context.Context, send func(*binlogdatapb.VSt
}
// StreamResults streams results of the query with the gtid.
-func (vse *Engine) StreamResults(ctx context.Context, query string, send func(*binlogdatapb.VStreamResultsResponse) error) error {
+func (vse *Engine) StreamResults(ctx context.Context, query string,
+ send func(*binlogdatapb.VStreamResultsResponse) error) error {
+
// Create stream and add it to the map.
resultStreamer, idx, err := func() (*resultStreamer, int, error) {
if atomic.LoadInt32(&vse.isOpen) == 0 {
@@ -465,7 +476,7 @@ func (vse *Engine) setWatch() {
}
func getPacketSize() int64 {
- return int64(defaultPacketSize)
+ return int64(vttablet.VStreamerDefaultPacketSize)
}
// waitForMySQL ensures that the source is able to stay within defined bounds for
diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
index b0b31e256cc..7cafcc6d485 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
@@ -96,7 +96,7 @@ func TestUpdateVSchema(t *testing.T) {
// Stream should terminate immediately due to invalid pos.
_ = engine.Stream(ctx, "invalid", nil, filter, throttlerapp.VStreamerName, func(_ []*binlogdatapb.VEvent) error {
return nil
- })
+ }, nil)
startCount := expectUpdateCount(t, 1)
@@ -106,6 +106,7 @@ func TestUpdateVSchema(t *testing.T) {
expectUpdateCount(t, startCount+1)
want := `{
+ "mirror_rules": {},
"routing_rules": {},
"keyspaces": {
"vttest": {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
index 81f4d3f7975..675677bf820 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
@@ -142,6 +142,11 @@ type TestQuery struct {
type TestRowChange struct {
before []string
after []string
+
+ // If you need to customize the image you can use the raw types.
+ beforeRaw *query.Row
+ afterRaw *query.Row
+ dataColumnsRaw *binlogdatapb.RowChange_Bitmap
}
// TestRowEventSpec is used for defining a custom row event.
@@ -158,10 +163,15 @@ func (s *TestRowEventSpec) String() string {
TableName: s.table,
}
var rowChanges []*binlogdatapb.RowChange
- if s.changes != nil && len(s.changes) > 0 {
+ if len(s.changes) > 0 {
for _, c := range s.changes {
rowChange := binlogdatapb.RowChange{}
- if c.before != nil && len(c.before) > 0 {
+ if c.dataColumnsRaw != nil {
+ rowChange.DataColumns = c.dataColumnsRaw
+ }
+ if c.beforeRaw != nil {
+ rowChange.Before = c.beforeRaw
+ } else if len(c.before) > 0 {
rowChange.Before = &query.Row{}
for _, val := range c.before {
if val == sqltypes.NullStr {
@@ -171,7 +181,9 @@ func (s *TestRowEventSpec) String() string {
rowChange.Before.Values = append(rowChange.Before.Values, []byte(val)...)
}
}
- if c.after != nil && len(c.after) > 0 {
+ if c.afterRaw != nil {
+ rowChange.After = c.afterRaw
+ } else if len(c.after) > 0 {
rowChange.After = &query.Row{}
for i, val := range c.after {
if val == sqltypes.NullStr {
@@ -318,7 +330,7 @@ func (ts *TestSpec) Init() {
// Close() should be called (via defer) at the end of the test to clean up the tables created in the test.
func (ts *TestSpec) Close() {
- dropStatement := fmt.Sprintf("drop tables %s", strings.Join(ts.schema.TableNames(), ", "))
+ dropStatement := fmt.Sprintf("drop table if exists %s", strings.Join(ts.schema.TableNames(), ", "))
execStatement(ts.t, dropStatement)
}
@@ -354,6 +366,7 @@ func (ts *TestSpec) getBindVarsForUpdate(stmt sqlparser.Statement) (string, map[
require.True(ts.t, ok, "field event for table %s not found", table)
index := int64(0)
state := ts.getCurrentState(table)
+ require.NotNil(ts.t, state)
for i, col := range fe.cols {
bv[col.name] = string(state.Values[index : index+state.Lengths[i]])
index += state.Lengths[i]
diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_test.go
index aade1511060..18b5e1f1b3b 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/main_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/main_test.go
@@ -22,11 +22,14 @@ import (
"io"
"os"
"slices"
+ "strconv"
"strings"
"sync"
"testing"
"time"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
+
"github.com/stretchr/testify/require"
_flag "vitess.io/vitess/go/internal/flag"
@@ -309,6 +312,14 @@ func vstream(ctx context.Context, t *testing.T, pos string, tablePKs []*binlogda
}},
}
}
+
+ // Some unit tests currently change the packet size options for the scope of those tests. We want to pass those
+ // values to the VStreamer for the duration of this test.
+ var options binlogdatapb.VStreamOptions
+ options.ConfigOverrides = make(map[string]string)
+ options.ConfigOverrides["vstream_dynamic_packet_size"] = strconv.FormatBool(vttablet.VStreamerUseDynamicPacketSize)
+ options.ConfigOverrides["vstream_packet_size"] = strconv.Itoa(vttablet.VStreamerDefaultPacketSize)
+
return engine.Stream(ctx, pos, tablePKs, filter, throttlerapp.VStreamerName, func(evs []*binlogdatapb.VEvent) error {
timer := time.NewTimer(2 * time.Second)
defer timer.Stop()
@@ -323,7 +334,7 @@ func vstream(ctx context.Context, t *testing.T, pos string, tablePKs []*binlogda
return io.EOF
}
return nil
- })
+ }, &options)
}
func execStatement(t *testing.T, query string) {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/packet_size.go b/go/vt/vttablet/tabletserver/vstreamer/packet_size.go
index 50bc1e57cd2..7ce430d1811 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/packet_size.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/packet_size.go
@@ -21,14 +21,9 @@ import (
"github.com/spf13/pflag"
- "vitess.io/vitess/go/vt/servenv"
-
"vitess.io/vitess/go/mathstats"
-)
-
-var (
- defaultPacketSize = 250000
- useDynamicPacketSize = true
+ "vitess.io/vitess/go/vt/servenv"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
)
func init() {
@@ -38,9 +33,9 @@ func init() {
func registerPacketSizeFlags(fs *pflag.FlagSet) {
// defaultPacketSize is the suggested packet size for VReplication streamer.
- fs.IntVar(&defaultPacketSize, "vstream_packet_size", defaultPacketSize, "Suggested packet size for VReplication streamer. This is used only as a recommendation. The actual packet size may be more or less than this amount.")
+ fs.IntVar(&vttablet.VStreamerDefaultPacketSize, "vstream_packet_size", vttablet.VStreamerDefaultPacketSize, "Suggested packet size for VReplication streamer. This is used only as a recommendation. The actual packet size may be more or less than this amount.")
// useDynamicPacketSize controls whether to use dynamic packet size adjustments to increase performance while streaming
- fs.BoolVar(&useDynamicPacketSize, "vstream_dynamic_packet_size", useDynamicPacketSize, "Enable dynamic packet sizing for VReplication. This will adjust the packet size during replication to improve performance.")
+ fs.BoolVar(&vttablet.VStreamerUseDynamicPacketSize, "vstream_dynamic_packet_size", vttablet.VStreamerUseDynamicPacketSize, "Enable dynamic packet sizing for VReplication. This will adjust the packet size during replication to improve performance.")
}
// PacketSizer is a controller that adjusts the size of the packets being sent by the vstreamer at runtime
@@ -50,28 +45,29 @@ type PacketSizer interface {
Limit() int
}
-// DefaultPacketSizer creates a new PacketSizer using the default settings.
+// DefaultPacketSizer creates a new PacketSizer based on the provided values.
// If dynamic packet sizing is enabled, this will return a dynamicPacketSizer.
-func DefaultPacketSizer() PacketSizer {
+// Otherwise it will return a fixedPacketSize of packetSize.
+func DefaultPacketSizer(useDynamicPacketSize bool, packetSize int) PacketSizer {
if useDynamicPacketSize {
- return newDynamicPacketSizer(defaultPacketSize)
+ return newDynamicPacketSizer(packetSize)
}
- return newFixedPacketSize(defaultPacketSize)
+ return newFixedPacketSize(packetSize)
}
// AdjustPacketSize temporarily adjusts the default packet sizes to the given value.
// Calling the returned cleanup function resets them to their original value.
// This function is only used for testing.
func AdjustPacketSize(size int) func() {
- originalSize := defaultPacketSize
- originalDyn := useDynamicPacketSize
+ originalSize := vttablet.VStreamerDefaultPacketSize
+ originalDyn := vttablet.VStreamerUseDynamicPacketSize
- defaultPacketSize = size
- useDynamicPacketSize = false
+ vttablet.VStreamerDefaultPacketSize = size
+ vttablet.VStreamerUseDynamicPacketSize = false
return func() {
- defaultPacketSize = originalSize
- useDynamicPacketSize = originalDyn
+ vttablet.VStreamerDefaultPacketSize = originalSize
+ vttablet.VStreamerUseDynamicPacketSize = originalDyn
}
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
index 2c768d7d3c6..e5115afe6d3 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
@@ -64,6 +64,9 @@ type Plan struct {
EnumSetValuesMap map[int](map[int]string)
env *vtenv.Environment
+
+ // IsInternal is set to true if the plan is for a sidecar table.
+ IsInternal bool
}
// Opcode enumerates the operators supported in a where clause
@@ -86,6 +89,8 @@ const (
NotEqual
// IsNotNull is used to filter a column if it is NULL
IsNotNull
+ // In is used to filter a comparable column if equals any of the values from a specific tuple
+ In
)
// Filter contains opcodes for filtering.
@@ -94,6 +99,9 @@ type Filter struct {
ColNum int
Value sqltypes.Value
+ // Values will be used to store tuple/list values.
+ Values []sqltypes.Value
+
// Parameters for VindexMatch.
// Vindex, VindexColumns and KeyRange, if set, will be used
// to filter the row.
@@ -163,6 +171,8 @@ func getOpcode(comparison *sqlparser.ComparisonExpr) (Opcode, error) {
opcode = GreaterThanEqual
case sqlparser.NotEqualOp:
opcode = NotEqual
+ case sqlparser.InOp:
+ opcode = In
default:
return -1, fmt.Errorf("comparison operator %s not supported", comparison.Operator.ToString())
}
@@ -235,6 +245,24 @@ func (plan *Plan) filter(values, result []sqltypes.Value, charsets []collations.
if values[filter.ColNum].IsNull() {
return false, nil
}
+ case In:
+ if filter.Values == nil {
+ return false, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected empty filter values when performing IN operator")
+ }
+ found := false
+ for _, filterValue := range filter.Values {
+ match, err := compare(Equal, values[filter.ColNum], filterValue, plan.env.CollationEnv(), charsets[filter.ColNum])
+ if err != nil {
+ return false, err
+ }
+ if match {
+ found = true
+ break
+ }
+ }
+ if !found {
+ return false, nil
+ }
default:
match, err := compare(filter.Opcode, values[filter.ColNum], filter.Value, plan.env.CollationEnv(), charsets[filter.ColNum])
if err != nil {
@@ -511,6 +539,27 @@ func (plan *Plan) getColumnFuncExpr(columnName string) *sqlparser.FuncExpr {
return nil
}
+func (plan *Plan) appendTupleFilter(values sqlparser.ValTuple, opcode Opcode, colnum int) error {
+ pv, err := evalengine.Translate(values, &evalengine.Config{
+ Collation: plan.env.CollationEnv().DefaultConnectionCharset(),
+ Environment: plan.env,
+ })
+ if err != nil {
+ return err
+ }
+ env := evalengine.EmptyExpressionEnv(plan.env)
+ resolved, err := env.Evaluate(pv)
+ if err != nil {
+ return err
+ }
+ plan.Filters = append(plan.Filters, Filter{
+ Opcode: opcode,
+ ColNum: colnum,
+ Values: resolved.TupleValues(),
+ })
+ return nil
+}
+
func (plan *Plan) analyzeWhere(vschema *localVSchema, where *sqlparser.Where) error {
if where == nil {
return nil
@@ -534,6 +583,20 @@ func (plan *Plan) analyzeWhere(vschema *localVSchema, where *sqlparser.Where) er
if err != nil {
return err
}
+ // The Right Expr is typically expected to be a Literal value,
+ // except for the IN operator, where a Tuple value is expected.
+ // Handle the IN operator case first.
+ if opcode == In {
+ values, ok := expr.Right.(sqlparser.ValTuple)
+ if !ok {
+ return fmt.Errorf("unexpected: %v", sqlparser.String(expr))
+ }
+ err := plan.appendTupleFilter(values, opcode, colnum)
+ if err != nil {
+ return err
+ }
+ continue
+ }
val, ok := expr.Right.(*sqlparser.Literal)
if !ok {
return fmt.Errorf("unexpected: %v", sqlparser.String(expr))
diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
index ba345b2a00b..aba74368802 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
@@ -710,9 +710,15 @@ func TestPlanBuilderFilterComparison(t *testing.T) {
outFilters: []Filter{{Opcode: LessThan, ColNum: 0, Value: sqltypes.NewInt64(2)},
{Opcode: LessThanEqual, ColNum: 1, Value: sqltypes.NewVarChar("xyz")},
},
+ }, {
+ name: "in-operator",
+ inFilter: "select * from t1 where id in (1, 2)",
+ outFilters: []Filter{
+ {Opcode: In, ColNum: 0, Values: []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewInt64(2)}},
+ },
}, {
name: "vindex-and-operators",
- inFilter: "select * from t1 where in_keyrange(id, 'hash', '-80') and id = 2 and val <> 'xyz'",
+ inFilter: "select * from t1 where in_keyrange(id, 'hash', '-80') and id = 2 and val <> 'xyz' and id in (100, 30)",
outFilters: []Filter{
{
Opcode: VindexMatch,
@@ -727,6 +733,7 @@ func TestPlanBuilderFilterComparison(t *testing.T) {
},
{Opcode: Equal, ColNum: 0, Value: sqltypes.NewInt64(2)},
{Opcode: NotEqual, ColNum: 1, Value: sqltypes.NewVarChar("xyz")},
+ {Opcode: In, ColNum: 0, Values: []sqltypes.Value{sqltypes.NewInt64(100), sqltypes.NewInt64(30)}},
},
}}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
index b6294cd1939..ca3929979c3 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
@@ -25,6 +25,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/sqlparser"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -55,7 +56,7 @@ func newResultStreamer(ctx context.Context, cp dbconfigs.Connector, query string
query: query,
send: send,
vse: vse,
- pktsize: DefaultPacketSizer(),
+ pktsize: DefaultPacketSizer(vttablet.VStreamerUseDynamicPacketSize, vttablet.VStreamerDefaultPacketSize),
}
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
index 6015590dad7..31c47674233 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
@@ -33,7 +33,7 @@ import (
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
@@ -79,14 +79,20 @@ type rowStreamer struct {
vse *Engine
pktsize PacketSizer
- mode RowStreamerMode
- conn *snapshotConn
+ mode RowStreamerMode
+ conn *snapshotConn
+ options *binlogdatapb.VStreamOptions
+ config *vttablet.VReplicationConfig
}
func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string,
lastpk []sqltypes.Value, vschema *localVSchema, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine,
- mode RowStreamerMode, conn *snapshotConn) *rowStreamer {
+ mode RowStreamerMode, conn *snapshotConn, options *binlogdatapb.VStreamOptions) *rowStreamer {
+ config, err := GetVReplicationConfig(options)
+ if err != nil {
+ return nil
+ }
ctx, cancel := context.WithCancel(ctx)
return &rowStreamer{
ctx: ctx,
@@ -98,9 +104,11 @@ func newRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engi
send: send,
vschema: vschema,
vse: vse,
- pktsize: DefaultPacketSizer(),
+ pktsize: DefaultPacketSizer(config.VStreamDynamicPacketSize, config.VStreamPacketSize),
mode: mode,
conn: conn,
+ options: options,
+ config: config,
}
}
@@ -128,10 +136,10 @@ func (rs *rowStreamer) Stream() error {
if _, err := rs.conn.ExecuteFetch("set names 'binary'", 1, false); err != nil {
return err
}
- if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v", vttablet.VReplicationNetReadTimeout), 1, false); err != nil {
+ if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v", rs.config.NetReadTimeout), 1, false); err != nil {
return err
}
- if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v", vttablet.VReplicationNetWriteTimeout), 1, false); err != nil {
+ if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v", rs.config.NetReadTimeout), 1, false); err != nil {
return err
}
}
@@ -240,7 +248,7 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err
func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error) {
buf := sqlparser.NewTrackedBuffer(nil)
// We could have used select *, but being explicit is more predictable.
- buf.Myprintf("select %s", GetVReplicationMaxExecutionTimeQueryHint())
+ buf.Myprintf("select %s", GetVReplicationMaxExecutionTimeQueryHint(rs.config.CopyPhaseDuration))
prefix := ""
for _, col := range rs.plan.Table.Fields {
if rs.plan.isConvertColumnUsingUTF8(col.Name) {
@@ -455,6 +463,6 @@ func (rs *rowStreamer) streamQuery(send func(*binlogdatapb.VStreamRowsResponse)
return nil
}
-func GetVReplicationMaxExecutionTimeQueryHint() string {
- return fmt.Sprintf("/*+ MAX_EXECUTION_TIME(%v) */ ", vttablet.CopyPhaseDuration.Milliseconds())
+func GetVReplicationMaxExecutionTimeQueryHint(copyPhaseDuration time.Duration) string {
+ return fmt.Sprintf("/*+ MAX_EXECUTION_TIME(%v) */ ", copyPhaseDuration.Milliseconds())
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
index 48d11d9e856..47db5fb839a 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
@@ -20,8 +20,11 @@ import (
"context"
"fmt"
"regexp"
+ "strconv"
"testing"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
+
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/mysql"
@@ -79,7 +82,7 @@ func TestRowStreamerQuery(t *testing.T) {
})
}
return nil
- })
+ }, nil)
require.NoError(t, err)
}
@@ -199,7 +202,7 @@ func TestStreamRowsScan(t *testing.T) {
`fields:{name:"id" type:INT32 table:"t3" org_table:"t3" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t3" org_table:"t3" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"} pkfields:{name:"id" type:INT32 charset:63} pkfields:{name:"val" type:VARBINARY charset:63}`,
`rows:{lengths:1 lengths:3 values:"2bbb"} lastpk:{lengths:1 lengths:3 values:"2bbb"}`,
}
- wantQuery = "select /*+ MAX_EXECUTION_TIME(3600000) */ id, val from t3 where (id = 1 and val > 'aaa') or (id > 1) order by id, val"
+ wantQuery = "select /*+ MAX_EXECUTION_TIME(3600000) */ id, val from t3 where (id = 1 and val > _binary'aaa') or (id > 1) order by id, val"
checkStream(t, "select * from t3", []sqltypes.Value{sqltypes.NewInt64(1), sqltypes.NewVarBinary("aaa")}, wantQuery, wantStream)
// t4: all rows
@@ -285,7 +288,7 @@ func TestStreamRowsUnicode(t *testing.T) {
t.Errorf("rows.Rows[0].Values: %s, want %s", got, want)
}
return nil
- })
+ }, nil)
require.NoError(t, err)
}
@@ -423,10 +426,14 @@ func TestStreamRowsCancel(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
+ var options binlogdatapb.VStreamOptions
+ options.ConfigOverrides = make(map[string]string)
+ options.ConfigOverrides["vstream_dynamic_packet_size"] = "false"
+ options.ConfigOverrides["vstream_packet_size"] = "10"
err := engine.StreamRows(ctx, "select * from t1", nil, func(rows *binlogdatapb.VStreamRowsResponse) error {
cancel()
return nil
- })
+ }, &options)
if got, want := err.Error(), "stream ended: context canceled"; got != want {
t.Errorf("err: %v, want %s", err, want)
}
@@ -442,6 +449,10 @@ func checkStream(t *testing.T, query string, lastpk []sqltypes.Value, wantQuery
go func() {
first := true
defer close(ch)
+ var options binlogdatapb.VStreamOptions
+ options.ConfigOverrides = make(map[string]string)
+ options.ConfigOverrides["vstream_dynamic_packet_size"] = strconv.FormatBool(vttablet.VStreamerUseDynamicPacketSize)
+ options.ConfigOverrides["vstream_packet_size"] = strconv.Itoa(vttablet.VStreamerDefaultPacketSize)
err := engine.StreamRows(context.Background(), query, lastpk, func(rows *binlogdatapb.VStreamRowsResponse) error {
if first {
if rows.Gtid == "" {
@@ -469,7 +480,7 @@ func checkStream(t *testing.T, query string, lastpk []sqltypes.Value, wantQuery
}
i++
return nil
- })
+ }, &options)
if err != nil {
ch <- err
}
@@ -486,7 +497,7 @@ func expectStreamError(t *testing.T, query string, want string) {
defer close(ch)
err := engine.StreamRows(context.Background(), query, nil, func(rows *binlogdatapb.VStreamRowsResponse) error {
return nil
- })
+ }, nil)
require.EqualError(t, err, want, "Got incorrect error")
}()
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
index f3eda83bd45..ee141ce9859 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
@@ -25,23 +25,17 @@ import (
"github.com/spf13/pflag"
+ "vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/mysql/sqlerror"
-
- "vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
)
-// If the current binary log is greater than this byte size, we
-// will attempt to rotate it before starting a GTID snapshot
-// based stream.
-// Default is 64MiB.
-var binlogRotationThreshold = int64(64 * 1024 * 1024) // 64MiB
-
// snapshotConn is wrapper on mysql.Conn capable of
// reading a table along with a GTID snapshot.
type snapshotConn struct {
@@ -55,7 +49,7 @@ func init() {
}
func registerSnapshotConnFlags(fs *pflag.FlagSet) {
- fs.Int64Var(&binlogRotationThreshold, "vstream-binlog-rotation-threshold", binlogRotationThreshold, "Byte size at which a VStreamer will attempt to rotate the source's open binary log before starting a GTID snapshot based stream (e.g. a ResultStreamer or RowStreamer)")
+ fs.Int64Var(&vttablet.VStreamerBinlogRotationThreshold, "vstream-binlog-rotation-threshold", vttablet.VStreamerBinlogRotationThreshold, "Byte size at which a VStreamer will attempt to rotate the source's open binary log before starting a GTID snapshot based stream (e.g. a ResultStreamer or RowStreamer)")
}
func snapshotConnect(ctx context.Context, cp dbconfigs.Connector) (*snapshotConn, error) {
@@ -138,29 +132,6 @@ func (conn *snapshotConn) startSnapshot(ctx context.Context, table string) (gtid
return replication.EncodePosition(mpos), nil
}
-// startSnapshotWithConsistentGTID performs the snapshotting without locking tables. This assumes
-// session_track_gtids = START_GTID, which is a contribution to MySQL and is not in vanilla MySQL at the
-// time of this writing.
-func (conn *snapshotConn) startSnapshotWithConsistentGTID(ctx context.Context) (gtid string, err error) {
- if _, err := conn.ExecuteFetch("set transaction isolation level repeatable read", 1, false); err != nil {
- return "", err
- }
- result, err := conn.ExecuteFetch("start transaction with consistent snapshot, read only", 1, false)
- if err != nil {
- return "", err
- }
- // The "session_track_gtids = START_GTID" patch is only applicable to MySQL56 GTID, which is
- // why we hardcode the position as mysql.Mysql56FlavorID
- mpos, err := replication.ParsePosition(replication.Mysql56FlavorID, result.SessionStateChanges)
- if err != nil {
- return "", err
- }
- if _, err := conn.ExecuteFetch("set @@session.time_zone = '+00:00'", 1, false); err != nil {
- return "", err
- }
- return replication.EncodePosition(mpos), nil
-}
-
// Close rolls back any open transactions and closes the connection.
func (conn *snapshotConn) Close() {
_, _ = conn.ExecuteFetch("rollback", 1, false)
@@ -189,7 +160,7 @@ func (conn *snapshotConn) limitOpenBinlogSize() (bool, error) {
if err != nil {
return rotatedLog, err
}
- if curLogSize > atomic.LoadInt64(&binlogRotationThreshold) {
+ if curLogSize > atomic.LoadInt64(&vttablet.VStreamerBinlogRotationThreshold) {
if _, err = conn.ExecuteFetch("FLUSH BINARY LOGS", 0, false); err != nil {
return rotatedLog, err
}
@@ -202,14 +173,14 @@ func (conn *snapshotConn) limitOpenBinlogSize() (bool, error) {
// will attempt to rotate the binary log before starting a GTID snapshot based
// stream (e.g. a ResultStreamer or RowStreamer).
func GetBinlogRotationThreshold() int64 {
- return atomic.LoadInt64(&binlogRotationThreshold)
+ return atomic.LoadInt64(&vttablet.VStreamerBinlogRotationThreshold)
}
// SetBinlogRotationThreshold sets the byte size at which a VStreamer will
// attempt to rotate the binary log before starting a GTID snapshot based
// stream (e.g. a ResultStreamer or RowStreamer).
func SetBinlogRotationThreshold(threshold int64) {
- atomic.StoreInt64(&binlogRotationThreshold, threshold)
+ atomic.StoreInt64(&vttablet.VStreamerBinlogRotationThreshold, threshold)
}
// startSnapshotAllTables starts a streaming query with a snapshot view of all tables, returning the
diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
index d3bbd136f12..6ed291387e1 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
@@ -28,7 +28,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/mysqlctl/tmutils"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -60,10 +60,17 @@ type tableStreamer struct {
snapshotConn *snapshotConn
tables []string
gtid string
+ options *binlogdatapb.VStreamOptions
+ config *vttablet.VReplicationConfig
}
func newTableStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, vschema *localVSchema,
- send func(response *binlogdatapb.VStreamTablesResponse) error, vse *Engine) *tableStreamer {
+ send func(response *binlogdatapb.VStreamTablesResponse) error, vse *Engine, options *binlogdatapb.VStreamOptions) *tableStreamer {
+
+ config, err := GetVReplicationConfig(options)
+ if err != nil {
+ return nil
+ }
ctx, cancel := context.WithCancel(ctx)
return &tableStreamer{
ctx: ctx,
@@ -73,6 +80,8 @@ func newTableStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.En
send: send,
vschema: vschema,
vse: vse,
+ options: options,
+ config: config,
}
}
@@ -103,10 +112,10 @@ func (ts *tableStreamer) Stream() error {
if _, err := conn.ExecuteFetch("set names 'binary'", 1, false); err != nil {
return err
}
- if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v", vttablet.VReplicationNetReadTimeout), 1, false); err != nil {
+ if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_read_timeout = %v", ts.config.NetReadTimeout), 1, false); err != nil {
return err
}
- if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v", vttablet.VReplicationNetWriteTimeout), 1, false); err != nil {
+ if _, err := conn.ExecuteFetch(fmt.Sprintf("set @@session.net_write_timeout = %v", ts.config.NetWriteTimeout), 1, false); err != nil {
return err
}
@@ -149,7 +158,7 @@ func (ts *tableStreamer) newRowStreamer(ctx context.Context, query string, lastp
defer vse.mu.Unlock()
rowStreamer := newRowStreamer(ctx, vse.env.Config().DB.FilteredWithDB(), vse.se, query, lastpk, vse.lvschema,
- send, vse, RowStreamerModeAllTables, ts.snapshotConn)
+ send, vse, RowStreamerModeAllTables, ts.snapshotConn, ts.options)
idx := vse.streamIdx
vse.rowStreamers[idx] = rowStreamer
diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go
index 9be3940c01d..8fe3849b8a4 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer_test.go
@@ -69,7 +69,7 @@ func TestTableStreamer(t *testing.T) {
}
gotStream = append(gotStream, fmt.Sprintf("%v", response))
return nil
- })
+ }, nil)
require.NoError(t, err)
require.EqualValues(t, wantStream, gotStream)
require.Equal(t, int64(4), engine.tableStreamerNumTables.Get())
diff --git a/go/vt/vttablet/tabletserver/vstreamer/utils.go b/go/vt/vttablet/tabletserver/vstreamer/utils.go
new file mode 100644
index 00000000000..9597f80c07c
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/vstreamer/utils.go
@@ -0,0 +1,37 @@
+/*
+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 vstreamer
+
+import (
+ "vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/vterrors"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
+
+ binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+)
+
+func GetVReplicationConfig(options *binlogdatapb.VStreamOptions) (*vttablet.VReplicationConfig, error) {
+ if options == nil {
+ return vttablet.InitVReplicationConfigDefaults(), nil
+ }
+ config, err := vttablet.NewVReplicationConfig(options.ConfigOverrides)
+ if err != nil {
+ log.Errorf("Error parsing VReplicationConfig: %v", err)
+ return nil, vterrors.Wrapf(err, "failed to parse VReplicationConfig")
+ }
+ return config, nil
+}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go
index 854157b1546..ea475d19676 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer.go
@@ -88,7 +88,8 @@ type uvstreamer struct {
config *uvstreamerConfig
- vs *vstreamer // last vstreamer created in uvstreamer
+ vs *vstreamer // last vstreamer created in uvstreamer
+ options *binlogdatapb.VStreamOptions
}
type uvstreamerConfig struct {
@@ -96,7 +97,10 @@ type uvstreamerConfig struct {
CatchupRetryTime time.Duration
}
-func newUVStreamer(ctx context.Context, vse *Engine, cp dbconfigs.Connector, se *schema.Engine, startPos string, tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, vschema *localVSchema, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error) *uvstreamer {
+func newUVStreamer(ctx context.Context, vse *Engine, cp dbconfigs.Connector, se *schema.Engine, startPos string,
+ tablePKs []*binlogdatapb.TableLastPK, filter *binlogdatapb.Filter, vschema *localVSchema,
+ throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, options *binlogdatapb.VStreamOptions) *uvstreamer {
+
ctx, cancel := context.WithCancel(ctx)
config := &uvstreamerConfig{
MaxReplicationLag: 1 * time.Nanosecond,
@@ -123,6 +127,7 @@ func newUVStreamer(ctx context.Context, vse *Engine, cp dbconfigs.Connector, se
config: config,
inTablePKs: tablePKs,
throttlerApp: throttlerApp,
+ options: options,
}
return uvs
@@ -427,7 +432,7 @@ func (uvs *uvstreamer) Stream() error {
}
}
vs := newVStreamer(uvs.ctx, uvs.cp, uvs.se, replication.EncodePosition(uvs.pos), replication.EncodePosition(uvs.stopPos),
- uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send, "replicate", uvs.vse)
+ uvs.filter, uvs.getVSchema(), uvs.throttlerApp, uvs.send, "replicate", uvs.vse, uvs.options)
uvs.setVs(vs)
return vs.Stream()
diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
index 389c06a671e..e2972bb7071 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
@@ -454,7 +454,7 @@ func startVStreamCopy(ctx context.Context, t *testing.T, filter *binlogdatapb.Fi
pos := ""
go func() {
err := engine.Stream(ctx, pos, tablePKs, filter, throttlerapp.VStreamerName, func(evs []*binlogdatapb.VEvent) error {
- //t.Logf("Received events: %v", evs)
+ // t.Logf("Received events: %v", evs)
muAllEvents.Lock()
defer muAllEvents.Unlock()
for _, ev := range evs {
@@ -474,7 +474,7 @@ func startVStreamCopy(ctx context.Context, t *testing.T, filter *binlogdatapb.Fi
allEvents = append(allEvents, ev)
}
return nil
- })
+ }, nil)
require.Nil(t, err)
}()
}
@@ -503,7 +503,7 @@ var expectedEvents = []string{
"type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
"type:ROW row_event:{table_name:\"t1\" row_changes:{after:{lengths:2 lengths:3 values:\"11110\"}}}",
"type:GTID",
- "type:COMMIT", //insert for t2 done along with t1 does not generate an event since t2 is not yet copied
+ "type:COMMIT", // insert for t2 done along with t1 does not generate an event since t2 is not yet copied
fmt.Sprintf("type:OTHER gtid:\"%s t2\"", copyPhaseStart),
"type:BEGIN",
"type:FIELD field_event:{table_name:\"t1\" fields:{name:\"id11\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id11\" column_length:11 charset:63 column_type:\"int(11)\"} fields:{name:\"id12\" type:INT32 table:\"t1\" org_table:\"t1\" database:\"vttest\" org_name:\"id12\" column_length:11 charset:63 column_type:\"int(11)\"} enum_set_string_values:true}",
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
index 634c9a5d40c..59db723ff2b 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
@@ -31,15 +31,15 @@ import (
"vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/sqltypes"
- "vitess.io/vitess/go/timer"
"vitess.io/vitess/go/vt/binlog"
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logutil"
vtschema "vitess.io/vitess/go/vt/schema"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vttablet"
+ vttablet "vitess.io/vitess/go/vt/vttablet/common"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
@@ -84,8 +84,10 @@ type vstreamer struct {
pos replication.Position
stopPos string
- phase string
- vse *Engine
+ phase string
+ vse *Engine
+ options *binlogdatapb.VStreamOptions
+ config *vttablet.VReplicationConfig
}
// streamerPlan extends the original plan to also include
@@ -106,7 +108,7 @@ type streamerPlan struct {
// filter: the list of filtering rules. If a rule has a select expression for its filter,
//
// the select list can only reference direct columns. No other expressions are allowed.
-// The select expression is allowed to contain the special 'keyspace_id()' function which
+// The select expression is allowed to contain the special 'in_keyrange()' function which
// will return the keyspace id of the row. Examples:
// "select * from t", same as an empty Filter,
// "select * from t where in_keyrange('-80')", same as "-80",
@@ -118,7 +120,14 @@ type streamerPlan struct {
//
// vschema: the current vschema. This value can later be changed through the SetVSchema method.
// send: callback function to send events.
-func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, startPos string, stopPos string, filter *binlogdatapb.Filter, vschema *localVSchema, throttlerApp throttlerapp.Name, send func([]*binlogdatapb.VEvent) error, phase string, vse *Engine) *vstreamer {
+func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, startPos string, stopPos string,
+ filter *binlogdatapb.Filter, vschema *localVSchema, throttlerApp throttlerapp.Name,
+ send func([]*binlogdatapb.VEvent) error, phase string, vse *Engine, options *binlogdatapb.VStreamOptions) *vstreamer {
+
+ config, err := GetVReplicationConfig(options)
+ if err != nil {
+ return nil
+ }
ctx, cancel := context.WithCancel(ctx)
return &vstreamer{
ctx: ctx,
@@ -135,6 +144,8 @@ func newVStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine
plans: make(map[uint64]*streamerPlan),
phase: phase,
vse: vse,
+ options: options,
+ config: config,
}
}
@@ -246,7 +257,7 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
return vs.send(vevents)
case binlogdatapb.VEventType_INSERT, binlogdatapb.VEventType_DELETE, binlogdatapb.VEventType_UPDATE, binlogdatapb.VEventType_REPLACE:
newSize := len(vevent.GetDml())
- if curSize+newSize > defaultPacketSize {
+ if curSize+newSize > vs.config.VStreamPacketSize {
vs.vse.vstreamerNumPackets.Add(1)
vevents := bufferedEvents
bufferedEvents = []*binlogdatapb.VEvent{vevent}
@@ -267,7 +278,7 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
newSize += len(rowChange.After.Values)
}
}
- if curSize+newSize > defaultPacketSize {
+ if curSize+newSize > vs.config.VStreamPacketSize {
vs.vse.vstreamerNumPackets.Add(1)
vevents := bufferedEvents
bufferedEvents = []*binlogdatapb.VEvent{vevent}
@@ -288,11 +299,11 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
defer hbTimer.Stop()
injectHeartbeat := func(throttled bool, throttledReason string) error {
- now := time.Now().UnixNano()
select {
case <-ctx.Done():
return vterrors.Errorf(vtrpcpb.Code_CANCELED, "context has expired")
default:
+ now := time.Now().UnixNano()
err := bufferAndTransmit(&binlogdatapb.VEvent{
Type: binlogdatapb.VEventType_HEARTBEAT,
Timestamp: now / 1e9,
@@ -305,24 +316,22 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
}
logger := logutil.NewThrottledLogger(vs.vse.GetTabletInfo(), throttledLoggerInterval)
+ wfNameLog := ""
+ if vs.filter != nil && vs.filter.WorkflowName != "" {
+ wfNameLog = fmt.Sprintf(" in workflow %s", vs.filter.WorkflowName)
+ }
throttleEvents := func(throttledEvents chan mysql.BinlogEvent) {
- throttledHeartbeatsRateLimiter := timer.NewRateLimiter(HeartbeatTime)
- defer throttledHeartbeatsRateLimiter.Stop()
for {
- // check throttler.
+ // Check throttler.
if checkResult, ok := vs.vse.throttlerClient.ThrottleCheckOKOrWaitAppName(ctx, vs.throttlerApp); !ok {
- // make sure to leave if context is cancelled
+ // Make sure to leave if context is cancelled.
select {
case <-ctx.Done():
return
default:
- // do nothing special
+ // Do nothing special.
}
- throttledHeartbeatsRateLimiter.Do(func() error {
- return injectHeartbeat(true, checkResult.Summary())
- })
- // we won't process events, until we're no longer throttling
- logger.Infof("throttled.")
+ logger.Infof("vstreamer throttled%s: %s.", wfNameLog, checkResult.Summary())
continue
}
select {
@@ -366,7 +375,7 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
}
return fmt.Errorf("unexpected server EOF")
}
- vevents, err := vs.parseEvent(ev)
+ vevents, err := vs.parseEvent(ev, bufferAndTransmit)
if err != nil {
vs.vse.errorCounts.Add("ParseEvent", 1)
return err
@@ -394,7 +403,8 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
case <-ctx.Done():
return nil
case <-hbTimer.C:
- if err := injectHeartbeat(false, ""); err != nil {
+ checkResult, ok := vs.vse.throttlerClient.ThrottleCheckOK(ctx, vs.throttlerApp)
+ if err := injectHeartbeat(!ok, checkResult.Summary()); err != nil {
if err == io.EOF {
return nil
}
@@ -406,7 +416,11 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
}
// parseEvent parses an event from the binlog and converts it to a list of VEvents.
-func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, error) {
+// The bufferAndTransmit function must be passed if the event is a TransactionPayloadEvent
+// as for larger payloads (> ZstdInMemoryDecompressorMaxSize) the internal events need
+// to be streamed directly here in order to avoid holding the entire payload's contents,
+// which can be 10s or even 100s of GiBs, all in memory.
+func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent, bufferAndTransmit func(vevent *binlogdatapb.VEvent) error) ([]*binlogdatapb.VEvent, error) {
if !ev.IsValid() {
return nil, fmt.Errorf("can't parse binlog event: invalid data: %#v", ev)
}
@@ -567,26 +581,28 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e
if err != nil {
return nil, err
}
-
if plan, ok := vs.plans[id]; ok {
// When the underlying mysql server restarts the table map can change.
// Usually the vstreamer will also error out when this happens, and vstreamer re-initializes its table map.
// But if the vstreamer is not aware of the restart, we could get an id that matches one in the cache, but
// is for a different table. We then invalidate and recompute the plan for this id.
- if plan == nil || plan.Table.Name == tm.Name {
+ isInternal := tm.Database == sidecar.GetName()
+ if plan == nil ||
+ (plan.Table.Name == tm.Name && isInternal == plan.IsInternal) {
return nil, nil
}
vs.plans[id] = nil
log.Infof("table map changed: id %d for %s has changed to %s", id, plan.Table.Name, tm.Name)
}
- if tm.Database == sidecar.GetName() && tm.Name == "resharding_journal" {
- // A journal is a special case that generates a JOURNAL event.
- return nil, vs.buildJournalPlan(id, tm)
- } else if tm.Database == sidecar.GetName() && tm.Name == "schema_version" && !vs.se.SkipMetaCheck {
- // Generates a Version event when it detects that a schema is stored in the schema_version table.
- return nil, vs.buildVersionPlan(id, tm)
+ // The database connector `vs.cp` points to the keyspace's database.
+ // If this is also setup as the sidecar database name, as is the case in the distributed transaction unit tests,
+ // for example, we stream all tables as usual.
+ // If not, we only stream the schema_version and journal tables and those specified in the internal_tables list.
+ if tm.Database == sidecar.GetName() && vs.cp.DBName() != sidecar.GetName() {
+ return vs.buildSidecarTablePlan(id, tm)
}
+
if tm.Database != "" && tm.Database != vs.cp.DBName() {
vs.plans[id] = nil
return nil, nil
@@ -660,11 +676,31 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e
}
return nil, err
}
- tpvevents, err := vs.parseEvent(tpevent)
+ tpvevents, err := vs.parseEvent(tpevent, nil) // Parse the internal event
if err != nil {
return nil, vterrors.Wrap(err, "failed to parse transaction payload's internal event")
}
- vevents = append(vevents, tpvevents...)
+ if tp.StreamingContents {
+ // Transmit each internal event individually to avoid buffering
+ // the large transaction's entire payload of events in memory, as
+ // the uncompressed size can be 10s or even 100s of GiBs in size.
+ if bufferAndTransmit == nil {
+ return nil, vterrors.New(vtrpcpb.Code_INTERNAL, "[bug] cannot stream compressed transaction payload's internal events as no bufferAndTransmit function was provided")
+ }
+ for _, tpvevent := range tpvevents {
+ tpvevent.Timestamp = int64(ev.Timestamp())
+ tpvevent.CurrentTime = time.Now().UnixNano()
+ if err := bufferAndTransmit(tpvevent); err != nil {
+ if err == io.EOF {
+ return nil, nil
+ }
+ vs.vse.errorCounts.Add("TransactionPayloadBufferAndTransmit", 1)
+ return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "error sending compressed transaction payload's internal event: %v", err)
+ }
+ }
+ } else { // Process the payload's internal events all at once
+ vevents = append(vevents, tpvevents...)
+ }
}
vs.vse.vstreamerCompressedTransactionsDecoded.Add(1)
}
@@ -675,72 +711,84 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e
return vevents, nil
}
-func (vs *vstreamer) buildJournalPlan(id uint64, tm *mysql.TableMap) error {
- conn, err := vs.cp.Connect(vs.ctx)
- if err != nil {
- return err
- }
- defer conn.Close()
- qr, err := conn.ExecuteFetch(sqlparser.BuildParsedQuery("select * from %s.resharding_journal where 1 != 1",
- sidecar.GetIdentifier()).Query, 1, true)
- if err != nil {
- return err
- }
- fields := qr.Fields
- if len(fields) < len(tm.Types) {
- return fmt.Errorf("cannot determine table columns for %s: event has %v, schema has %v", tm.Name, tm.Types, fields)
- }
- table := &Table{
- Name: fmt.Sprintf("%s.resharding_journal", sidecar.GetIdentifier()),
- Fields: fields[:len(tm.Types)],
- }
- // Build a normal table plan, which means, return all rows
- // and columns as is. Special handling is done when we actually
- // receive the row event. We'll build a JOURNAL event instead.
- plan, err := buildREPlan(vs.se.Environment(), table, nil, "")
- if err != nil {
- return err
- }
- vs.plans[id] = &streamerPlan{
- Plan: plan,
- TableMap: tm,
+func (vs *vstreamer) buildSidecarTablePlan(id uint64, tm *mysql.TableMap) ([]*binlogdatapb.VEvent, error) {
+ tableName := tm.Name
+ switch tableName {
+ case "resharding_journal":
+ // A journal is a special case that generates a JOURNAL event.
+ case "schema_version":
+ // Generates a Version event when it detects that a schema is stored in the schema_version table.
+
+ // SkipMetaCheck is set during PITR restore: some table metadata is not fetched in that case.
+ if vs.se.SkipMetaCheck {
+ return nil, nil
+ }
+ default:
+ if vs.options == nil {
+ return nil, nil
+ }
+ found := false
+ for _, table := range vs.options.InternalTables {
+ if table == tableName {
+ found = true
+ break
+ }
+ }
+ if !found {
+ return nil, nil
+ }
}
- vs.journalTableID = id
- return nil
-}
-func (vs *vstreamer) buildVersionPlan(id uint64, tm *mysql.TableMap) error {
conn, err := vs.cp.Connect(vs.ctx)
if err != nil {
- return err
+ return nil, err
}
defer conn.Close()
- qr, err := conn.ExecuteFetch(sqlparser.BuildParsedQuery("select * from %s.schema_version where 1 != 1",
- sidecar.GetIdentifier()).Query, 1, true)
+ qr, err := conn.ExecuteFetch(sqlparser.BuildParsedQuery("select * from %s.%s where 1 != 1",
+ sidecar.GetIdentifier(), tableName).Query, 1, true)
if err != nil {
- return err
+ return nil, err
}
fields := qr.Fields
if len(fields) < len(tm.Types) {
- return fmt.Errorf("cannot determine table columns for %s: event has %v, schema has %v", tm.Name, tm.Types, fields)
+ return nil, fmt.Errorf("cannot determine table columns for %s: event has %v, schema has %v", tm.Name, tm.Types, fields)
}
table := &Table{
- Name: fmt.Sprintf("%s.schema_version", sidecar.GetIdentifier()),
+ Name: tableName,
Fields: fields[:len(tm.Types)],
}
+
// Build a normal table plan, which means, return all rows
- // and columns as is. Special handling is done when we actually
- // receive the row event. We'll build a JOURNAL event instead.
+ // and columns as is. Special handling may be done when we actually
+ // receive the row event, example: we'll build a JOURNAL or VERSION event instead.
plan, err := buildREPlan(vs.se.Environment(), table, nil, "")
if err != nil {
- return err
+ return nil, err
}
+ plan.IsInternal = true
vs.plans[id] = &streamerPlan{
Plan: plan,
TableMap: tm,
}
- vs.versionTableID = id
- return nil
+
+ var vevents []*binlogdatapb.VEvent
+ switch tm.Name {
+ case "resharding_journal":
+ vs.journalTableID = id
+ case "schema_version":
+ vs.versionTableID = id
+ default:
+ vevents = append(vevents, &binlogdatapb.VEvent{
+ Type: binlogdatapb.VEventType_FIELD,
+ FieldEvent: &binlogdatapb.FieldEvent{
+ TableName: tableName,
+ Fields: plan.fields(),
+ Keyspace: vs.vse.keyspace,
+ Shard: vs.vse.shard,
+ IsInternalTable: plan.IsInternal,
+ }})
+ }
+ return vevents, nil
}
func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatapb.VEvent, error) {
@@ -760,7 +808,7 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
vs.plans[id] = nil
return nil, nil
}
- if err := addEnumAndSetMappingstoPlan(plan, cols, tm.Metadata); err != nil {
+ if err := addEnumAndSetMappingstoPlan(vs.se.Environment(), plan, cols, tm.Metadata); err != nil {
return nil, vterrors.Wrapf(err, "failed to build ENUM and SET column integer to string mappings")
}
vs.plans[id] = &streamerPlan{
@@ -980,7 +1028,7 @@ func (vs *vstreamer) processRowEvent(vevents []*binlogdatapb.VEvent, plan *strea
}
if afterOK {
rowChange.After = sqltypes.RowToProto3(afterValues)
- if (vttablet.VReplicationExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage != 0) &&
+ if (vs.config.ExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage != 0) &&
partial {
rowChange.DataColumns = &binlogdatapb.RowChange_Bitmap{
@@ -995,11 +1043,12 @@ func (vs *vstreamer) processRowEvent(vevents []*binlogdatapb.VEvent, plan *strea
vevents = append(vevents, &binlogdatapb.VEvent{
Type: binlogdatapb.VEventType_ROW,
RowEvent: &binlogdatapb.RowEvent{
- TableName: plan.Table.Name,
- RowChanges: rowChanges,
- Keyspace: vs.vse.keyspace,
- Shard: vs.vse.shard,
- Flags: uint32(rows.Flags),
+ TableName: plan.Table.Name,
+ RowChanges: rowChanges,
+ Keyspace: vs.vse.keyspace,
+ Shard: vs.vse.shard,
+ Flags: uint32(rows.Flags),
+ IsInternalTable: plan.IsInternal,
},
})
}
@@ -1043,7 +1092,7 @@ func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataCo
partial := false
for colNum := 0; colNum < dataColumns.Count(); colNum++ {
if !dataColumns.Bit(colNum) {
- if vttablet.VReplicationExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage == 0 {
+ if vs.config.ExperimentalFlags /**/ & /**/ vttablet.VReplicationExperimentalFlagAllowNoBlobBinlogRowImage == 0 {
return false, nil, false, fmt.Errorf("partial row image encountered: ensure binlog_row_image is set to 'full'")
} else {
partial = true
@@ -1073,13 +1122,13 @@ func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataCo
// Convert the integer values in the binlog event for any SET and ENUM fields into their
// string representations.
if plan.Table.Fields[colNum].Type == querypb.Type_ENUM || mysqlType == mysqlbinlog.TypeEnum {
- value, err = buildEnumStringValue(plan, colNum, value)
+ value, err = buildEnumStringValue(vs.se.Environment(), plan, colNum, value)
if err != nil {
return false, nil, false, vterrors.Wrapf(err, "failed to perform ENUM column integer to string value mapping")
}
}
if plan.Table.Fields[colNum].Type == querypb.Type_SET || mysqlType == mysqlbinlog.TypeSet {
- value, err = buildSetStringValue(plan, colNum, value)
+ value, err = buildSetStringValue(vs.se.Environment(), plan, colNum, value)
if err != nil {
return false, nil, false, vterrors.Wrapf(err, "failed to perform SET column integer to string value mapping")
}
@@ -1096,7 +1145,7 @@ func (vs *vstreamer) extractRowAndFilter(plan *streamerPlan, data []byte, dataCo
}
// addEnumAndSetMappingstoPlan sets up any necessary ENUM and SET integer to string mappings.
-func addEnumAndSetMappingstoPlan(plan *Plan, cols []*querypb.Field, metadata []uint16) error {
+func addEnumAndSetMappingstoPlan(env *vtenv.Environment, plan *Plan, cols []*querypb.Field, metadata []uint16) error {
plan.EnumSetValuesMap = make(map[int]map[int]string)
for i, col := range cols {
// If the column is a CHAR based type with a binary collation (e.g. utf8mb4_bin) then
@@ -1115,21 +1164,25 @@ func addEnumAndSetMappingstoPlan(plan *Plan, cols []*querypb.Field, metadata []u
return fmt.Errorf("enum or set column %s does not have valid string values: %s",
col.Name, col.ColumnType)
}
- plan.EnumSetValuesMap[i] = vtschema.ParseEnumOrSetTokensMap(col.ColumnType[begin+1 : end])
+ var err error
+ plan.EnumSetValuesMap[i], err = vtschema.ParseEnumOrSetTokensMap(env, col.ColumnType[begin+1:end])
+ if err != nil {
+ return err
+ }
}
}
return nil
}
// buildEnumStringValue takes the integer value of an ENUM column and returns the string value.
-func buildEnumStringValue(plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
+func buildEnumStringValue(env *vtenv.Environment, plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
if value.IsNull() { // No work is needed
return value, nil
}
// Add the mappings just-in-time in case we haven't properly received and processed a
// table map event to initialize it.
if plan.EnumSetValuesMap == nil {
- if err := addEnumAndSetMappingstoPlan(plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
+ if err := addEnumAndSetMappingstoPlan(env, plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
return sqltypes.Value{}, err
}
}
@@ -1157,14 +1210,14 @@ func buildEnumStringValue(plan *streamerPlan, colNum int, value sqltypes.Value)
}
// buildSetStringValue takes the integer value of a SET column and returns the string value.
-func buildSetStringValue(plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
+func buildSetStringValue(env *vtenv.Environment, plan *streamerPlan, colNum int, value sqltypes.Value) (sqltypes.Value, error) {
if value.IsNull() { // No work is needed
return value, nil
}
// Add the mappings just-in-time in case we haven't properly received and processed a
// table map event to initialize it.
if plan.EnumSetValuesMap == nil {
- if err := addEnumAndSetMappingstoPlan(plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
+ if err := addEnumAndSetMappingstoPlan(env, plan.Plan, plan.Table.Fields, plan.TableMap.Metadata); err != nil {
return sqltypes.Value{}, err
}
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
index 4d9f66f1809..5282b5f372d 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
@@ -20,6 +20,7 @@ import (
"context"
"fmt"
"io"
+ "slices"
"strconv"
"strings"
"sync"
@@ -40,6 +41,7 @@ import (
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ querypb "vitess.io/vitess/go/vt/proto/query"
)
type testcase struct {
@@ -51,10 +53,7 @@ func checkIfOptionIsSupported(t *testing.T, variable string) bool {
qr, err := env.Mysqld.FetchSuperQuery(context.Background(), fmt.Sprintf("show variables like '%s'", variable))
require.NoError(t, err)
require.NotNil(t, qr)
- if qr.Rows != nil && len(qr.Rows) == 1 {
- return true
- }
- return false
+ return len(qr.Rows) == 1
}
// TestPlayerNoBlob sets up a new environment with mysql running with
@@ -89,6 +88,9 @@ func TestNoBlob(t *testing.T) {
"create table t2(id int, txt text, val varchar(4), unique key(id, val))",
// t3 has a text column and a primary key. The text column will not be in update row events.
"create table t3(id int, txt text, val varchar(4), primary key(id))",
+ // t4 has a blob column and a primary key, along with a generated virtual column. The blob
+ // column will not be in update row events.
+ "create table t4(id int, cOl varbinary(8) generated always as (concat(val, 'tsty')) virtual, blb blob, val varbinary(4), primary key(id))",
},
options: &TestSpecOptions{
noblob: true,
@@ -96,6 +98,18 @@ func TestNoBlob(t *testing.T) {
}
defer ts.Close()
ts.Init()
+
+ insertGeneratedFE := &TestFieldEvent{
+ table: "t4",
+ db: testenv.DBName,
+ cols: []*TestColumn{
+ {name: "id", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
+ {name: "cOl", dataType: "VARBINARY", colType: "varbinary(8)", len: 8, collationID: 63},
+ {name: "blb", dataType: "BLOB", colType: "blob", len: 65535, collationID: 63},
+ {name: "val", dataType: "VARBINARY", colType: "varbinary(4)", len: 4, collationID: 63},
+ },
+ }
+
ts.tests = [][]*TestQuery{{
{"begin", nil},
{"insert into t1 values (1, 'blob1', 'aaa')", nil},
@@ -109,6 +123,29 @@ func TestNoBlob(t *testing.T) {
{"insert into t3 values (1, 'text1', 'aaa')", nil},
{"update t3 set val = 'bbb'", nil},
{"commit", nil},
+ }, {{"begin", nil},
+ {"insert into t4 (id, blb, val) values (1, 'text1', 'aaa')", []TestRowEvent{
+ {event: insertGeneratedFE.String()},
+ {spec: &TestRowEventSpec{table: "t4", changes: []TestRowChange{{after: []string{"1", "aaatsty", "text1", "aaa"}}}}},
+ }},
+ {"update t4 set val = 'bbb'", []TestRowEvent{
+ // The blob column is not in the update row event's before or after image.
+ {spec: &TestRowEventSpec{table: "t4", changes: []TestRowChange{{
+ beforeRaw: &querypb.Row{
+ Lengths: []int64{1, 7, -1, 3}, // -1 for the 3rd column / blob field, as it's not present
+ Values: []byte("1aaatstyaaa"),
+ },
+ afterRaw: &querypb.Row{
+ Lengths: []int64{1, 7, -1, 3}, // -1 for the 3rd column / blob field, as it's not present
+ Values: []byte("1bbbtstybbb"),
+ },
+ dataColumnsRaw: &binlogdatapb.RowChange_Bitmap{
+ Count: 4,
+ Cols: []byte{0x0b}, // Columns bitmap of 00001011 as the third column/bit position representing the blob column has no data
+ },
+ }}}},
+ }},
+ {"commit", nil},
}}
ts.Run()
}
@@ -398,6 +435,70 @@ func TestMissingTables(t *testing.T) {
runCases(t, filter, testcases, startPos, nil)
}
+// TestSidecarDBTables tests streaming of sidecar db tables.
+func TestSidecarDBTables(t *testing.T) {
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id11 int, id12 int, primary key(id11))",
+ "create table _vt.internal1(id int, primary key(id))",
+ "create table _vt.internal2(id int, primary key(id))",
+ },
+ }
+ ts.Init()
+ defer func() {
+ execStatements(t, []string{
+ "drop table _vt.internal1",
+ "drop table _vt.internal2",
+ })
+ }()
+ defer ts.Close()
+ position := primaryPosition(t)
+ filter := &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{{
+ Match: "t1",
+ Filter: "select * from t1",
+ }},
+ }
+ execStatements(t, []string{
+ "insert into t1 values (1, 1)",
+ "insert into t1 values (2, 2)",
+ "insert into _vt.internal1 values (1)",
+ "insert into _vt.internal2 values (1)",
+ "insert into _vt.internal2 values (2)",
+ })
+ options := &binlogdatapb.VStreamOptions{
+ InternalTables: []string{"internal1", "internal2"},
+ }
+ ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(10*time.Second))
+ defer cancel()
+ wantRowEvents := map[string]int{
+ "t1": 2,
+ "internal1": 1,
+ "internal2": 2,
+ }
+ gotRowEvents := make(map[string]int)
+ gotFieldEvents := make(map[string]int)
+ err := engine.Stream(ctx, position, nil, filter, "", func(events []*binlogdatapb.VEvent) error {
+ for _, ev := range events {
+ if ev.Type == binlogdatapb.VEventType_ROW {
+ gotRowEvents[ev.RowEvent.TableName]++
+ require.Equal(t, slices.Contains(options.InternalTables, ev.RowEvent.TableName), ev.RowEvent.IsInternalTable)
+ }
+ if ev.Type == binlogdatapb.VEventType_FIELD {
+ require.Equal(t, slices.Contains(options.InternalTables, ev.FieldEvent.TableName), ev.FieldEvent.IsInternalTable)
+ gotFieldEvents[ev.FieldEvent.TableName]++
+ }
+ }
+ return nil
+ }, options)
+ require.NoError(t, err)
+ require.EqualValues(t, wantRowEvents, gotRowEvents)
+ for k, v := range gotFieldEvents {
+ require.Equal(t, 1, v, "gotFieldEvents[%s] = %d", k, v)
+ }
+}
+
// TestVStreamMissingFieldsInLastPK tests that we error out if the lastpk for a table is missing the fields spec.
func TestVStreamMissingFieldsInLastPK(t *testing.T) {
ts := &TestSpec{
@@ -408,6 +509,7 @@ func TestVStreamMissingFieldsInLastPK(t *testing.T) {
}
ts.Init()
defer ts.Close()
+
filter := &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{{
Match: "t1",
@@ -435,7 +537,6 @@ func TestVStreamCopySimpleFlow(t *testing.T) {
}
ts.Init()
defer ts.Close()
-
log.Infof("Pos before bulk insert: %s", primaryPosition(t))
insertSomeRows(t, 10)
log.Infof("Pos after bulk insert: %s", primaryPosition(t))
@@ -663,11 +764,11 @@ func TestVStreamCopyWithDifferentFilters(t *testing.T) {
return io.EOF
}
return nil
- })
+ }, nil)
}()
wg.Wait()
if errGoroutine != nil {
- t.Fatalf(errGoroutine.Error())
+ t.Fatal(errGoroutine.Error())
}
}
@@ -1487,7 +1588,7 @@ func TestBestEffortNameInFieldEvent(t *testing.T) {
// todo: migrate to new framework
// test that vstreamer ignores tables created by OnlineDDL
-func TestInternalTables(t *testing.T) {
+func TestOnlineDDLTables(t *testing.T) {
if version.GoOS == "darwin" {
t.Skip("internal online ddl table matching doesn't work on Mac because it is case insensitive")
}
@@ -1763,7 +1864,7 @@ func TestMinimalMode(t *testing.T) {
engine = oldEngine
env = oldEnv
}()
- err := engine.Stream(context.Background(), "current", nil, nil, throttlerapp.VStreamerName, func(evs []*binlogdatapb.VEvent) error { return nil })
+ err := engine.Stream(context.Background(), "current", nil, nil, throttlerapp.VStreamerName, func(evs []*binlogdatapb.VEvent) error { return nil }, nil)
require.Error(t, err, "minimal binlog_row_image is not supported by Vitess VReplication")
}
@@ -1865,7 +1966,7 @@ func TestFilteredMultipleWhere(t *testing.T) {
filter: &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{{
Match: "t1",
- Filter: "select id1, val from t1 where in_keyrange('-80') and id2 = 200 and id3 = 1000 and val = 'newton'",
+ Filter: "select id1, val from t1 where in_keyrange('-80') and id2 = 200 and id3 = 1000 and val = 'newton' and id1 in (1, 2, 129)",
}},
},
customFieldEvents: true,
@@ -1887,9 +1988,7 @@ func TestFilteredMultipleWhere(t *testing.T) {
{spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"2", "newton"}}}}},
}},
{"insert into t1 values (3, 100, 2000, 'kepler')", noEvents},
- {"insert into t1 values (128, 200, 1000, 'newton')", []TestRowEvent{
- {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"128", "newton"}}}}},
- }},
+ {"insert into t1 values (128, 200, 1000, 'newton')", noEvents},
{"insert into t1 values (5, 200, 2000, 'kepler')", noEvents},
{"insert into t1 values (129, 200, 1000, 'kepler')", noEvents},
{"commit", nil},
@@ -1979,3 +2078,33 @@ func TestGeneratedInvisiblePrimaryKey(t *testing.T) {
}}
ts.Run()
}
+
+func TestFilteredInOperator(t *testing.T) {
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id1 int, id2 int, val varbinary(128), primary key(id1))",
+ },
+ options: &TestSpecOptions{
+ filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{{
+ Match: "t1",
+ Filter: "select id1, val from t1 where val in ('eee', 'bbb', 'ddd') and id1 in (4, 5)",
+ }},
+ },
+ },
+ }
+ defer ts.Close()
+ ts.Init()
+ ts.fieldEvents["t1"].cols[1].skip = true
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 100, 'aaa')", noEvents},
+ {"insert into t1 values (2, 200, 'bbb')", noEvents},
+ {"insert into t1 values (3, 100, 'ccc')", noEvents},
+ {"insert into t1 values (4, 200, 'ddd')", nil},
+ {"insert into t1 values (5, 200, 'eee')", nil},
+ {"commit", nil},
+ }}
+ ts.Run()
+}
diff --git a/go/vt/vttablet/tabletservermock/controller.go b/go/vt/vttablet/tabletservermock/controller.go
index 7c7055b3e15..9d570b8f6c7 100644
--- a/go/vt/vttablet/tabletservermock/controller.go
+++ b/go/vt/vttablet/tabletservermock/controller.go
@@ -24,6 +24,7 @@ import (
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/mysqlctl"
+ "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/vttablet/queryservice"
@@ -91,6 +92,8 @@ type Controller struct {
// queryRulesMap has the latest query rules.
queryRulesMap map[string]*rules.Rules
+
+ MethodCalled map[string]bool
}
// NewController returns a mock of tabletserver.Controller
@@ -101,6 +104,7 @@ func NewController() *Controller {
BroadcastData: make(chan *BroadcastData, 10),
StateChanges: make(chan *StateChange, 10),
queryRulesMap: make(map[string]*rules.Rules),
+ MethodCalled: make(map[string]bool),
}
}
@@ -226,6 +230,50 @@ func (tqsc *Controller) GetThrottlerStatus(ctx context.Context) *throttle.Thrott
return nil
}
+// RedoPreparedTransactions is part of the tabletserver.Controller interface
+func (tqsc *Controller) RedoPreparedTransactions() {}
+
+// SetTwoPCAllowed sets whether TwoPC is allowed or not. It also takes the reason of why it is being set.
+// The reason should be an enum value defined in the tabletserver.
+func (tqsc *Controller) SetTwoPCAllowed(int, bool) {
+}
+
+// UnresolvedTransactions is part of the tabletserver.Controller interface
+func (tqsc *Controller) UnresolvedTransactions(context.Context, *querypb.Target, int64) ([]*querypb.TransactionMetadata, error) {
+ tqsc.MethodCalled["UnresolvedTransactions"] = true
+ return nil, nil
+}
+
+// ReadTransaction is part of the tabletserver.Controller interface
+func (tqsc *Controller) ReadTransaction(ctx context.Context, target *querypb.Target, dtid string) (*querypb.TransactionMetadata, error) {
+ tqsc.MethodCalled["ReadTransaction"] = true
+ return nil, nil
+}
+
+// GetTransactionInfo is part of the tabletserver.Controller interface
+func (tqsc *Controller) GetTransactionInfo(ctx context.Context, target *querypb.Target, dtid string) (*tabletmanagerdata.GetTransactionInfoResponse, error) {
+ tqsc.MethodCalled["GetTransactionInfo"] = true
+ return nil, nil
+}
+
+// ConcludeTransaction is part of the tabletserver.Controller interface
+func (tqsc *Controller) ConcludeTransaction(context.Context, *querypb.Target, string) error {
+ tqsc.MethodCalled["ConcludeTransaction"] = true
+ return nil
+}
+
+// RollbackPrepared is part of the tabletserver.Controller interface
+func (tqsc *Controller) RollbackPrepared(context.Context, *querypb.Target, string, int64) error {
+ tqsc.MethodCalled["RollbackPrepared"] = true
+ return nil
+}
+
+// WaitForPreparedTwoPCTransactions is part of the tabletserver.Controller interface
+func (tqsc *Controller) WaitForPreparedTwoPCTransactions(context.Context) error {
+ tqsc.MethodCalled["WaitForPreparedTwoPCTransactions"] = true
+ return nil
+}
+
// EnterLameduck implements tabletserver.Controller.
func (tqsc *Controller) EnterLameduck() {
tqsc.mu.Lock()
diff --git a/go/vt/vttablet/tmclient/rpc_client_api.go b/go/vt/vttablet/tmclient/rpc_client_api.go
index 7da1a6196dd..2b5cc967e9f 100644
--- a/go/vt/vttablet/tmclient/rpc_client_api.go
+++ b/go/vt/vttablet/tmclient/rpc_client_api.go
@@ -91,6 +91,9 @@ type TabletManagerClient interface {
// SetReadWrite makes the mysql instance read-write
SetReadWrite(ctx context.Context, tablet *topodatapb.Tablet) error
+ // ChangeTags asks the remote tablet to change its tags
+ ChangeTags(ctx context.Context, tablet *topodatapb.Tablet, tabletTags map[string]string, replace bool) (*tabletmanagerdatapb.ChangeTagsResponse, error)
+
// ChangeType asks the remote tablet to change its type
ChangeType(ctx context.Context, tablet *topodatapb.Tablet, dbType topodatapb.TabletType, semiSync bool) error
@@ -146,6 +149,24 @@ type TabletManagerClient interface {
// query faster. Close() should close the pool in that case.
ExecuteFetchAsApp(ctx context.Context, tablet *topodatapb.Tablet, usePool bool, req *tabletmanagerdatapb.ExecuteFetchAsAppRequest) (*querypb.QueryResult, error)
+ //
+ // Distributed Transaction related methods
+ //
+
+ // GetUnresolvedTransactions returns the list of unresolved transactions for the tablet.
+ GetUnresolvedTransactions(ctx context.Context, tablet *topodatapb.Tablet, abandonAge int64) ([]*querypb.TransactionMetadata, error)
+
+ // ReadTransaction returns the metadata for the specified distributed transaction ID.
+ ReadTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*querypb.TransactionMetadata, error)
+
+ GetTransactionInfo(ctx context.Context, tablet *topodatapb.Tablet, dtid string) (*tabletmanagerdatapb.GetTransactionInfoResponse, error)
+
+ // ConcludeTransaction conclude the transaction on the tablet.
+ ConcludeTransaction(ctx context.Context, tablet *topodatapb.Tablet, dtid string, mm bool) error
+
+ // MysqlHostMetrics returns mysql system metrics
+ MysqlHostMetrics(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error)
+
//
// Replication related methods
//
@@ -186,12 +207,14 @@ type TabletManagerClient interface {
//
CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error)
+ DeleteTableData(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error)
DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error)
HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error)
ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error)
ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error)
UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error)
UpdateVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error)
+ ValidateVReplicationPermissions(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error)
// VReplicationExec executes a VReplication command
VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error)
VReplicationWaitForPos(ctx context.Context, tablet *topodatapb.Tablet, id int32, pos string) error
@@ -215,6 +238,9 @@ type TabletManagerClient interface {
// its reparent_journal table.
PopulateReparentJournal(ctx context.Context, tablet *topodatapb.Tablet, timeCreatedNS int64, actionName string, tabletAlias *topodatapb.TabletAlias, pos string) error
+ // ReadReparentJournalInfo reads the information from reparent journal
+ ReadReparentJournalInfo(ctx context.Context, tablet *topodatapb.Tablet) (int, error)
+
// InitReplica tells a tablet to start replicating from the
// passed in primary tablet alias, and wait for the row in the
// reparent_journal table.
diff --git a/go/vt/vttablet/tmrpctest/test_tm_rpc.go b/go/vt/vttablet/tmrpctest/test_tm_rpc.go
index 9ba01b13d5a..a106b43bf2c 100644
--- a/go/vt/vttablet/tmrpctest/test_tm_rpc.go
+++ b/go/vt/vttablet/tmrpctest/test_tm_rpc.go
@@ -26,6 +26,7 @@ import (
"testing"
"time"
+ "github.com/stretchr/testify/require"
"google.golang.org/protobuf/proto"
"vitess.io/vitess/go/mysql/replication"
@@ -56,47 +57,57 @@ type fakeRPCTM struct {
}
func (fra *fakeRPCTM) CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) {
- //TODO implement me
+ // TODO implement me
+ panic("implement me")
+}
+
+func (fra *fakeRPCTM) DeleteTableData(ctx context.Context, req *tabletmanagerdatapb.DeleteTableDataRequest) (*tabletmanagerdatapb.DeleteTableDataResponse, error) {
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) HasVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) ReadVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) ResetSequences(ctx context.Context, tables []string) error {
- //TODO implement me
+ // TODO implement me
+ panic("implement me")
+}
+
+func (fra *fakeRPCTM) ValidateVReplicationPermissions(ctx context.Context, req *tabletmanagerdatapb.ValidateVReplicationPermissionsRequest) (*tabletmanagerdatapb.ValidateVReplicationPermissionsResponse, error) {
+ // TODO implement me
panic("implement me")
}
func (fra *fakeRPCTM) VDiff(ctx context.Context, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) {
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
@@ -108,6 +119,11 @@ func (fra *fakeRPCTM) UnlockTables(ctx context.Context) error {
panic("implement me")
}
+func (fra *fakeRPCTM) MysqlHostMetrics(ctx context.Context, req *tabletmanagerdatapb.MysqlHostMetricsRequest) (*tabletmanagerdatapb.MysqlHostMetricsResponse, error) {
+ // TODO implement me
+ panic("implement me")
+}
+
func (fra *fakeRPCTM) setSlow(slow bool) {
fra.mu.Lock()
fra.slow = slow
@@ -404,6 +420,36 @@ func tmRPCTestGetGlobalStatusVarsPanic(ctx context.Context, t *testing.T, client
expectHandleRPCPanic(t, "GetGlobalStatusVars", false /*verbose*/, err)
}
+func tmRPCTestGetUnresolvedTransactions(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.GetUnresolvedTransactions(ctx, tablet, 0)
+ require.NoError(t, err)
+}
+
+func tmRPCTestGetUnresolvedTransactionsPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.GetUnresolvedTransactions(ctx, tablet, 0)
+ expectHandleRPCPanic(t, "GetUnresolvedTransactions", false /*verbose*/, err)
+}
+
+func tmRPCTestReadTransaction(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.ReadTransaction(ctx, tablet, "aa")
+ require.NoError(t, err)
+}
+
+func tmRPCTestReadTransactionPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.ReadTransaction(ctx, tablet, "aa")
+ expectHandleRPCPanic(t, "ReadTransaction", false /*verbose*/, err)
+}
+
+func tmRPCTestGetTransactionInfo(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.GetTransactionInfo(ctx, tablet, "aa")
+ require.NoError(t, err)
+}
+
+func tmRPCTestGetTransactionInfoPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.GetTransactionInfo(ctx, tablet, "aa")
+ expectHandleRPCPanic(t, "GetTransactionInfo", false /*verbose*/, err)
+}
+
//
// Various read-write methods
//
@@ -440,6 +486,30 @@ func tmRPCTestSetReadOnlyPanic(ctx context.Context, t *testing.T, client tmclien
expectHandleRPCPanic(t, "SetReadWrite", true /*verbose*/, err)
}
+var testChangeTagsValue = map[string]string{
+ "test": "12345",
+}
+
+func (fra *fakeRPCTM) ChangeTags(ctx context.Context, tabletTags map[string]string, replace bool) (map[string]string, error) {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ compare(fra.t, "ChangeTags tabletType", tabletTags, testChangeTagsValue)
+ return tabletTags, nil
+}
+
+func tmRPCTestChangeTags(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.ChangeTags(ctx, tablet, testChangeTagsValue, false)
+ if err != nil {
+ t.Errorf("ChangeTags failed: %v", err)
+ }
+}
+
+func tmRPCTestChangeTagsPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.ChangeTags(ctx, tablet, testChangeTagsValue, false)
+ expectHandleRPCPanic(t, "ChangeTags", true /*verbose*/, err)
+}
+
var testChangeTypeValue = topodatapb.TabletType_REPLICA
func (fra *fakeRPCTM) ChangeType(ctx context.Context, tabletType topodatapb.TabletType, semiSync bool) error {
@@ -726,6 +796,34 @@ func (fra *fakeRPCTM) ExecuteFetchAsApp(ctx context.Context, req *tabletmanagerd
return testExecuteFetchResult, nil
}
+func (fra *fakeRPCTM) GetUnresolvedTransactions(ctx context.Context, abandonAgeSeconds int64) ([]*querypb.TransactionMetadata, error) {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ return nil, nil
+}
+
+func (fra *fakeRPCTM) ReadTransaction(ctx context.Context, req *tabletmanagerdatapb.ReadTransactionRequest) (*querypb.TransactionMetadata, error) {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ return nil, nil
+}
+
+func (fra *fakeRPCTM) GetTransactionInfo(ctx context.Context, req *tabletmanagerdatapb.GetTransactionInfoRequest) (*tabletmanagerdatapb.GetTransactionInfoResponse, error) {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ return nil, nil
+}
+
+func (fra *fakeRPCTM) ConcludeTransaction(ctx context.Context, req *tabletmanagerdatapb.ConcludeTransactionRequest) error {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ return nil
+}
+
func tmRPCTestExecuteFetch(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
// using pool
qr, err := client.ExecuteFetchAsDba(ctx, tablet, true, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{
@@ -1081,6 +1179,15 @@ func (fra *fakeRPCTM) PopulateReparentJournal(ctx context.Context, timeCreatedNS
return nil
}
+var testReparentJournalLen = 10
+
+func (fra *fakeRPCTM) ReadReparentJournalInfo(context.Context) (int, error) {
+ if fra.panics {
+ panic(fmt.Errorf("test-triggered panic"))
+ }
+ return testReparentJournalLen, nil
+}
+
func tmRPCTestPopulateReparentJournal(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
err := client.PopulateReparentJournal(ctx, tablet, testTimeCreatedNS, testActionName, testPrimaryAlias, testReplicationPosition)
compareError(t, "PopulateReparentJournal", err, true, testPopulateReparentJournalCalled)
@@ -1091,6 +1198,16 @@ func tmRPCTestPopulateReparentJournalPanic(ctx context.Context, t *testing.T, cl
expectHandleRPCPanic(t, "PopulateReparentJournal", true /*verbose*/, err)
}
+func tmRPCTestReadReparentJournalInfo(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ length, err := client.ReadReparentJournalInfo(ctx, tablet)
+ compareError(t, "ReadReparentJournalInfo", err, length, testReparentJournalLen)
+}
+
+func tmRPCTestReadReparentJournalInfoPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ _, err := client.ReadReparentJournalInfo(ctx, tablet)
+ expectHandleRPCPanic(t, "ReadReparentJournalInfo", true /*verbose*/, err)
+}
+
func tmRPCTestWaitForPositionPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
err := client.WaitForPosition(ctx, tablet, testReplicationPosition)
expectHandleRPCPanic(t, "WaitForPosition", true /*verbose*/, err)
@@ -1311,6 +1428,10 @@ func (fra *fakeRPCTM) Backup(ctx context.Context, logger logutil.Logger, request
return nil
}
+func (fra *fakeRPCTM) IsBackupRunning() bool {
+ return false
+}
+
func tmRPCTestBackup(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
req := &tabletmanagerdatapb.BackupRequest{Concurrency: testBackupConcurrency, AllowPrimary: testBackupAllowPrimary}
stream, err := client.Backup(ctx, tablet, req)
@@ -1348,7 +1469,7 @@ func (fra *fakeRPCTM) CheckThrottler(ctx context.Context, req *tabletmanagerdata
panic(fmt.Errorf("test-triggered panic"))
}
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
@@ -1357,7 +1478,7 @@ func (fra *fakeRPCTM) GetThrottlerStatus(ctx context.Context, req *tabletmanager
panic(fmt.Errorf("test-triggered panic"))
}
- //TODO implement me
+ // TODO implement me
panic("implement me")
}
@@ -1424,6 +1545,9 @@ func Run(t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.T
tmRPCTestGetSchema(ctx, t, client, tablet)
tmRPCTestGetPermissions(ctx, t, client, tablet)
tmRPCTestGetGlobalStatusVars(ctx, t, client, tablet)
+ tmRPCTestGetUnresolvedTransactions(ctx, t, client, tablet)
+ tmRPCTestReadTransaction(ctx, t, client, tablet)
+ tmRPCTestGetTransactionInfo(ctx, t, client, tablet)
// Various read-write methods
tmRPCTestSetReadOnly(ctx, t, client, tablet)
@@ -1457,6 +1581,7 @@ func Run(t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.T
tmRPCTestResetReplication(ctx, t, client, tablet)
tmRPCTestInitPrimary(ctx, t, client, tablet)
tmRPCTestPopulateReparentJournal(ctx, t, client, tablet)
+ tmRPCTestReadReparentJournalInfo(ctx, t, client, tablet)
tmRPCTestDemotePrimary(ctx, t, client, tablet)
tmRPCTestUndoDemotePrimary(ctx, t, client, tablet)
tmRPCTestSetReplicationSource(ctx, t, client, tablet)
@@ -1485,6 +1610,9 @@ func Run(t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.T
tmRPCTestGetSchemaPanic(ctx, t, client, tablet)
tmRPCTestGetPermissionsPanic(ctx, t, client, tablet)
tmRPCTestGetGlobalStatusVarsPanic(ctx, t, client, tablet)
+ tmRPCTestGetUnresolvedTransactionsPanic(ctx, t, client, tablet)
+ tmRPCTestReadTransactionPanic(ctx, t, client, tablet)
+ tmRPCTestGetTransactionInfoPanic(ctx, t, client, tablet)
// Various read-write methods
tmRPCTestSetReadOnlyPanic(ctx, t, client, tablet)
@@ -1514,6 +1642,7 @@ func Run(t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.T
tmRPCTestResetReplicationPanic(ctx, t, client, tablet)
tmRPCTestInitPrimaryPanic(ctx, t, client, tablet)
tmRPCTestPopulateReparentJournalPanic(ctx, t, client, tablet)
+ tmRPCTestReadReparentJournalInfoPanic(ctx, t, client, tablet)
tmRPCTestWaitForPositionPanic(ctx, t, client, tablet)
tmRPCTestDemotePrimaryPanic(ctx, t, client, tablet)
tmRPCTestUndoDemotePrimaryPanic(ctx, t, client, tablet)
diff --git a/go/vt/vttest/local_cluster.go b/go/vt/vttest/local_cluster.go
index 406269ef749..fa7e40feb12 100644
--- a/go/vt/vttest/local_cluster.go
+++ b/go/vt/vttest/local_cluster.go
@@ -292,6 +292,15 @@ func (db *LocalCluster) MySQLConnParams() mysql.ConnParams {
return connParams
}
+func (db *LocalCluster) MySQLTCPConnParams() mysql.ConnParams {
+ connParams := db.mysql.Params(db.DbName())
+ _, port := db.mysql.Address()
+ connParams.UnixSocket = ""
+ connParams.Host = "127.0.0.1"
+ connParams.Port = port
+ return connParams
+}
+
// MySQLAppDebugConnParams returns a mysql.ConnParams struct that can be used
// to connect directly to the mysqld service in the self-contained cluster,
// using the appdebug user. It's valid only if you used MySQLOnly option.
@@ -651,6 +660,7 @@ func (db *LocalCluster) JSONConfig() any {
config := map[string]any{
"bind_address": db.vt.BindAddress,
"port": db.vt.Port,
+ "grpc_bind_address": db.vt.BindAddressGprc,
"socket": db.mysql.UnixSocket(),
"vtcombo_mysql_port": db.Env.PortForProtocol("vtcombo_mysql_port", ""),
"mysql": db.Env.PortForProtocol("mysql", ""),
diff --git a/go/vt/vttest/vtprocess.go b/go/vt/vttest/vtprocess.go
index 3f34994bb75..6371811a60e 100644
--- a/go/vt/vttest/vtprocess.go
+++ b/go/vt/vttest/vtprocess.go
@@ -44,16 +44,17 @@ type HealthChecker func(addr string) bool
// It can be spawned manually or through one of the available
// helper methods.
type VtProcess struct {
- Name string
- Directory string
- LogDirectory string
- Binary string
- ExtraArgs []string
- Env []string
- BindAddress string
- Port int
- PortGrpc int
- HealthCheck HealthChecker
+ Name string
+ Directory string
+ LogDirectory string
+ Binary string
+ ExtraArgs []string
+ Env []string
+ BindAddress string
+ BindAddressGprc string
+ Port int
+ PortGrpc int
+ HealthCheck HealthChecker
proc *exec.Cmd
exit chan error
@@ -139,6 +140,11 @@ func (vtp *VtProcess) WaitStart() (err error) {
vtp.proc.Args = append(vtp.proc.Args, fmt.Sprintf("%d", vtp.PortGrpc))
}
+ if vtp.BindAddressGprc != "" {
+ vtp.proc.Args = append(vtp.proc.Args, "--grpc_bind_address")
+ vtp.proc.Args = append(vtp.proc.Args, vtp.BindAddressGprc)
+ }
+
vtp.proc.Args = append(vtp.proc.Args, vtp.ExtraArgs...)
vtp.proc.Env = append(vtp.proc.Env, os.Environ()...)
vtp.proc.Env = append(vtp.proc.Env, vtp.Env...)
@@ -199,16 +205,22 @@ func VtcomboProcess(environment Environment, args *Config, mysql MySQLManager) (
if args.VtComboBindAddress != "" {
vtcomboBindAddress = args.VtComboBindAddress
}
+ grpcBindAddress := "127.0.0.1"
+ if servenv.GRPCBindAddress() != "" {
+ grpcBindAddress = servenv.GRPCBindAddress()
+ }
+
vt := &VtProcess{
- Name: "vtcombo",
- Directory: environment.Directory(),
- LogDirectory: environment.LogDirectory(),
- Binary: environment.BinaryPath("vtcombo"),
- BindAddress: vtcomboBindAddress,
- Port: environment.PortForProtocol("vtcombo", ""),
- PortGrpc: environment.PortForProtocol("vtcombo", "grpc"),
- HealthCheck: environment.ProcessHealthCheck("vtcombo"),
- Env: environment.EnvVars(),
+ Name: "vtcombo",
+ Directory: environment.Directory(),
+ LogDirectory: environment.LogDirectory(),
+ Binary: environment.BinaryPath("vtcombo"),
+ BindAddress: vtcomboBindAddress,
+ BindAddressGprc: grpcBindAddress,
+ Port: environment.PortForProtocol("vtcombo", ""),
+ PortGrpc: environment.PortForProtocol("vtcombo", "grpc"),
+ HealthCheck: environment.ProcessHealthCheck("vtcombo"),
+ Env: environment.EnvVars(),
}
user, pass := mysql.Auth()
diff --git a/go/vt/wrangler/fake_dbclient_test.go b/go/vt/wrangler/fake_dbclient_test.go
index 14ef0913383..02ee79210d7 100644
--- a/go/vt/wrangler/fake_dbclient_test.go
+++ b/go/vt/wrangler/fake_dbclient_test.go
@@ -153,6 +153,10 @@ func (dc *fakeDBClient) Rollback() error {
func (dc *fakeDBClient) Close() {
}
+func (dc *fakeDBClient) IsClosed() bool {
+ return false
+}
+
// ExecuteFetch is part of the DBClient interface
func (dc *fakeDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) {
dc.mu.Lock()
diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go
index a5f7d6ae0bf..98551a084c9 100644
--- a/go/vt/wrangler/keyspace.go
+++ b/go/vt/wrangler/keyspace.go
@@ -19,6 +19,7 @@ package wrangler
import (
"bytes"
"context"
+ "errors"
"fmt"
"sync"
"time"
@@ -82,7 +83,7 @@ func (wr *Wrangler) validateNewWorkflow(ctx context.Context, keyspace, workflow
return
}
if p3qr != nil && len(p3qr.Rows) != 0 {
- allErrors.RecordError(vterrors.Wrap(fmt.Errorf(validation.msg), "validateWorkflowName.VReplicationExec"))
+ allErrors.RecordError(vterrors.Wrap(errors.New(validation.msg), "validateWorkflowName.VReplicationExec"))
return
}
}
diff --git a/go/vt/wrangler/materializer.go b/go/vt/wrangler/materializer.go
index 9367c43c310..bd7ae553130 100644
--- a/go/vt/wrangler/materializer.go
+++ b/go/vt/wrangler/materializer.go
@@ -18,6 +18,7 @@ package wrangler
import (
"context"
+ "errors"
"fmt"
"hash/fnv"
"math"
@@ -355,7 +356,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta
migrationID, strings.Join(tablets, ","))
msg += fmt.Sprintf("please review and delete it before proceeding and restart the workflow using the Workflow %s.%s start",
workflow, targetKeyspace)
- return fmt.Errorf(msg)
+ return errors.New(msg)
}
}
if autoStart {
@@ -550,11 +551,8 @@ func (wr *Wrangler) prepareCreateLookup(ctx context.Context, keyspace string, sp
if len(vindexFromCols) != 1 {
return nil, nil, nil, fmt.Errorf("unique vindex 'from' should have only one column: %v", vindex)
}
- } else {
- if len(vindexFromCols) < 2 {
- return nil, nil, nil, fmt.Errorf("non-unique vindex 'from' should have more than one column: %v", vindex)
- }
}
+
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.
@@ -1397,7 +1395,19 @@ func (mz *materializer) generateInserts(ctx context.Context, sourceShards []*top
for _, mappedCol := range mappedCols {
subExprs = append(subExprs, mappedCol)
}
- vindexName := fmt.Sprintf("%s.%s", mz.ms.TargetKeyspace, cv.Name)
+ var vindexName string
+ if mz.getWorkflowType() == binlogdatapb.VReplicationWorkflowType_Migrate {
+ // For a Migrate, if the TargetKeyspace name is different from the SourceKeyspace name, we need to use the
+ // SourceKeyspace name to determine the vindex since the TargetKeyspace name is not known to the source.
+ // Note: it is expected that the source and target keyspaces have the same vindex name and data type.
+ keyspace := mz.ms.TargetKeyspace
+ if mz.ms.ExternalCluster != "" {
+ keyspace = mz.ms.SourceKeyspace
+ }
+ vindexName = fmt.Sprintf("%s.%s", keyspace, cv.Name)
+ } else {
+ vindexName = fmt.Sprintf("%s.%s", mz.ms.TargetKeyspace, cv.Name)
+ }
subExprs = append(subExprs, sqlparser.NewStrLiteral(vindexName))
subExprs = append(subExprs, sqlparser.NewStrLiteral("{{.keyrange}}"))
inKeyRange := &sqlparser.FuncExpr{
@@ -1442,7 +1452,7 @@ func (mz *materializer) generateInserts(ctx context.Context, sourceShards []*top
ig.AddRow(mz.ms.Workflow, bls, "", mz.ms.Cell, tabletTypeStr,
workflowType,
workflowSubType,
- mz.ms.DeferSecondaryKeys,
+ mz.ms.DeferSecondaryKeys, "",
)
}
return ig.String(), nil
diff --git a/go/vt/wrangler/materializer_test.go b/go/vt/wrangler/materializer_test.go
index 1728ba6efc2..1871d778c6b 100644
--- a/go/vt/wrangler/materializer_test.go
+++ b/go/vt/wrangler/materializer_test.go
@@ -1599,7 +1599,7 @@ func TestCreateLookupVindexFailures(t *testing.T) {
},
err: "unique vindex 'from' should have only one column",
}, {
- description: "non-unique lookup should have more than one column",
+ description: "non-unique lookup can have only one column",
input: &vschemapb.Keyspace{
Vindexes: map[string]*vschemapb.Vindex{
"v": {
@@ -1612,7 +1612,7 @@ func TestCreateLookupVindexFailures(t *testing.T) {
},
},
},
- err: "non-unique vindex 'from' should have more than one column",
+ err: "",
}, {
description: "vindex not found",
input: &vschemapb.Keyspace{
diff --git a/go/vt/wrangler/resharder.go b/go/vt/wrangler/resharder.go
index 536f4c643cc..b041ce32041 100644
--- a/go/vt/wrangler/resharder.go
+++ b/go/vt/wrangler/resharder.go
@@ -341,7 +341,7 @@ func (rs *resharder) createStreams(ctx context.Context) error {
ig.AddRow(rs.workflow, bls, "", rs.cell, rs.tabletTypes,
binlogdatapb.VReplicationWorkflowType_Reshard,
binlogdatapb.VReplicationWorkflowSubType_None,
- rs.deferSecondaryKeys)
+ rs.deferSecondaryKeys, "")
}
for _, rstream := range rs.refStreams {
@@ -349,7 +349,7 @@ func (rs *resharder) createStreams(ctx context.Context) error {
//todo: fix based on original stream
binlogdatapb.VReplicationWorkflowType_Reshard,
binlogdatapb.VReplicationWorkflowSubType_None,
- rs.deferSecondaryKeys)
+ rs.deferSecondaryKeys, "")
}
query := ig.String()
if _, err := rs.wr.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil {
diff --git a/go/vt/wrangler/testlib/backup_test.go b/go/vt/wrangler/testlib/backup_test.go
index 5e73d266705..b540fc9f8f0 100644
--- a/go/vt/wrangler/testlib/backup_test.go
+++ b/go/vt/wrangler/testlib/backup_test.go
@@ -28,7 +28,6 @@ import (
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/mysql"
- "vitess.io/vitess/go/mysql/capabilities"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/sqltypes"
@@ -36,8 +35,8 @@ import (
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/mysqlctl"
"vitess.io/vitess/go/vt/mysqlctl/backupstorage"
+ "vitess.io/vitess/go/vt/mysqlctl/blackbox"
"vitess.io/vitess/go/vt/mysqlctl/filebackupstorage"
- "vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/memorytopo"
"vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vtenv"
@@ -133,7 +132,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
require.NoError(t, os.MkdirAll(s, os.ModePerm))
}
- needIt, err := needInnoDBRedoLogSubdir()
+ needIt, err := blackbox.NeedInnoDBRedoLogSubdir()
require.NoError(t, err)
if needIt {
newPath := path.Join(sourceInnodbLogDir, mysql.DynamicRedoLogSubdir)
@@ -150,7 +149,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
primary := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_PRIMARY, db)
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -158,7 +157,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
Sequence: 457,
},
},
- }
+ })
// start primary so that replica can fetch primary position from it
primary.StartActionLoop(t, wr)
@@ -170,7 +169,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
sourceTablet.FakeMysqlDaemon.ReadOnly = true
sourceTablet.FakeMysqlDaemon.Replicating = true
sourceTablet.FakeMysqlDaemon.SetReplicationSourceInputs = []string{fmt.Sprintf("%s:%d", primary.Tablet.MysqlHostname, primary.Tablet.MysqlPort)}
- sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ sourceTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -178,7 +177,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
Sequence: 457,
},
},
- }
+ })
sourceTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
"STOP REPLICA",
@@ -221,7 +220,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
destTablet := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
destTablet.FakeMysqlDaemon.ReadOnly = true
destTablet.FakeMysqlDaemon.Replicating = true
- destTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ destTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -229,7 +228,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
Sequence: 457,
},
},
- }
+ })
destTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
"STOP REPLICA",
@@ -248,7 +247,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
destTablet.FakeMysqlDaemon.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {},
}
- destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition
+ destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.GetPrimaryPositionLocked()
destTablet.FakeMysqlDaemon.SetReplicationSourceInputs = append(destTablet.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
destTablet.StartActionLoop(t, wr)
@@ -264,7 +263,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
RelayLogInfoPath: path.Join(root, "relay-log.info"),
}
- err = destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* backupTime */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
+ err = destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* backupTime */, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout)
if err != nil {
return err
}
@@ -301,10 +300,10 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
"START REPLICA",
}
- primary.FakeMysqlDaemon.SetReplicationPositionPos = primary.FakeMysqlDaemon.CurrentPrimaryPosition
+ primary.FakeMysqlDaemon.SetReplicationPositionPos = primary.FakeMysqlDaemon.GetPrimaryPositionLocked()
// restore primary from latest backup
- require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
+ require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout),
"RestoreData failed")
// tablet was created as PRIMARY, so it's baseTabletType is PRIMARY
assert.Equal(t, topodatapb.TabletType_PRIMARY, primary.Tablet.Type)
@@ -320,7 +319,7 @@ func testBackupRestore(t *testing.T, cDetails *compressionDetails) error {
}
// Test restore with the backup timestamp
- require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, backupTime, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout),
+ require.NoError(t, primary.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, backupTime, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout),
"RestoreData with backup timestamp failed")
assert.Equal(t, topodatapb.TabletType_PRIMARY, primary.Tablet.Type)
assert.False(t, primary.FakeMysqlDaemon.Replicating)
@@ -372,7 +371,7 @@ func TestBackupRestoreLagged(t *testing.T) {
}
require.NoError(t, os.WriteFile(path.Join(sourceInnodbDataDir, "innodb_data_1"), []byte("innodb data 1 contents"), os.ModePerm))
- needIt, err := needInnoDBRedoLogSubdir()
+ needIt, err := blackbox.NeedInnoDBRedoLogSubdir()
require.NoError(t, err)
if needIt {
newPath := path.Join(sourceInnodbLogDir, mysql.DynamicRedoLogSubdir)
@@ -388,7 +387,7 @@ func TestBackupRestoreLagged(t *testing.T) {
primary := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_PRIMARY, db)
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -396,7 +395,7 @@ func TestBackupRestoreLagged(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
// start primary so that replica can fetch primary position from it
primary.StartActionLoop(t, wr)
@@ -407,7 +406,7 @@ func TestBackupRestoreLagged(t *testing.T) {
sourceTablet := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, db)
sourceTablet.FakeMysqlDaemon.ReadOnly = true
sourceTablet.FakeMysqlDaemon.Replicating = true
- sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ sourceTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -415,7 +414,7 @@ func TestBackupRestoreLagged(t *testing.T) {
Sequence: 456,
},
},
- }
+ })
sourceTablet.FakeMysqlDaemon.SetReplicationSourceInputs = []string{fmt.Sprintf("%s:%d", primary.Tablet.MysqlHostname, primary.Tablet.MysqlPort)}
sourceTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
@@ -449,7 +448,7 @@ func TestBackupRestoreLagged(t *testing.T) {
timer := time.NewTicker(1 * time.Second)
<-timer.C
- sourceTablet.FakeMysqlDaemon.CurrentPrimaryPositionLocked(replication.Position{
+ sourceTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -468,7 +467,7 @@ func TestBackupRestoreLagged(t *testing.T) {
require.NoError(t, sourceTablet.FakeMysqlDaemon.CheckSuperQueryList())
assert.True(t, sourceTablet.FakeMysqlDaemon.Replicating)
assert.True(t, sourceTablet.FakeMysqlDaemon.Running)
- assert.Equal(t, primary.FakeMysqlDaemon.CurrentPrimaryPosition, sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition)
+ assert.Equal(t, primary.FakeMysqlDaemon.GetPrimaryPositionLocked(), sourceTablet.FakeMysqlDaemon.GetPrimaryPositionLocked())
case <-timer2.C:
require.FailNow(t, "Backup timed out")
}
@@ -477,7 +476,7 @@ func TestBackupRestoreLagged(t *testing.T) {
destTablet := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
destTablet.FakeMysqlDaemon.ReadOnly = true
destTablet.FakeMysqlDaemon.Replicating = true
- destTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ destTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -485,7 +484,7 @@ func TestBackupRestoreLagged(t *testing.T) {
Sequence: 456,
},
},
- }
+ })
destTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
"STOP REPLICA",
@@ -504,7 +503,7 @@ func TestBackupRestoreLagged(t *testing.T) {
destTablet.FakeMysqlDaemon.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {},
}
- destTablet.FakeMysqlDaemon.SetReplicationPositionPos = destTablet.FakeMysqlDaemon.CurrentPrimaryPosition
+ destTablet.FakeMysqlDaemon.SetReplicationPositionPos = destTablet.FakeMysqlDaemon.GetPrimaryPositionLocked()
destTablet.FakeMysqlDaemon.SetReplicationSourceInputs = append(destTablet.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
destTablet.StartActionLoop(t, wr)
@@ -522,12 +521,12 @@ func TestBackupRestoreLagged(t *testing.T) {
errCh = make(chan error, 1)
go func(ctx context.Context, tablet *FakeTablet) {
- errCh <- tablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout)
+ errCh <- tablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout)
}(ctx, destTablet)
timer = time.NewTicker(1 * time.Second)
<-timer.C
- destTablet.FakeMysqlDaemon.CurrentPrimaryPositionLocked(replication.Position{
+ destTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -545,7 +544,7 @@ func TestBackupRestoreLagged(t *testing.T) {
require.NoError(t, destTablet.FakeMysqlDaemon.CheckSuperQueryList(), "destTablet.FakeMysqlDaemon.CheckSuperQueryList failed")
assert.True(t, destTablet.FakeMysqlDaemon.Replicating)
assert.True(t, destTablet.FakeMysqlDaemon.Running)
- assert.Equal(t, primary.FakeMysqlDaemon.CurrentPrimaryPosition, destTablet.FakeMysqlDaemon.CurrentPrimaryPosition)
+ assert.Equal(t, primary.FakeMysqlDaemon.GetPrimaryPositionLocked(), destTablet.FakeMysqlDaemon.GetPrimaryPositionLocked())
case <-timer2.C:
require.FailNow(t, "Restore timed out")
}
@@ -592,7 +591,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
}
require.NoError(t, os.WriteFile(path.Join(sourceInnodbDataDir, "innodb_data_1"), []byte("innodb data 1 contents"), os.ModePerm))
- needIt, err := needInnoDBRedoLogSubdir()
+ needIt, err := blackbox.NeedInnoDBRedoLogSubdir()
require.NoError(t, err)
if needIt {
newPath := path.Join(sourceInnodbLogDir, mysql.DynamicRedoLogSubdir)
@@ -608,7 +607,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
primary := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_PRIMARY, db)
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -616,7 +615,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
// start primary so that replica can fetch primary position from it
primary.StartActionLoop(t, wr)
@@ -626,7 +625,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
sourceTablet := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, db)
sourceTablet.FakeMysqlDaemon.ReadOnly = true
sourceTablet.FakeMysqlDaemon.Replicating = true
- sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ sourceTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -634,7 +633,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
sourceTablet.FakeMysqlDaemon.SetReplicationSourceInputs = []string{fmt.Sprintf("%s:%d", primary.Tablet.MysqlHostname, primary.Tablet.MysqlPort)}
sourceTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
@@ -668,7 +667,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
destTablet := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
destTablet.FakeMysqlDaemon.ReadOnly = true
destTablet.FakeMysqlDaemon.Replicating = true
- destTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ destTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -676,7 +675,7 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
destTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
"STOP REPLICA",
@@ -688,14 +687,11 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
"FAKE RESET REPLICA ALL",
"FAKE RESET BINARY LOGS AND GTIDS",
"FAKE SET GLOBAL gtid_purged",
- "STOP REPLICA",
- "FAKE SET SOURCE",
- "START REPLICA",
}
destTablet.FakeMysqlDaemon.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {},
}
- destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition
+ destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.GetPrimaryPositionLocked()
destTablet.FakeMysqlDaemon.SetReplicationSourceInputs = append(destTablet.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
destTablet.StartActionLoop(t, wr)
@@ -714,13 +710,16 @@ func TestRestoreUnreachablePrimary(t *testing.T) {
// stop primary so that it is unreachable
primary.StopActionLoop(t)
- // set a short timeout so that we don't have to wait 30 seconds
- topo.RemoteOperationTimeout = 2 * time.Second
- // Restore should still succeed
- require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
+ // Attempt to fix the test, but its still failing :man_shrugging.
+ ctx, cancel = context.WithTimeout(ctx, 2*time.Second)
+ defer cancel()
+ // Restore will return an error while trying to contact the primary for its position, but otherwise will succeed.
+ // The replication won't be running however, since we can't run errant GTID detection without the primary being online.
+ err = destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout)
+ require.ErrorContains(t, err, "DeadlineExceeded")
// verify the full status
require.NoError(t, destTablet.FakeMysqlDaemon.CheckSuperQueryList(), "destTablet.FakeMysqlDaemon.CheckSuperQueryList failed")
- assert.True(t, destTablet.FakeMysqlDaemon.Replicating)
+ assert.False(t, destTablet.FakeMysqlDaemon.Replicating)
assert.True(t, destTablet.FakeMysqlDaemon.Running)
}
@@ -768,7 +767,7 @@ func TestDisableActiveReparents(t *testing.T) {
}
require.NoError(t, os.WriteFile(path.Join(sourceInnodbDataDir, "innodb_data_1"), []byte("innodb data 1 contents"), os.ModePerm))
- needIt, err := needInnoDBRedoLogSubdir()
+ needIt, err := blackbox.NeedInnoDBRedoLogSubdir()
require.NoError(t, err)
if needIt {
newPath := path.Join(sourceInnodbLogDir, mysql.DynamicRedoLogSubdir)
@@ -784,7 +783,7 @@ func TestDisableActiveReparents(t *testing.T) {
primary := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_PRIMARY, db)
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -792,7 +791,7 @@ func TestDisableActiveReparents(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
// start primary so that replica can fetch primary position from it
primary.StartActionLoop(t, wr)
@@ -803,7 +802,7 @@ func TestDisableActiveReparents(t *testing.T) {
sourceTablet := NewFakeTablet(t, wr, "cell1", 1, topodatapb.TabletType_REPLICA, db)
sourceTablet.FakeMysqlDaemon.ReadOnly = true
sourceTablet.FakeMysqlDaemon.Replicating = true
- sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ sourceTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -811,7 +810,7 @@ func TestDisableActiveReparents(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
sourceTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"STOP REPLICA",
}
@@ -836,7 +835,7 @@ func TestDisableActiveReparents(t *testing.T) {
destTablet := NewFakeTablet(t, wr, "cell1", 2, topodatapb.TabletType_REPLICA, db)
destTablet.FakeMysqlDaemon.ReadOnly = true
destTablet.FakeMysqlDaemon.Replicating = true
- destTablet.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ destTablet.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -844,7 +843,7 @@ func TestDisableActiveReparents(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
destTablet.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE RESET BINARY LOGS AND GTIDS",
"FAKE SET GLOBAL gtid_purged",
@@ -856,7 +855,7 @@ func TestDisableActiveReparents(t *testing.T) {
destTablet.FakeMysqlDaemon.FetchSuperQueryMap = map[string]*sqltypes.Result{
"SHOW DATABASES": {},
}
- destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.CurrentPrimaryPosition
+ destTablet.FakeMysqlDaemon.SetReplicationPositionPos = sourceTablet.FakeMysqlDaemon.GetPrimaryPositionLocked()
destTablet.FakeMysqlDaemon.SetReplicationSourceInputs = append(destTablet.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
destTablet.StartActionLoop(t, wr)
@@ -872,31 +871,9 @@ func TestDisableActiveReparents(t *testing.T) {
RelayLogInfoPath: path.Join(root, "relay-log.info"),
}
- require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "", mysqlShutdownTimeout))
+ require.NoError(t, destTablet.TM.RestoreData(ctx, logutil.NewConsoleLogger(), 0 /* waitForBackupInterval */, false /* deleteBeforeRestore */, time.Time{} /* restoreFromBackupTs */, time.Time{} /* restoreToTimestamp */, "" /* restoreToPos */, []string{} /* ignoreBackupEngines */, mysqlShutdownTimeout))
// verify the full status
require.NoError(t, destTablet.FakeMysqlDaemon.CheckSuperQueryList(), "destTablet.FakeMysqlDaemon.CheckSuperQueryList failed")
assert.False(t, destTablet.FakeMysqlDaemon.Replicating)
assert.True(t, destTablet.FakeMysqlDaemon.Running)
}
-
-// needInnoDBRedoLogSubdir indicates whether we need to create a redo log subdirectory.
-// Starting with MySQL 8.0.30, the InnoDB redo logs are stored in a subdirectory of the
-// (/. by default) called "#innodb_redo". See:
-//
-// https://dev.mysql.com/doc/refman/8.0/en/innodb-redo-log.html#innodb-modifying-redo-log-capacity
-func needInnoDBRedoLogSubdir() (needIt bool, err error) {
- mysqldVersionStr, err := mysqlctl.GetVersionString()
- if err != nil {
- return needIt, err
- }
- _, sv, err := mysqlctl.ParseVersionString(mysqldVersionStr)
- if err != nil {
- return needIt, err
- }
- versionStr := fmt.Sprintf("%d.%d.%d", sv.Major, sv.Minor, sv.Patch)
- capableOf := mysql.ServerVersionCapableOf(versionStr)
- if capableOf == nil {
- return needIt, fmt.Errorf("cannot determine database flavor details for version %s", versionStr)
- }
- return capableOf(capabilities.DynamicRedoLogCapacityFlavorCapability)
-}
diff --git a/go/vt/wrangler/testlib/emergency_reparent_shard_test.go b/go/vt/wrangler/testlib/emergency_reparent_shard_test.go
index 96f9df74405..3167be5e512 100644
--- a/go/vt/wrangler/testlib/emergency_reparent_shard_test.go
+++ b/go/vt/wrangler/testlib/emergency_reparent_shard_test.go
@@ -63,7 +63,7 @@ func TestEmergencyReparentShard(t *testing.T) {
reparenttestutil.SetKeyspaceDurability(context.Background(), t, ts, "test_keyspace", "semi_sync")
oldPrimary.FakeMysqlDaemon.Replicating = false
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -71,7 +71,7 @@ func TestEmergencyReparentShard(t *testing.T) {
Sequence: 456,
},
},
- }
+ })
currentPrimaryFilePosition, _ := replication.ParseFilePosGTIDSet("mariadb-bin.000010:456")
oldPrimary.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: currentPrimaryFilePosition,
@@ -80,7 +80,7 @@ func TestEmergencyReparentShard(t *testing.T) {
// new primary
newPrimary.FakeMysqlDaemon.ReadOnly = true
newPrimary.FakeMysqlDaemon.Replicating = true
- newPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ newPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -88,7 +88,7 @@ func TestEmergencyReparentShard(t *testing.T) {
Sequence: 456,
},
},
- }
+ })
newPrimaryRelayLogPos, _ := replication.ParseFilePosGTIDSet("relay-bin.000004:456")
newPrimary.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: newPrimaryRelayLogPos,
@@ -123,7 +123,7 @@ func TestEmergencyReparentShard(t *testing.T) {
// good replica 1 is replicating
goodReplica1.FakeMysqlDaemon.ReadOnly = true
goodReplica1.FakeMysqlDaemon.Replicating = true
- goodReplica1.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ goodReplica1.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -131,7 +131,7 @@ func TestEmergencyReparentShard(t *testing.T) {
Sequence: 455,
},
},
- }
+ })
goodReplica1RelayLogPos, _ := replication.ParseFilePosGTIDSet("relay-bin.000004:455")
goodReplica1.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: goodReplica1RelayLogPos,
@@ -154,7 +154,7 @@ func TestEmergencyReparentShard(t *testing.T) {
// good replica 2 is not replicating
goodReplica2.FakeMysqlDaemon.ReadOnly = true
goodReplica2.FakeMysqlDaemon.Replicating = false
- goodReplica2.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ goodReplica2.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -162,7 +162,7 @@ func TestEmergencyReparentShard(t *testing.T) {
Sequence: 454,
},
},
- }
+ })
goodReplica2RelayLogPos, _ := replication.ParseFilePosGTIDSet("relay-bin.000004:454")
goodReplica2.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: goodReplica2RelayLogPos,
@@ -217,7 +217,7 @@ func TestEmergencyReparentShardPrimaryElectNotBest(t *testing.T) {
newPrimary.FakeMysqlDaemon.Replicating = true
// It has transactions in its relay log, but not as many as
// moreAdvancedReplica
- newPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ newPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -225,7 +225,7 @@ func TestEmergencyReparentShardPrimaryElectNotBest(t *testing.T) {
Sequence: 456,
},
},
- }
+ })
newPrimaryRelayLogPos, _ := replication.ParseFilePosGTIDSet("relay-bin.000004:456")
newPrimary.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: newPrimaryRelayLogPos,
@@ -250,7 +250,7 @@ func TestEmergencyReparentShardPrimaryElectNotBest(t *testing.T) {
// more advanced replica
moreAdvancedReplica.FakeMysqlDaemon.Replicating = true
// relay log position is more advanced than desired new primary
- moreAdvancedReplica.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ moreAdvancedReplica.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
2: replication.MariadbGTID{
Domain: 2,
@@ -258,14 +258,14 @@ func TestEmergencyReparentShardPrimaryElectNotBest(t *testing.T) {
Sequence: 457,
},
},
- }
+ })
moreAdvancedReplicaLogPos, _ := replication.ParseFilePosGTIDSet("relay-bin.000004:457")
moreAdvancedReplica.FakeMysqlDaemon.CurrentSourceFilePosition = replication.Position{
GTIDSet: moreAdvancedReplicaLogPos,
}
moreAdvancedReplica.FakeMysqlDaemon.SetReplicationSourceInputs = append(moreAdvancedReplica.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet), topoproto.MysqlAddr(oldPrimary.Tablet))
moreAdvancedReplica.FakeMysqlDaemon.WaitPrimaryPositions = append(moreAdvancedReplica.FakeMysqlDaemon.WaitPrimaryPositions, moreAdvancedReplica.FakeMysqlDaemon.CurrentSourceFilePosition)
- newPrimary.FakeMysqlDaemon.WaitPrimaryPositions = append(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions, moreAdvancedReplica.FakeMysqlDaemon.CurrentPrimaryPosition)
+ newPrimary.FakeMysqlDaemon.WaitPrimaryPositions = append(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions, moreAdvancedReplica.FakeMysqlDaemon.GetPrimaryPositionLocked())
moreAdvancedReplica.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
// These 3 statements come from tablet startup
"STOP REPLICA",
diff --git a/go/vt/wrangler/testlib/planned_reparent_shard_test.go b/go/vt/wrangler/testlib/planned_reparent_shard_test.go
index 28ffd34b756..1894c6bb4eb 100644
--- a/go/vt/wrangler/testlib/planned_reparent_shard_test.go
+++ b/go/vt/wrangler/testlib/planned_reparent_shard_test.go
@@ -96,7 +96,7 @@ func TestPlannedReparentShardNoPrimaryProvided(t *testing.T) {
oldPrimary.FakeMysqlDaemon.ReadOnly = false
oldPrimary.FakeMysqlDaemon.Replicating = false
oldPrimary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0]
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0])
oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs = append(oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet))
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE SET SOURCE",
@@ -213,7 +213,7 @@ func TestPlannedReparentShardNoError(t *testing.T) {
oldPrimary.FakeMysqlDaemon.ReadOnly = false
oldPrimary.FakeMysqlDaemon.Replicating = false
oldPrimary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0]
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0])
oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs = append(oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet))
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE SET SOURCE",
@@ -434,7 +434,7 @@ func TestPlannedReparentShardWaitForPositionFail(t *testing.T) {
oldPrimary.FakeMysqlDaemon.ReadOnly = false
oldPrimary.FakeMysqlDaemon.Replicating = false
// set to incorrect value to make promote fail on WaitForReplicationPos
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.PromoteResult
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.PromoteResult)
oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs = append(oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet))
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE SET SOURCE",
@@ -542,7 +542,7 @@ func TestPlannedReparentShardWaitForPositionTimeout(t *testing.T) {
// old primary
oldPrimary.FakeMysqlDaemon.ReadOnly = false
oldPrimary.FakeMysqlDaemon.Replicating = false
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0]
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0])
oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs = append(oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet))
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE SET SOURCE",
@@ -616,7 +616,7 @@ func TestPlannedReparentShardRelayLogError(t *testing.T) {
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
primary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
7: replication.MariadbGTID{
Domain: 7,
@@ -624,7 +624,7 @@ func TestPlannedReparentShardRelayLogError(t *testing.T) {
Sequence: 990,
},
},
- }
+ })
primary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"SUBINSERT INTO _vt.reparent_journal (time_created_ns, action_name, primary_alias, replication_position) VALUES",
}
@@ -697,7 +697,7 @@ func TestPlannedReparentShardRelayLogErrorStartReplication(t *testing.T) {
primary.FakeMysqlDaemon.ReadOnly = false
primary.FakeMysqlDaemon.Replicating = false
primary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
7: replication.MariadbGTID{
Domain: 7,
@@ -705,7 +705,7 @@ func TestPlannedReparentShardRelayLogErrorStartReplication(t *testing.T) {
Sequence: 990,
},
},
- }
+ })
primary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"SUBINSERT INTO _vt.reparent_journal (time_created_ns, action_name, primary_alias, replication_position) VALUES",
}
@@ -815,7 +815,7 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) {
oldPrimary.FakeMysqlDaemon.ReadOnly = false
oldPrimary.FakeMysqlDaemon.Replicating = false
oldPrimary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0]
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0])
oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs = append(oldPrimary.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(newPrimary.Tablet))
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"FAKE SET SOURCE",
@@ -823,7 +823,7 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) {
// We call a SetReplicationSource explicitly
"FAKE SET SOURCE",
"START REPLICA",
- // extra SetReplicationSource call due to retry
+ // extra SetReplicationSource call due to retry)
"FAKE SET SOURCE",
"START REPLICA",
}
@@ -885,7 +885,7 @@ func TestPlannedReparentShardPromoteReplicaFail(t *testing.T) {
// retrying should work
newPrimary.FakeMysqlDaemon.PromoteError = nil
- newPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0]
+ newPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(newPrimary.FakeMysqlDaemon.WaitPrimaryPositions[0])
// run PlannedReparentShard
err = vp.Run([]string{"PlannedReparentShard", "--wait_replicas_timeout", "10s", "--keyspace_shard", newPrimary.Tablet.Keyspace + "/" + newPrimary.Tablet.Shard, "--new_primary", topoproto.TabletAliasString(newPrimary.Tablet.Alias)})
@@ -922,7 +922,7 @@ func TestPlannedReparentShardSamePrimary(t *testing.T) {
oldPrimary.FakeMysqlDaemon.ReadOnly = true
oldPrimary.FakeMysqlDaemon.Replicating = false
oldPrimary.FakeMysqlDaemon.ReplicationStatusError = mysql.ErrNotReplica
- oldPrimary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ oldPrimary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
7: replication.MariadbGTID{
Domain: 7,
@@ -930,7 +930,7 @@ func TestPlannedReparentShardSamePrimary(t *testing.T) {
Sequence: 990,
},
},
- }
+ })
oldPrimary.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
"SUBINSERT INTO _vt.reparent_journal (time_created_ns, action_name, primary_alias, replication_position) VALUES",
}
diff --git a/go/vt/wrangler/testlib/reparent_utils_test.go b/go/vt/wrangler/testlib/reparent_utils_test.go
index e0a2077c778..b199a64340a 100644
--- a/go/vt/wrangler/testlib/reparent_utils_test.go
+++ b/go/vt/wrangler/testlib/reparent_utils_test.go
@@ -67,7 +67,7 @@ func TestShardReplicationStatuses(t *testing.T) {
}
// primary action loop (to initialize host and port)
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
5: replication.MariadbGTID{
Domain: 5,
@@ -75,12 +75,12 @@ func TestShardReplicationStatuses(t *testing.T) {
Sequence: 892,
},
},
- }
+ })
primary.StartActionLoop(t, wr)
defer primary.StopActionLoop(t)
// replica loop
- replica.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ replica.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
5: replication.MariadbGTID{
Domain: 5,
@@ -88,7 +88,7 @@ func TestShardReplicationStatuses(t *testing.T) {
Sequence: 890,
},
},
- }
+ })
replica.FakeMysqlDaemon.CurrentSourceHost = primary.Tablet.MysqlHostname
replica.FakeMysqlDaemon.CurrentSourcePort = primary.Tablet.MysqlPort
replica.FakeMysqlDaemon.SetReplicationSourceInputs = append(replica.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
@@ -205,6 +205,10 @@ func TestSetReplicationSource(t *testing.T) {
return nil
})
require.NoError(t, err, "UpdateShardFields failed")
+ pos, err := replication.DecodePosition("MySQL56/8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-8")
+ require.NoError(t, err)
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(pos)
+ primary.FakeMysqlDaemon.ServerUUID = "8bc65c84-3fe4-11ed-a912-257f0fcdd6c9"
// primary action loop (to initialize host and port)
primary.StartActionLoop(t, wr)
@@ -246,6 +250,36 @@ func TestSetReplicationSource(t *testing.T) {
checkSemiSyncEnabled(t, false, true, replica)
})
+ t.Run("Errant GTIDs on the replica", func(t *testing.T) {
+ replica := NewFakeTablet(t, wr, "cell1", 4, topodatapb.TabletType_REPLICA, nil)
+ // replica loop
+ replica.FakeMysqlDaemon.Replicating = true
+ replica.FakeMysqlDaemon.IOThreadRunning = true
+ replica.FakeMysqlDaemon.SetReplicationSourceInputs = append(replica.FakeMysqlDaemon.SetReplicationSourceInputs, topoproto.MysqlAddr(primary.Tablet))
+ replica.FakeMysqlDaemon.ExpectedExecuteSuperQueryList = []string{
+ // These 3 statements come from tablet startup
+ "STOP REPLICA",
+ "FAKE SET SOURCE",
+ "START REPLICA",
+ }
+ replica.StartActionLoop(t, wr)
+ defer replica.StopActionLoop(t)
+
+ // Set replica's GTID to have a write that the primary's GTID doesn't have
+ pos, err = replication.DecodePosition("MySQL56/8bc65c84-3fe4-11ed-a912-257f0fcdd6c9:1-7,8bc65cca-3fe4-11ed-bbfb-091034d48b3e:1")
+ require.NoError(t, err)
+ replica.FakeMysqlDaemon.CurrentRelayLogPosition = pos
+
+ // run SetReplicationSource
+ err = wr.SetReplicationSource(ctx, replica.Tablet)
+ require.ErrorContains(t, err, "Errant GTID detected")
+
+ // check what was run
+ err = replica.FakeMysqlDaemon.CheckSuperQueryList()
+ require.NoError(t, err, "CheckSuperQueryList failed")
+ checkSemiSyncEnabled(t, false, true, replica)
+ })
+
// test setting an empty hostname because of primary shutdown
t.Run("Primary tablet already shutdown", func(t *testing.T) {
replica := NewFakeTablet(t, wr, "cell1", 3, topodatapb.TabletType_REPLICA, nil)
diff --git a/go/vt/wrangler/testlib/version_test.go b/go/vt/wrangler/testlib/version_test.go
index cf5f3fd1487..552f7f29c04 100644
--- a/go/vt/wrangler/testlib/version_test.go
+++ b/go/vt/wrangler/testlib/version_test.go
@@ -56,7 +56,7 @@ func expvarHandler(gitRev *string) func(http.ResponseWriter, *http.Request) {
http.Error(w, fmt.Sprintf("cannot marshal json: %s", err), http.StatusInternalServerError)
return
}
- fmt.Fprintf(w, string(result)+"\n")
+ fmt.Fprint(w, string(result)+"\n")
}
}
diff --git a/go/vt/wrangler/traffic_switcher.go b/go/vt/wrangler/traffic_switcher.go
index 448f4f99734..d337c1ee515 100644
--- a/go/vt/wrangler/traffic_switcher.go
+++ b/go/vt/wrangler/traffic_switcher.go
@@ -278,12 +278,12 @@ func (wr *Wrangler) getWorkflowState(ctx context.Context, targetKeyspace, workfl
}
}
} else {
- state.RdonlyCellsSwitched, state.RdonlyCellsNotSwitched, err = ws.GetCellsWithTableReadsSwitched(ctx, targetKeyspace, table, topodatapb.TabletType_RDONLY)
+ state.RdonlyCellsSwitched, state.RdonlyCellsNotSwitched, err = ws.GetCellsWithTableReadsSwitched(ctx, sourceKeyspace, targetKeyspace, table, topodatapb.TabletType_RDONLY)
if err != nil {
return nil, nil, err
}
- state.ReplicaCellsSwitched, state.ReplicaCellsNotSwitched, err = ws.GetCellsWithTableReadsSwitched(ctx, targetKeyspace, table, topodatapb.TabletType_REPLICA)
+ state.ReplicaCellsSwitched, state.ReplicaCellsNotSwitched, err = ws.GetCellsWithTableReadsSwitched(ctx, sourceKeyspace, targetKeyspace, table, topodatapb.TabletType_REPLICA)
if err != nil {
return nil, nil, err
}
@@ -335,7 +335,7 @@ func (wr *Wrangler) SwitchReads(ctx context.Context, targetKeyspace, workflowNam
cells []string, direction workflow.TrafficSwitchDirection, dryRun bool) (*[]string, error) {
// Consistently handle errors by logging and returning them.
handleError := func(message string, err error) (*[]string, error) {
- werr := vterrors.Errorf(vtrpcpb.Code_INTERNAL, fmt.Sprintf("%s: %v", message, err))
+ werr := vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s: %v", message, err)
wr.Logger().Error(werr)
return nil, werr
}
@@ -346,7 +346,7 @@ func (wr *Wrangler) SwitchReads(ctx context.Context, targetKeyspace, workflowNam
}
if ts == nil {
errorMsg := fmt.Sprintf("workflow %s not found in keyspace %s", workflowName, targetKeyspace)
- return handleError("failed to get the current state of the workflow", fmt.Errorf(errorMsg))
+ return handleError("failed to get the current state of the workflow", errors.New(errorMsg))
}
log.Infof("Switching reads: %s.%s tt %+v, cells %+v, workflow state: %+v", targetKeyspace, workflowName, servedTypes, cells, ws)
var switchReplicas, switchRdonly bool
@@ -478,7 +478,7 @@ func (wr *Wrangler) SwitchWrites(ctx context.Context, targetKeyspace, workflowNa
cancel, reverse, reverseReplication bool, dryRun, initializeTargetSequences bool) (journalID int64, dryRunResults *[]string, err error) {
// Consistently handle errors by logging and returning them.
handleError := func(message string, err error) (int64, *[]string, error) {
- werr := vterrors.Errorf(vtrpcpb.Code_INTERNAL, fmt.Sprintf("%s: %v", message, err))
+ werr := vterrors.Errorf(vtrpcpb.Code_INTERNAL, "%s: %v", message, err)
wr.Logger().Error(werr)
return 0, nil, werr
}
@@ -490,7 +490,7 @@ func (wr *Wrangler) SwitchWrites(ctx context.Context, targetKeyspace, workflowNa
}
if ts == nil {
errorMsg := fmt.Sprintf("workflow %s not found in keyspace %s", workflowName, targetKeyspace)
- return handleError("failed to get the current workflow state", fmt.Errorf(errorMsg))
+ return handleError("failed to get the current workflow state", errors.New(errorMsg))
}
var sw iswitcher
@@ -1723,7 +1723,7 @@ func doValidateWorkflowHasCompleted(ctx context.Context, ts *trafficSwitcher) er
_ = ts.ForAllSources(func(source *workflow.MigrationSource) error {
wg.Add(1)
if source.GetShard().IsPrimaryServing {
- rec.RecordError(fmt.Errorf(fmt.Sprintf("Shard %s is still serving", source.GetShard().ShardName())))
+ rec.RecordError(fmt.Errorf("Shard %s is still serving", source.GetShard().ShardName()))
}
wg.Done()
return nil
diff --git a/go/vt/wrangler/traffic_switcher_env_test.go b/go/vt/wrangler/traffic_switcher_env_test.go
index 4e58024785d..a99e6ba2c43 100644
--- a/go/vt/wrangler/traffic_switcher_env_test.go
+++ b/go/vt/wrangler/traffic_switcher_env_test.go
@@ -776,7 +776,7 @@ func (tme *testMigraterEnv) createDBClients(ctx context.Context, t *testing.T) {
func (tme *testMigraterEnv) setPrimaryPositions() {
for _, primary := range tme.sourcePrimaries {
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
5: replication.MariadbGTID{
Domain: 5,
@@ -784,10 +784,10 @@ func (tme *testMigraterEnv) setPrimaryPositions() {
Sequence: 892,
},
},
- }
+ })
}
for _, primary := range tme.targetPrimaries {
- primary.FakeMysqlDaemon.CurrentPrimaryPosition = replication.Position{
+ primary.FakeMysqlDaemon.SetPrimaryPositionLocked(replication.Position{
GTIDSet: replication.MariadbGTIDSet{
5: replication.MariadbGTID{
Domain: 5,
@@ -795,7 +795,7 @@ func (tme *testMigraterEnv) setPrimaryPositions() {
Sequence: 893,
},
},
- }
+ })
}
}
diff --git a/go/vt/wrangler/vexec.go b/go/vt/wrangler/vexec.go
index 2c279c5c6cf..41f02ef9e63 100644
--- a/go/vt/wrangler/vexec.go
+++ b/go/vt/wrangler/vexec.go
@@ -445,9 +445,9 @@ func (wr *Wrangler) execWorkflowAction(ctx context.Context, workflow, keyspace,
changes = true
dryRunChanges.WriteString(fmt.Sprintf(" tablet_types=%q\n", topoproto.MakeStringTypeCSV(rpcReq.TabletTypes)))
}
- if !textutil.ValueIsSimulatedNull(rpcReq.OnDdl) {
+ if rpcReq.OnDdl != nil {
changes = true
- dryRunChanges.WriteString(fmt.Sprintf(" on_ddl=%q\n", binlogdatapb.OnDDLAction_name[int32(rpcReq.OnDdl)]))
+ dryRunChanges.WriteString(fmt.Sprintf(" on_ddl=%q\n", binlogdatapb.OnDDLAction_name[int32(*rpcReq.OnDdl)]))
}
if !changes {
return nil, fmt.Errorf("no updates were provided; use --cells, --tablet-types, or --on-ddl to specify new values")
diff --git a/go/vt/wrangler/vexec_test.go b/go/vt/wrangler/vexec_test.go
index 80cd2aef565..223e338b303 100644
--- a/go/vt/wrangler/vexec_test.go
+++ b/go/vt/wrangler/vexec_test.go
@@ -29,13 +29,15 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/ptr"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/vt/logutil"
+ "vitess.io/vitess/go/vt/vtenv"
+
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
- "vitess.io/vitess/go/vt/vtenv"
)
var (
@@ -397,49 +399,45 @@ func TestWorkflowUpdate(t *testing.T) {
defer env.close()
logger := logutil.NewMemoryLogger()
wr := New(vtenv.NewTestEnv(), logger, env.topoServ, env.tmc)
- nullSlice := textutil.SimulatedNullStringSlice // Used to represent a non-provided value
- nullOnDDL := binlogdatapb.OnDDLAction(textutil.SimulatedNullInt) // Used to represent a non-provided value
+
tests := []struct {
name string
cells []string
tabletTypes []topodatapb.TabletType
- onDDL binlogdatapb.OnDDLAction
+ onDDL *binlogdatapb.OnDDLAction
output string
wantErr string
}{
{
name: "no flags",
- cells: nullSlice,
- tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)},
- onDDL: nullOnDDL,
+ cells: textutil.SimulatedNullStringSlice,
+ tabletTypes: textutil.SimulatedNullTabletTypeSlice,
wantErr: "no updates were provided; use --cells, --tablet-types, or --on-ddl to specify new values",
},
{
name: "only cells",
cells: []string{"zone1"},
- tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)},
- onDDL: nullOnDDL,
+ tabletTypes: textutil.SimulatedNullTabletTypeSlice,
output: "The following workflow fields will be updated:\n cells=\"zone1\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n",
},
{
name: "only tablet types",
- cells: nullSlice,
+ cells: textutil.SimulatedNullStringSlice,
tabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY, topodatapb.TabletType_REPLICA},
- onDDL: nullOnDDL,
output: "The following workflow fields will be updated:\n tablet_types=\"primary,replica\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n",
},
{
name: "only on-ddl",
- cells: nullSlice,
- tabletTypes: []topodatapb.TabletType{topodatapb.TabletType(textutil.SimulatedNullInt)},
- onDDL: binlogdatapb.OnDDLAction_EXEC_IGNORE,
+ cells: textutil.SimulatedNullStringSlice,
+ tabletTypes: textutil.SimulatedNullTabletTypeSlice,
+ onDDL: ptr.Of(binlogdatapb.OnDDLAction_EXEC_IGNORE),
output: "The following workflow fields will be updated:\n on_ddl=\"EXEC_IGNORE\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n",
},
{
name: "all flags",
cells: []string{"zone1", "zone2"},
tabletTypes: []topodatapb.TabletType{topodatapb.TabletType_RDONLY, topodatapb.TabletType_SPARE},
- onDDL: binlogdatapb.OnDDLAction_EXEC,
+ onDDL: ptr.Of(binlogdatapb.OnDDLAction_EXEC),
output: "The following workflow fields will be updated:\n cells=\"zone1,zone2\"\n tablet_types=\"rdonly,spare\"\n on_ddl=\"EXEC\"\nOn the following tablets in the target keyspace for workflow wrWorkflow:\n zone1-0000000200 (target/-80)\n zone1-0000000210 (target/80-)\n",
},
}
diff --git a/go/vt/wrangler/workflow.go b/go/vt/wrangler/workflow.go
index 6862f5f4d3f..6e74e4c9ebd 100644
--- a/go/vt/wrangler/workflow.go
+++ b/go/vt/wrangler/workflow.go
@@ -2,6 +2,7 @@ package wrangler
import (
"context"
+ "errors"
"fmt"
"sort"
"strings"
@@ -114,7 +115,7 @@ func (wr *Wrangler) NewVReplicationWorkflow(ctx context.Context, workflowType VR
return nil, err
}
log.Infof("Workflow state is %+v", ws)
- if ts != nil { //Other than on create we need to get SourceKeyspace from the workflow
+ if ts != nil { // Other than on create we need to get SourceKeyspace from the workflow
vrw.params.TargetKeyspace = ts.targetKeyspace
vrw.params.Workflow = ts.workflow
vrw.params.SourceKeyspace = ts.sourceKeyspace
@@ -379,7 +380,7 @@ func (vrw *VReplicationWorkflow) Complete() (*[]string, error) {
}
if !ws.WritesSwitched || len(ws.ReplicaCellsNotSwitched) > 0 || len(ws.RdonlyCellsNotSwitched) > 0 {
- return nil, fmt.Errorf(ErrWorkflowNotFullySwitched)
+ return nil, errors.New(ErrWorkflowNotFullySwitched)
}
var renameTable workflow.TableRemovalType
if vrw.params.RenameTables {
@@ -404,7 +405,7 @@ func (vrw *VReplicationWorkflow) Cancel() error {
}
if ws.WritesSwitched || len(ws.ReplicaCellsSwitched) > 0 || len(ws.RdonlyCellsSwitched) > 0 {
- return fmt.Errorf(ErrWorkflowPartiallySwitched)
+ return errors.New(ErrWorkflowPartiallySwitched)
}
if _, err := vrw.wr.DropTargets(vrw.ctx, vrw.ws.TargetKeyspace, vrw.ws.Workflow, vrw.params.KeepData, vrw.params.KeepRoutingRules, false); err != nil {
return err
diff --git a/go/vt/wrangler/workflow_test.go b/go/vt/wrangler/workflow_test.go
index 4f508766330..92996c9d931 100644
--- a/go/vt/wrangler/workflow_test.go
+++ b/go/vt/wrangler/workflow_test.go
@@ -325,6 +325,18 @@ func TestPartialMoveTables(t *testing.T) {
tme := newTestTablePartialMigrater(ctx, t, shards, shards[0:1], "select * %s")
defer tme.stopTablets(t)
+ // Add the schema for the primary tablets, so that we don't fail while applying the denied table rules.
+ schm := &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{
+ Name: "t1",
+ }, {
+ Name: "t2",
+ }},
+ }
+ for _, primary := range append(tme.sourcePrimaries, tme.targetPrimaries...) {
+ primary.FakeMysqlDaemon.Schema = schm
+ }
+
// Save some unrelated shard routing rules to be sure that
// they don't interfere in any way.
srr, err := tme.ts.GetShardRoutingRules(ctx)
@@ -400,6 +412,17 @@ func TestPartialMoveTablesShardSubset(t *testing.T) {
}
tme := newTestTablePartialMigrater(ctx, t, shards, shardsToMove, "select * %s")
defer tme.stopTablets(t)
+ // Add the schema for the primary tablets, so that we don't fail while applying the denied table rules.
+ schm := &tabletmanagerdatapb.SchemaDefinition{
+ TableDefinitions: []*tabletmanagerdatapb.TableDefinition{{
+ Name: "t1",
+ }, {
+ Name: "t2",
+ }},
+ }
+ for _, primary := range append(tme.sourcePrimaries, tme.targetPrimaries...) {
+ primary.FakeMysqlDaemon.Schema = schm
+ }
// Save some unrelated shard routing rules to be sure that
// they don't interfere in any way.
diff --git a/java/client/pom.xml b/java/client/pom.xml
index fc78d17b564..1cb1ecb5862 100644
--- a/java/client/pom.xml
+++ b/java/client/pom.xml
@@ -5,7 +5,7 @@
io.vitess
vitess-parent
- 21.0.0-SNAPSHOT
+ 22.0.0-SNAPSHOT
vitess-client
diff --git a/java/client/src/main/java/io/vitess/client/VTGateBlockingConnection.java b/java/client/src/main/java/io/vitess/client/VTGateBlockingConnection.java
index 4c24c12bfcf..8a19659ef65 100644
--- a/java/client/src/main/java/io/vitess/client/VTGateBlockingConnection.java
+++ b/java/client/src/main/java/io/vitess/client/VTGateBlockingConnection.java
@@ -111,9 +111,6 @@ public Cursor streamExecute(Context ctx,
return vtGateConnection.streamExecute(ctx, query, bindVars, vtSession);
}
- /**
- * @inheritDoc
- */
@Override
public void close() throws IOException {
vtGateConnection.close();
diff --git a/java/client/src/main/java/io/vitess/client/VTGateConnection.java b/java/client/src/main/java/io/vitess/client/VTGateConnection.java
index ee2fb270a9a..683f4bf5f15 100644
--- a/java/client/src/main/java/io/vitess/client/VTGateConnection.java
+++ b/java/client/src/main/java/io/vitess/client/VTGateConnection.java
@@ -205,9 +205,6 @@ StreamIterator getVStream(Context ctx, VStreamRequest vstreamRe
return client.getVStream(ctx, request);
}
- /**
- * @inheritDoc
- */
@Override
public void close() throws IOException {
client.close();
diff --git a/java/client/src/main/java/io/vitess/client/VTSession.java b/java/client/src/main/java/io/vitess/client/VTSession.java
index 9974e5f034a..3b52b4d3c2e 100644
--- a/java/client/src/main/java/io/vitess/client/VTSession.java
+++ b/java/client/src/main/java/io/vitess/client/VTSession.java
@@ -103,7 +103,7 @@ public Query.ExecuteOptions.TransactionIsolation getTransactionIsolation() {
/**
* Sets this session's transaction isolation level.
*
- * @param Transaction Isolation Level of the Session
+ * @param isolation Isolation Level of the Session
*/
public void setTransactionIsolation(Query.ExecuteOptions.TransactionIsolation isolation) {
this.session = this.session.toBuilder()
diff --git a/java/client/src/main/java/io/vitess/client/grpc/tls/TlsOptions.java b/java/client/src/main/java/io/vitess/client/grpc/tls/TlsOptions.java
index ffd570d413b..4defac70378 100644
--- a/java/client/src/main/java/io/vitess/client/grpc/tls/TlsOptions.java
+++ b/java/client/src/main/java/io/vitess/client/grpc/tls/TlsOptions.java
@@ -17,11 +17,10 @@
package io.vitess.client.grpc.tls;
import java.io.File;
-import java.net.InetSocketAddress;
/**
* A wrapper type holding TLS-related fields for the
- * {@link io.vitess.client.RpcClientFactory#createTls(InetSocketAddress, TlsOptions)} method, so
+ * createTls(InetSocketAddress, TlsOptions) method, so
* that this method won't have an unwieldy number of direct parameters.
*
* This path uses a builder pattern style:
diff --git a/java/example/pom.xml b/java/example/pom.xml
index c2b226b6806..fa3220f51bd 100644
--- a/java/example/pom.xml
+++ b/java/example/pom.xml
@@ -5,7 +5,7 @@
io.vitess
vitess-parent
- 21.0.0-SNAPSHOT
+ 22.0.0-SNAPSHOT
vitess-example
diff --git a/java/grpc-client/pom.xml b/java/grpc-client/pom.xml
index 41bd4de8291..55c406b0e8f 100644
--- a/java/grpc-client/pom.xml
+++ b/java/grpc-client/pom.xml
@@ -5,7 +5,7 @@
io.vitess
vitess-parent
- 21.0.0-SNAPSHOT
+ 22.0.0-SNAPSHOT
vitess-grpc-client
@@ -51,6 +51,10 @@
io.grpc
grpc-api
+
+ io.grpc
+ grpc-inprocess
+
io.netty
netty-handler
diff --git a/java/grpc-client/src/main/java/io/vitess/client/grpc/RetryingInterceptor.java b/java/grpc-client/src/main/java/io/vitess/client/grpc/RetryingInterceptor.java
index 9323a1b00d4..301ddaed282 100644
--- a/java/grpc-client/src/main/java/io/vitess/client/grpc/RetryingInterceptor.java
+++ b/java/grpc-client/src/main/java/io/vitess/client/grpc/RetryingInterceptor.java
@@ -39,8 +39,8 @@
/**
* RetryingInterceptor is used for retrying certain classes of failed requests in the underlying
- * gRPC connection. At this time it handles {@link MethodDescriptor.MethodType.UNARY} requests with
- * status {@link Status.Code.UNAVAILABLE}, which is according to the spec meant to be a transient
+ * gRPC connection. At this time it handles unary requests with
+ * status Unavailable, which is according to the spec meant to be a transient
* error. This class can be configured with {@link RetryingInterceptorConfig} to determine what
* level of exponential backoff to apply to the handled types of failing requests.
*
diff --git a/java/grpc-client/src/test/java/io/vitess/client/grpc/RetryingInterceptorTest.java b/java/grpc-client/src/test/java/io/vitess/client/grpc/RetryingInterceptorTest.java
index 376a920ad31..9b3644139fd 100644
--- a/java/grpc-client/src/test/java/io/vitess/client/grpc/RetryingInterceptorTest.java
+++ b/java/grpc-client/src/test/java/io/vitess/client/grpc/RetryingInterceptorTest.java
@@ -55,6 +55,8 @@ public void testNoopConfigPassesThrough() throws ExecutionException, Interrupted
Assert.fail("Should have failed after 1 attempt");
} catch (Exception e) {
Assert.assertEquals(1, forceRetryNTimesInterceptor.getNumRetryableFailures());
+ }finally {
+ channel.shutdownNow();
}
}
@@ -75,6 +77,8 @@ public void testRetryAfterBackoff() throws ExecutionException, InterruptedExcept
} catch (Exception e) {
e.printStackTrace();
Assert.assertEquals(3, forceRetryNTimesInterceptor.getNumRetryableFailures());
+ } finally {
+ channel.shutdownNow();
}
}
@@ -95,6 +99,8 @@ public void testRetryDeadlineExceeded() throws ExecutionException, InterruptedEx
Assert.fail("Should have failed");
} catch (Exception e) {
Assert.assertEquals(1, forceRetryNTimesInterceptor.getNumRetryableFailures());
+ }finally {
+ channel.shutdownNow();
}
}
diff --git a/java/jdbc/pom.xml b/java/jdbc/pom.xml
index 49a5d61779b..dd554e64501 100644
--- a/java/jdbc/pom.xml
+++ b/java/jdbc/pom.xml
@@ -5,7 +5,7 @@
io.vitess
vitess-parent
- 21.0.0-SNAPSHOT
+ 22.0.0-SNAPSHOT
vitess-jdbc
diff --git a/java/jdbc/src/main/java/io/vitess/jdbc/VitessConnection.java b/java/jdbc/src/main/java/io/vitess/jdbc/VitessConnection.java
index 5e51ff908dc..f4bc43ed65e 100644
--- a/java/jdbc/src/main/java/io/vitess/jdbc/VitessConnection.java
+++ b/java/jdbc/src/main/java/io/vitess/jdbc/VitessConnection.java
@@ -105,7 +105,7 @@ public Statement createStatement() throws SQLException {
}
/**
- * Create PreparedStatement for the given connection & sql
+ * Create PreparedStatement for the given connection and sql
*
* @param sql - Sql Statement
* @return PreparedStatement Object
@@ -338,7 +338,6 @@ public void setTransactionIsolation(int level) throws SQLException {
/**
* Return Warnings
- *
* TODO: Not implementing as Error is Thrown when occurred
*
* @return SQLWarning or null
diff --git a/java/jdbc/src/main/java/io/vitess/jdbc/VitessDriver.java b/java/jdbc/src/main/java/io/vitess/jdbc/VitessDriver.java
index 5b6a889103f..a77e1ec0860 100644
--- a/java/jdbc/src/main/java/io/vitess/jdbc/VitessDriver.java
+++ b/java/jdbc/src/main/java/io/vitess/jdbc/VitessDriver.java
@@ -60,12 +60,10 @@ public Connection connect(String url, Properties info) throws SQLException {
/**
* Checks whether a given url is in a valid format.
- *
* The current uri format is: jdbc:vitess://[host]:[port]
*
* @param url the URL of the database
* @return true, if this driver understands the given URL; false, otherwise
- *
* TODO: Write a better regex
*/
@Override
diff --git a/java/jdbc/src/main/java/io/vitess/jdbc/VitessJDBCUrl.java b/java/jdbc/src/main/java/io/vitess/jdbc/VitessJDBCUrl.java
index 6234d037807..e6a1b0187dd 100644
--- a/java/jdbc/src/main/java/io/vitess/jdbc/VitessJDBCUrl.java
+++ b/java/jdbc/src/main/java/io/vitess/jdbc/VitessJDBCUrl.java
@@ -93,7 +93,6 @@ public String toString() {
* trustAlias
=alias_under_which_certificate_chain_is_stored (if not set,
* then the first valid X509Certificate
found in the trustStore will be used)
*
- *
*
*
If useSSL=true
, and any of these additional properties are not set on the JDBC
* URL, then the driver will look to see if these corresponding property was set at JVM startup
@@ -109,7 +108,6 @@ public String toString() {
*
-Djavax.net.ssl.trustStorePassword
* -Djavax.net.ssl.trustStoreAlias
*
- *
*
*
See:
* https://mariadb.com/kb/en/mariadb/about-mariadb-connector-j/#tls-ssl
diff --git a/java/jdbc/src/main/java/io/vitess/jdbc/VitessStatement.java b/java/jdbc/src/main/java/io/vitess/jdbc/VitessStatement.java
index ddaf813af2d..90a83b56485 100644
--- a/java/jdbc/src/main/java/io/vitess/jdbc/VitessStatement.java
+++ b/java/jdbc/src/main/java/io/vitess/jdbc/VitessStatement.java
@@ -241,7 +241,6 @@ public void setQueryTimeout(int seconds) throws SQLException {
/**
* Return Warnings
- *
* Not implementing as Error is Thrown when occurred
*
* @return SQLWarning or null
diff --git a/java/jdbc/src/main/java/io/vitess/util/charset/CharsetMapping.java b/java/jdbc/src/main/java/io/vitess/util/charset/CharsetMapping.java
index c1388bfb7bb..343ae0e90eb 100644
--- a/java/jdbc/src/main/java/io/vitess/util/charset/CharsetMapping.java
+++ b/java/jdbc/src/main/java/io/vitess/util/charset/CharsetMapping.java
@@ -541,6 +541,7 @@ public class CharsetMapping {
/**
* MySQL charset could map to several Java encodings. So here we choose the one according to next
* rules:
+ *
* if there is no static mapping for this charset then return javaEncoding value as is
* because this could be a custom charset for example
* if static mapping exists and javaEncoding equals to one of Java encoding canonical names
@@ -550,6 +551,7 @@ public class CharsetMapping {
* if static mapping exists and javaEncoding doesn't match any Java encoding canonical
* names or aliases available for this mapping then return default Java encoding (the first in
* mapping list)
+ *
*/
public static String getJavaEncodingForCollationIndex(Integer collationIndex,
String javaEncoding) {
diff --git a/java/jdbc/src/test/java/io/vitess/jdbc/FieldWithMetadataTest.java b/java/jdbc/src/test/java/io/vitess/jdbc/FieldWithMetadataTest.java
index bcadc49d33a..26ad5fd11b3 100644
--- a/java/jdbc/src/test/java/io/vitess/jdbc/FieldWithMetadataTest.java
+++ b/java/jdbc/src/test/java/io/vitess/jdbc/FieldWithMetadataTest.java
@@ -16,6 +16,9 @@
package io.vitess.jdbc;
+import java.util.Set;
+import java.util.EnumSet;
+
import io.vitess.proto.Query;
import io.vitess.util.MysqlDefs;
import io.vitess.util.charset.CharsetMapping;
@@ -274,6 +277,16 @@ public void testNumericAndDateTimeEncoding() throws SQLException {
}
}
+ // Define the types to skip
+ Set typesToSkip = EnumSet.of(
+ Query.Type.UNRECOGNIZED,
+ Query.Type.EXPRESSION,
+ Query.Type.HEXVAL,
+ Query.Type.HEXNUM,
+ Query.Type.BITNUM,
+ Query.Type.RAW
+ );
+
@Test
public void testPrecisionAdjustFactor() throws SQLException {
VitessConnection conn = getVitessConnection();
@@ -294,7 +307,8 @@ public void testPrecisionAdjustFactor() throws SQLException {
conn.setIncludedFields(Query.ExecuteOptions.IncludedFields.TYPE_AND_NAME);
for (Query.Type type : Query.Type.values()) {
- if (type == Query.Type.UNRECOGNIZED || type == Query.Type.EXPRESSION || type == Query.Type.HEXVAL || type == Query.Type.HEXNUM || type == Query.Type.BITNUM) {
+ // Skip if the type is in the set
+ if (typesToSkip.contains(type)) {
continue;
}
diff --git a/java/pom.xml b/java/pom.xml
index 079f185ad70..6742258a6b8 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -11,7 +11,7 @@
io.vitess
vitess-parent
- 21.0.0-SNAPSHOT
+ 22.0.0-SNAPSHOT
pom
Vitess Java Client libraries [Parent]
@@ -66,16 +66,16 @@
UTF-8
- 1.57.1
+ 1.67.1
- 4.1.94.Final
- 2.0.61.Final
+ 4.1.110.Final
+ 2.0.65.Final
- 3.24.3
+ 4.28.3
3.24.3
3.0.0
- 2.17.1
+ 2.24.1
@@ -89,12 +89,12 @@
com.google.code.gson
gson
- 2.8.9
+ 2.11.0
com.google.guava
guava
- 32.0.0-jre
+ 33.3.1-jre
com.google.protobuf
@@ -109,7 +109,7 @@
commons-io
commons-io
- 2.7
+ 2.17.0
@@ -122,6 +122,11 @@
grpc-netty
${grpc.version}
+
+ io.grpc
+ grpc-inprocess
+ ${grpc.version}
+
io.grpc
grpc-stub
@@ -180,7 +185,7 @@
joda-time
joda-time
- 2.10
+ 2.13.0
@@ -227,7 +232,7 @@
org.apache.maven.plugins
maven-compiler-plugin
- 3.8.1
+ 3.13.0
1.8
1.8
@@ -288,7 +293,7 @@
org.sonatype.plugins
nexus-staging-maven-plugin
- 1.6.13
+ 1.7.0
true
ossrh
@@ -299,7 +304,7 @@
org.apache.maven.plugins
maven-source-plugin
- 2.4
+ 3.3.1
attach-sources
@@ -312,7 +317,7 @@
org.apache.maven.plugins
maven-javadoc-plugin
- 2.10.4
+ 3.3.1
attach-javadocs
@@ -325,7 +330,7 @@
org.apache.maven.plugins
maven-gpg-plugin
- 1.6
+ 3.2.7
sign-artifacts
diff --git a/misc/errcheck_excludes.txt b/misc/errcheck_excludes.txt
deleted file mode 100644
index fb689e26288..00000000000
--- a/misc/errcheck_excludes.txt
+++ /dev/null
@@ -1,49 +0,0 @@
-// This file contains one function signature per line, which errcheck should not warn about.
-//
-// The format for function signatures is `package.FunctionName`.
-// The format for method signatures is `(package.Receiver).MethodName` for value receivers,
-// and the format for pointer receivers is `(*package.Receiver).MethodName`.
-//
-// See https://github.com/kisielk/errcheck#excluding-functions for more details.
-
-flag.Set
-(*flag.FlagSet).Parse
-(flag.Value).Set
-
-fmt.Fprint
-fmt.Fprintf
-
-io.WriteString(fmt.State)
-io.WriteString(net/http.ResponseWriter)
-
-(net.Listener).Close
-(net/http.ResponseWriter).Write
-
-(*os.File).Close
-os.Remove
-os.RemoveAll
-os.Rename
-
-(*github.com/spf13/cobra.Command).Help
-(*github.com/spf13/cobra.Command).MarkFlagRequired
-(*github.com/spf13/cobra.Command).MarkPersistentFlagRequired
-(*github.com/spf13/cobra.Command).MarkPersistentFlagFilename
-
-(*github.com/spf13/pflag.FlagSet).MarkDeprecated
-
-(*google.golang.org/grpc.ClientConn).Close
-(*google.golang.org/grpc.Server).Serve
-
-(*vitess.io/vitess/go/bytes2.Buffer).Write
-(*vitess.io/vitess/go/bytes2.Buffer).WriteByte
-(*vitess.io/vitess/go/bytes2.Buffer).WriteString
-
-(vitess.io/vitess/go/sqltypes.BinWriter).Write
-
-vitess.io/vitess/go/vt/orchestrator/external/golib/log.Errore
-vitess.io/vitess/go/vt/orchestrator/external/golib/log.Errorf
-vitess.io/vitess/go/vt/orchestrator/external/golib/log.Fatal
-vitess.io/vitess/go/vt/orchestrator/external/golib/log.Fatale
-vitess.io/vitess/go/vt/orchestrator/external/golib/log.Fatalf
-
-(*vitess.io/vitess/go/vt/vttest.LocalCluster).TearDown
diff --git a/misc/git/hooks/golangci-lint b/misc/git/hooks/golangci-lint
index d152cb965a3..21313316a12 100755
--- a/misc/git/hooks/golangci-lint
+++ b/misc/git/hooks/golangci-lint
@@ -13,18 +13,58 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-GOLANGCI_LINT=$(command -v golangci-lint >/dev/null 2>&1)
-if [ $? -eq 1 ]; then
- echo "Downloading golangci-lint..."
- go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.56.2
+# Required version of golangci-lint
+REQUIRED_VERSION="v1.60.2"
+
+# Function to compare versions in pure Bash
+version_greater_or_equal() {
+ local IFS=.
+ local i
+ local ver1=($1)
+ local ver2=($2)
+
+ # Fill empty fields in ver1 with zeros
+ for ((i=${#ver1[@]}; i<${#ver2[@]}; i++)); do
+ ver1[i]=0
+ done
+ # Fill empty fields in ver2 with zeros
+ for ((i=${#ver2[@]}; i<${#ver1[@]}; i++)); do
+ ver2[i]=0
+ done
+
+ for ((i=0; i<${#ver1[@]}; i++)); do
+ if ((10#${ver1[i]} > 10#${ver2[i]})); then
+ return 0
+ elif ((10#${ver1[i]} < 10#${ver2[i]})); then
+ return 1
+ fi
+ done
+ return 0
+}
+
+# Check if golangci-lint is installed and capture the version
+if ! command -v golangci-lint >/dev/null 2>&1; then
+ echo "golangci-lint not found. Installing version $REQUIRED_VERSION..."
+ go install github.com/golangci/golangci-lint/cmd/golangci-lint@$REQUIRED_VERSION
+else
+ VERSION_OUTPUT=$(golangci-lint --version)
+ INSTALLED_VERSION=$(echo "$VERSION_OUTPUT" | sed -n 's/^golangci-lint has version v\([0-9.]*\).*/\1/p')
+ if ! version_greater_or_equal "$INSTALLED_VERSION" "${REQUIRED_VERSION#v}"; then
+ echo "golangci-lint version $INSTALLED_VERSION found, but $REQUIRED_VERSION or newer is required."
+ echo "Installing version $REQUIRED_VERSION..."
+ go install github.com/golangci/golangci-lint/cmd/golangci-lint@$REQUIRED_VERSION
+ fi
fi
+# Get list of Go files to lint
gofiles=$(git diff --cached --name-only --diff-filter=ACM | grep '^go/.*\.go$')
if [ -z "$gofiles" ]; then
exit 0
fi
+# Get unique directories of the Go files
gopackages=$(echo "$gofiles" | xargs -n1 dirname | sort -u | paste -sd ' ' -)
+# Lint the Go packages
echo "Linting $gopackages"
-golangci-lint run $gopackages
+golangci-lint run $gopackages
\ No newline at end of file
diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto
index 5f5bbd59c6e..e1df792776b 100644
--- a/proto/binlogdata.proto
+++ b/proto/binlogdata.proto
@@ -344,6 +344,7 @@ message RowEvent {
string keyspace = 3;
string shard = 4;
uint32 flags = 5; // https://dev.mysql.com/doc/dev/mysql-server/latest/classbinary__log_1_1Rows__event.html
+ bool is_internal_table = 6; // set for sidecardb tables
}
// FieldEvent represents the field info for a table.
@@ -352,6 +353,10 @@ message FieldEvent {
repeated query.Field fields = 2;
string keyspace = 3;
string shard = 4;
+
+ // Field numbers in the gap between shard (4) and enum_set_string_values
+ // (25) are NOT reserved and can be used.
+
// Are ENUM and SET field values already mapped to strings in the ROW
// events? This allows us to transition VTGate VStream consumers from
// the pre v20 behavior of having to do this mapping themselves to the
@@ -360,6 +365,10 @@ message FieldEvent {
// NOTE: because this is the use case, this is ONLY ever set today in
// vstreams managed by the vstreamManager.
bool enum_set_string_values = 25;
+ bool is_internal_table = 26; // set for sidecardb tables
+
+ // Add new members in the field number gap between shard (4) and
+ // enum_set_string_values (25).
}
// ShardGtid contains the GTID position for one shard.
@@ -476,6 +485,11 @@ message MinimalSchema {
repeated MinimalTable tables = 1;
}
+message VStreamOptions {
+ repeated string internal_tables = 1;
+ map config_overrides = 2;
+}
+
// VStreamRequest is the payload for VStreamer
message VStreamRequest {
vtrpc.CallerID effective_caller_id = 1;
@@ -485,6 +499,8 @@ message VStreamRequest {
string position = 4;
Filter filter = 5;
repeated TableLastPK table_last_p_ks = 6;
+
+ VStreamOptions options = 7;
}
// VStreamResponse is the response from VStreamer
@@ -500,6 +516,7 @@ message VStreamRowsRequest {
string query = 4;
query.QueryResult lastpk = 5;
+ VStreamOptions options = 6;
}
// VStreamRowsResponse is the response from VStreamRows
@@ -523,6 +540,7 @@ message VStreamTablesRequest {
vtrpc.CallerID effective_caller_id = 1;
query.VTGateCallerID immediate_caller_id = 2;
query.Target target = 3;
+ VStreamOptions options = 4;
}
// VStreamTablesResponse is the response from VStreamTables
diff --git a/proto/mysqlctl.proto b/proto/mysqlctl.proto
index 7e5fe13b991..095172c6a1b 100644
--- a/proto/mysqlctl.proto
+++ b/proto/mysqlctl.proto
@@ -24,6 +24,7 @@ package mysqlctl;
import "topodata.proto";
import "vttime.proto";
+import "vtrpc.proto";
message StartRequest{
repeated string mysqld_args = 1;
@@ -79,6 +80,22 @@ message VersionStringResponse{
string version = 1;
}
+message HostMetricsRequest{}
+
+message HostMetricsResponse{
+ message Metric {
+ // Name of the metric
+ string name = 1;
+ // Value is the metric value
+ double value = 2;
+ // Error indicates an error retrieving the value
+ vtrpc.RPCError error = 3;
+ }
+ // Metrics is a map (metric name -> metric value/error) so that the client has as much
+ // information as possible about all the checked metrics.
+ map metrics = 1;
+}
+
// MysqlCtl is the service definition
service MysqlCtl {
rpc Start(StartRequest) returns (StartResponse) {};
@@ -89,6 +106,7 @@ service MysqlCtl {
rpc ReinitConfig(ReinitConfigRequest) returns (ReinitConfigResponse) {};
rpc RefreshConfig(RefreshConfigRequest) returns (RefreshConfigResponse) {};
rpc VersionString(VersionStringRequest) returns (VersionStringResponse) {};
+ rpc HostMetrics(HostMetricsRequest) returns (HostMetricsResponse) {};
}
// BackupInfo is the read-only attributes of a mysqlctl/backupstorage.BackupHandle.
diff --git a/proto/query.proto b/proto/query.proto
index 1332de76ab2..c5f53ea6e5d 100644
--- a/proto/query.proto
+++ b/proto/query.proto
@@ -218,6 +218,8 @@ enum Type {
// VECTOR specifies a VECTOR type
// Properties: 35, IsQuoted.
VECTOR = 2083;
+ // RAW specifies a type which won't be quoted but the value used as-is while encoding.
+ RAW = 2084;
}
// Value represents a typed value.
@@ -356,6 +358,11 @@ message ExecuteOptions {
// priority specifies the priority of the query, between 0 and 100. This is leveraged by the transaction
// throttler to determine whether, under resource contention, a query should or should not be throttled.
string priority = 16;
+
+ // timeout specifies the query timeout in milliseconds. If not set, the default timeout is used.
+ oneof timeout {
+ int64 authoritative_timeout = 17;
+ }
}
// Field describes a single column returned by a query
@@ -602,8 +609,16 @@ message StartCommitRequest {
string dtid = 5;
}
+enum StartCommitState {
+ Unknown = 0;
+ Fail = 1;
+ Success = 2;
+}
+
// StartCommitResponse is the returned value from StartCommit
-message StartCommitResponse {}
+message StartCommitResponse {
+ StartCommitState state = 1;
+}
// SetRollbackRequest is the payload to SetRollback
message SetRollbackRequest {
@@ -646,6 +661,7 @@ message UnresolvedTransactionsRequest {
vtrpc.CallerID effective_caller_id = 1;
VTGateCallerID immediate_caller_id = 2;
Target target = 3;
+ int64 abandon_age = 4; // Unresolved Transactions older than this (in seconds).
}
// UnresolvedTransactionsResponse is the returned value from UnresolvedTransactions
diff --git a/proto/replicationdata.proto b/proto/replicationdata.proto
index 1a8b608f984..e788fc64bc3 100644
--- a/proto/replicationdata.proto
+++ b/proto/replicationdata.proto
@@ -50,6 +50,7 @@ message Status {
bool has_replication_filters = 22;
bool ssl_allowed = 23;
bool replication_lag_unknown = 24;
+ bool backup_running = 25;
}
// Configuration holds replication configuration information gathered from performance_schema and global variables.
@@ -77,6 +78,7 @@ enum StopReplicationMode {
message PrimaryStatus {
string position = 1;
string file_position = 2;
+ string server_uuid = 3;
}
// FullStatus contains the full status of MySQL including the replication information, semi-sync information, GTID information among others
diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto
index cd74e79fa5d..bb20e712e7f 100644
--- a/proto/tabletmanagerdata.proto
+++ b/proto/tabletmanagerdata.proto
@@ -29,6 +29,7 @@ import "replicationdata.proto";
import "logutil.proto";
import "vttime.proto";
import "vtrpc.proto";
+import "mysqlctl.proto";
//
// Data structures
@@ -309,6 +310,51 @@ message ExecuteFetchAsAppResponse {
query.QueryResult result = 1;
}
+message GetUnresolvedTransactionsRequest {
+ int64 abandon_age = 1;
+}
+
+message GetUnresolvedTransactionsResponse {
+ repeated query.TransactionMetadata transactions = 1;
+}
+
+message ReadTransactionRequest {
+ string dtid = 1;
+}
+
+message ReadTransactionResponse {
+ query.TransactionMetadata transaction = 1;
+}
+
+message GetTransactionInfoRequest {
+ string dtid = 1;
+}
+
+message GetTransactionInfoResponse {
+ string state = 1;
+ string message = 2;
+ int64 time_created = 3;
+ repeated string statements = 4;
+}
+
+
+message ConcludeTransactionRequest {
+ string dtid = 1;
+ bool mm = 2;
+}
+
+message ConcludeTransactionResponse {
+}
+
+
+message MysqlHostMetricsRequest {
+}
+
+message MysqlHostMetricsResponse {
+ mysqlctl.HostMetricsResponse HostMetrics = 1;
+}
+
+
message ReplicationStatusRequest {
}
@@ -414,6 +460,13 @@ message PopulateReparentJournalRequest {
message PopulateReparentJournalResponse {
}
+message ReadReparentJournalInfoRequest {
+}
+
+message ReadReparentJournalInfoResponse {
+ int32 length = 1;
+}
+
message InitReplicaRequest {
topodata.TabletAlias parent = 1;
string replication_position = 2;
@@ -516,6 +569,8 @@ message BackupRequest {
// UpgradeSafe indicates if the backup should be taken with innodb_fast_shutdown=0
// so that it's a backup that can be used for an upgrade.
bool upgrade_safe = 4;
+ // BackupEngine specifies if we want to use a particular backup engine for this backup request
+ optional string backup_engine = 5;
}
message BackupResponse {
@@ -533,6 +588,8 @@ message RestoreFromBackupRequest {
// RestoreToTimestamp, if given, requested an inremental restore up to (and excluding) the given timestamp.
// RestoreToTimestamp and RestoreToPos are mutually exclusive.
vttime.Time restore_to_timestamp = 4;
+ // AllowedBackupEngines, if present will filter out any backups taken with engines not included in the list
+ repeated string allowed_backup_engines = 5;
}
message RestoreFromBackupResponse {
@@ -567,6 +624,18 @@ message CreateVReplicationWorkflowResponse {
query.QueryResult result = 1;
}
+message DeleteTableDataRequest {
+ // The key is the table that we want to delete data from.
+ // The value is the filter or WHERE clause to use when deleting
+ // data in the table.
+ map table_filters = 1;
+ // BatchSize is the number of rows to delete in a single batch.
+ int64 batch_size = 2;
+}
+
+message DeleteTableDataResponse {
+}
+
message DeleteVReplicationWorkflowRequest {
string workflow = 1;
}
@@ -627,6 +696,18 @@ message ReadVReplicationWorkflowResponse {
}
repeated Stream streams = 11;
string options = 12;
+ map config_overrides = 13;
+}
+
+message ValidateVReplicationPermissionsRequest {
+}
+
+message ValidateVReplicationPermissionsResponse {
+ // The --db_filtered_user on the tablet.
+ string user = 1;
+ // Does the user have the minimum privileges needed to manage
+ // vreplication metadata.
+ bool ok = 2;
}
message VDiffRequest {
@@ -657,6 +738,7 @@ message VDiffReportOptions {
bool debug_query = 2;
string format = 3;
int64 max_sample_rows = 4;
+ int64 row_diff_column_truncate_at = 5;
}
message VDiffCoreOptions {
@@ -669,6 +751,7 @@ message VDiffCoreOptions {
int64 max_extra_rows_to_compare = 7;
bool update_table_stats = 8;
int64 max_diff_seconds = 9;
+ optional bool auto_start = 10;
}
message VDiffOptions {
@@ -677,6 +760,7 @@ message VDiffOptions {
VDiffReportOptions report_options = 3;
}
+
// UpdateVReplicationWorkflowRequest is used to update an existing VReplication
// workflow. Note that the following fields MUST have an explicit value provided
// if you do NOT wish to update the existing value to the given type's ZeroValue:
@@ -687,10 +771,11 @@ message UpdateVReplicationWorkflowRequest {
string workflow = 1;
repeated string cells = 2;
repeated topodata.TabletType tablet_types = 3;
- TabletSelectionPreference tablet_selection_preference = 4;
- binlogdata.OnDDLAction on_ddl = 5;
- binlogdata.VReplicationWorkflowState state = 6;
+ optional TabletSelectionPreference tablet_selection_preference = 4;
+ optional binlogdata.OnDDLAction on_ddl = 5;
+ optional binlogdata.VReplicationWorkflowState state = 6;
reserved 7; // unused, was: repeated string shards
+ map config_overrides = 8;
}
message UpdateVReplicationWorkflowResponse {
@@ -707,9 +792,9 @@ message UpdateVReplicationWorkflowsRequest {
bool all_workflows = 1;
repeated string include_workflows = 2;
repeated string exclude_workflows = 3;
- binlogdata.VReplicationWorkflowState state = 4;
- string message = 5;
- string stop_position = 6;
+ optional binlogdata.VReplicationWorkflowState state = 4;
+ optional string message = 5;
+ optional string stop_position = 6;
}
message UpdateVReplicationWorkflowsResponse {
@@ -854,3 +939,12 @@ message GetThrottlerStatusResponse {
// RecentApps is a map of app names to their recent check status
map recent_apps = 18;
}
+
+message ChangeTagsRequest {
+ map tags = 1;
+ bool replace = 2;
+}
+
+message ChangeTagsResponse {
+ map tags = 1;
+}
diff --git a/proto/tabletmanagerservice.proto b/proto/tabletmanagerservice.proto
index 2a593273a0c..3dac1aa1ee8 100644
--- a/proto/tabletmanagerservice.proto
+++ b/proto/tabletmanagerservice.proto
@@ -57,6 +57,9 @@ service TabletManager {
rpc SetReadWrite(tabletmanagerdata.SetReadWriteRequest) returns (tabletmanagerdata.SetReadWriteResponse) {};
+ // ChangeTags asks the remote tablet to change its tags
+ rpc ChangeTags(tabletmanagerdata.ChangeTagsRequest) returns (tabletmanagerdata.ChangeTagsResponse) {};
+
// ChangeType asks the remote tablet to change its type
rpc ChangeType(tabletmanagerdata.ChangeTypeRequest) returns (tabletmanagerdata.ChangeTypeResponse) {};
@@ -86,6 +89,20 @@ service TabletManager {
rpc ExecuteFetchAsApp(tabletmanagerdata.ExecuteFetchAsAppRequest) returns (tabletmanagerdata.ExecuteFetchAsAppResponse) {};
+ //
+ // Distributed Transaction related methods
+ //
+
+ rpc GetUnresolvedTransactions(tabletmanagerdata.GetUnresolvedTransactionsRequest) returns (tabletmanagerdata.GetUnresolvedTransactionsResponse) {};
+
+ rpc ReadTransaction(tabletmanagerdata.ReadTransactionRequest) returns (tabletmanagerdata.ReadTransactionResponse) {};
+
+ rpc GetTransactionInfo(tabletmanagerdata.GetTransactionInfoRequest) returns (tabletmanagerdata.GetTransactionInfoResponse) {};
+
+ rpc ConcludeTransaction(tabletmanagerdata.ConcludeTransactionRequest) returns (tabletmanagerdata.ConcludeTransactionResponse) {};
+
+ rpc MysqlHostMetrics(tabletmanagerdata.MysqlHostMetricsRequest) returns (tabletmanagerdata.MysqlHostMetricsResponse) {};
+
//
// Replication related methods
//
@@ -121,14 +138,16 @@ service TabletManager {
// VReplication API
rpc CreateVReplicationWorkflow(tabletmanagerdata.CreateVReplicationWorkflowRequest) returns (tabletmanagerdata.CreateVReplicationWorkflowResponse) {};
+ rpc DeleteTableData(tabletmanagerdata.DeleteTableDataRequest) returns(tabletmanagerdata.DeleteTableDataResponse) {}
rpc DeleteVReplicationWorkflow(tabletmanagerdata.DeleteVReplicationWorkflowRequest) returns(tabletmanagerdata.DeleteVReplicationWorkflowResponse) {};
rpc HasVReplicationWorkflows(tabletmanagerdata.HasVReplicationWorkflowsRequest) returns(tabletmanagerdata.HasVReplicationWorkflowsResponse) {};
rpc ReadVReplicationWorkflow(tabletmanagerdata.ReadVReplicationWorkflowRequest) returns(tabletmanagerdata.ReadVReplicationWorkflowResponse) {};
rpc ReadVReplicationWorkflows(tabletmanagerdata.ReadVReplicationWorkflowsRequest) returns(tabletmanagerdata.ReadVReplicationWorkflowsResponse) {};
- rpc VReplicationExec(tabletmanagerdata.VReplicationExecRequest) returns(tabletmanagerdata.VReplicationExecResponse) {};
- rpc VReplicationWaitForPos(tabletmanagerdata.VReplicationWaitForPosRequest) returns(tabletmanagerdata.VReplicationWaitForPosResponse) {};
rpc UpdateVReplicationWorkflow(tabletmanagerdata.UpdateVReplicationWorkflowRequest) returns(tabletmanagerdata.UpdateVReplicationWorkflowResponse) {};
rpc UpdateVReplicationWorkflows(tabletmanagerdata.UpdateVReplicationWorkflowsRequest) returns(tabletmanagerdata.UpdateVReplicationWorkflowsResponse) {};
+ rpc ValidateVReplicationPermissions(tabletmanagerdata.ValidateVReplicationPermissionsRequest) returns(tabletmanagerdata.ValidateVReplicationPermissionsResponse) {};
+ rpc VReplicationExec(tabletmanagerdata.VReplicationExecRequest) returns(tabletmanagerdata.VReplicationExecResponse) {};
+ rpc VReplicationWaitForPos(tabletmanagerdata.VReplicationWaitForPosRequest) returns(tabletmanagerdata.VReplicationWaitForPosResponse) {};
// VDiff API
rpc VDiff(tabletmanagerdata.VDiffRequest) returns(tabletmanagerdata.VDiffResponse) {};
@@ -147,6 +166,9 @@ service TabletManager {
// reparent journal
rpc PopulateReparentJournal(tabletmanagerdata.PopulateReparentJournalRequest) returns (tabletmanagerdata.PopulateReparentJournalResponse) {};
+ // ReadReparentJournalInfo reads the information from reparent journal
+ rpc ReadReparentJournalInfo(tabletmanagerdata.ReadReparentJournalInfoRequest) returns (tabletmanagerdata.ReadReparentJournalInfoResponse) {};
+
// InitReplica tells the tablet to reparent to the primary unconditionally
rpc InitReplica(tabletmanagerdata.InitReplicaRequest) returns (tabletmanagerdata.InitReplicaResponse) {};
diff --git a/proto/vtadmin.proto b/proto/vtadmin.proto
index d6f1047fc1e..963d1fa5779 100644
--- a/proto/vtadmin.proto
+++ b/proto/vtadmin.proto
@@ -44,6 +44,8 @@ service VTAdmin {
// CompleteSchemaMigration completes one or all migrations in the given
// cluster executed with --postpone-completion.
rpc CompleteSchemaMigration(CompleteSchemaMigrationRequest) returns (vtctldata.CompleteSchemaMigrationResponse) {};
+ // ConcludeTransaction concludes a distributed transaction identified by the provided dtid.
+ rpc ConcludeTransaction(ConcludeTransactionRequest) returns (vtctldata.ConcludeTransactionResponse) {};
// CreateKeyspace creates a new keyspace in the given cluster.
rpc CreateKeyspace(CreateKeyspaceRequest) returns (CreateKeyspaceResponse) {};
// CreateShard creates a new shard in the given cluster and keyspace.
@@ -114,6 +116,10 @@ service VTAdmin {
rpc GetTablets(GetTabletsRequest) returns (GetTabletsResponse) {};
// GetTopologyPath returns the cell located at the specified path in the topology server.
rpc GetTopologyPath(GetTopologyPathRequest) returns (vtctldata.GetTopologyPathResponse){};
+ // GetTransactionInfo returns the information about a single transaction.
+ rpc GetTransactionInfo(GetTransactionInfoRequest) returns (vtctldata.GetTransactionInfoResponse){};
+ // GetUnresolvedTransactions returns the unresolved transactions for the request.
+ rpc GetUnresolvedTransactions(GetUnresolvedTransactionsRequest) returns (vtctldata.GetUnresolvedTransactionsResponse){};
// GetVSchema returns a VSchema for the specified keyspace in the specified
// cluster.
rpc GetVSchema(GetVSchemaRequest) returns (VSchema) {};
@@ -126,9 +132,24 @@ service VTAdmin {
rpc GetWorkflow(GetWorkflowRequest) returns (Workflow) {};
// GetWorkflows returns the Workflows for all specified clusters.
rpc GetWorkflows(GetWorkflowsRequest) returns (GetWorkflowsResponse) {};
+ // GetWorkflowStatus returns the status for a specific workflow.
+ rpc GetWorkflowStatus(GetWorkflowStatusRequest) returns (vtctldata.WorkflowStatusResponse) {};
+ // StartWorkflow starts a vreplication workflow.
+ rpc StartWorkflow(StartWorkflowRequest) returns (vtctldata.WorkflowUpdateResponse) {};
+ // StopWorkflow stops a vreplication workflow.
+ rpc StopWorkflow(StopWorkflowRequest) returns (vtctldata.WorkflowUpdateResponse) {};
// LaunchSchemaMigration launches one or all migrations in the given
// cluster executed with --postpone-launch.
rpc LaunchSchemaMigration(LaunchSchemaMigrationRequest) returns (vtctldata.LaunchSchemaMigrationResponse) {};
+ // MoveTablesComplete completes the move and cleans up the workflow and
+ // its related artifacts.
+ rpc MoveTablesComplete(MoveTablesCompleteRequest) returns (vtctldata.MoveTablesCompleteResponse) {};
+ // MoveTablesCreate creates a workflow which moves one or more tables from a
+ // source keyspace to a target keyspace.
+ rpc MoveTablesCreate(MoveTablesCreateRequest) returns (vtctldata.WorkflowStatusResponse) {};
+ // MaterializeCreate creates a workflow to materialize one or more tables
+ // from a source keyspace to a target keyspace using a provided expressions.
+ rpc MaterializeCreate(MaterializeCreateRequest) returns (vtctldata.MaterializeCreateResponse) {};
// PingTablet checks that the specified tablet is awake and responding to
// RPCs. This command can be blocked by other in-flight operations.
rpc PingTablet(PingTabletRequest) returns (PingTabletResponse) {};
@@ -160,6 +181,8 @@ service VTAdmin {
rpc RetrySchemaMigration(RetrySchemaMigrationRequest) returns (vtctldata.RetrySchemaMigrationResponse) {};
// RunHealthCheck runs a healthcheck on the tablet.
rpc RunHealthCheck(RunHealthCheckRequest) returns (RunHealthCheckResponse) {};
+ // ReshardCreate creates a workflow to reshard a keyspace.
+ rpc ReshardCreate(ReshardCreateRequest) returns (vtctldata.WorkflowStatusResponse) {};
// SetReadOnly sets the tablet to read-only mode.
rpc SetReadOnly(SetReadOnlyRequest) returns (SetReadOnlyResponse) {};
// SetReadWrite sets the tablet to read-write mode.
@@ -197,9 +220,15 @@ service VTAdmin {
rpc ValidateVersionKeyspace(ValidateVersionKeyspaceRequest) returns (vtctldata.ValidateVersionKeyspaceResponse) {};
// ValidateVersionShard validates that the version on the primary matches all of the replicas.
rpc ValidateVersionShard(ValidateVersionShardRequest) returns (vtctldata.ValidateVersionShardResponse) {};
+ rpc VDiffCreate(VDiffCreateRequest) returns (vtctldata.VDiffCreateResponse) {};
+ rpc VDiffShow(VDiffShowRequest) returns (VDiffShowResponse) {};
// VTExplain provides information on how Vitess plans to execute a
// particular query.
rpc VTExplain(VTExplainRequest) returns (VTExplainResponse) {};
+ // WorkflowDelete deletes a vreplication workflow.
+ rpc WorkflowDelete(WorkflowDeleteRequest) returns (vtctldata.WorkflowDeleteResponse) {};
+ // WorkflowSwitchTraffic switches traffic for a VReplication workflow.
+ rpc WorkflowSwitchTraffic(WorkflowSwitchTrafficRequest) returns (vtctldata.WorkflowSwitchTrafficResponse) {};
}
/* Data types */
@@ -345,11 +374,25 @@ message Workflow {
vtctldata.Workflow workflow = 3;
}
+message WorkflowDeleteRequest {
+ string cluster_id = 1;
+ vtctldata.WorkflowDeleteRequest request = 2;
+}
+
+message WorkflowSwitchTrafficRequest {
+ string cluster_id = 1;
+ vtctldata.WorkflowSwitchTrafficRequest request = 2;
+}
+
/* Request/Response types */
message ApplySchemaRequest {
string cluster_id = 1;
- vtctldata.ApplySchemaRequest request = 2;
+ // Request.Sql will be overriden by this Sql field.
+ string sql = 2;
+ // Request.CallerId will be overriden by this CallerId field.
+ string caller_id = 3;
+ vtctldata.ApplySchemaRequest request = 4;
}
message CancelSchemaMigrationRequest {
@@ -367,6 +410,11 @@ message CompleteSchemaMigrationRequest {
vtctldata.CompleteSchemaMigrationRequest request = 2;
}
+message ConcludeTransactionRequest {
+ string cluster_id = 1;
+ string dtid = 2;
+}
+
message CreateKeyspaceRequest {
string cluster_id = 1;
vtctldata.CreateKeyspaceRequest options = 2;
@@ -613,6 +661,17 @@ message GetTopologyPathRequest {
string path = 2;
}
+message GetTransactionInfoRequest {
+ string cluster_id = 1;
+ vtctldata.GetTransactionInfoRequest request = 2;
+}
+
+message GetUnresolvedTransactionsRequest {
+ string cluster_id = 1;
+ string keyspace = 2;
+ int64 abandon_age = 3;
+}
+
message GetVSchemaRequest {
string cluster_id = 1;
string keyspace = 2;
@@ -641,6 +700,24 @@ message GetWorkflowRequest {
bool active_only = 4;
}
+message GetWorkflowStatusRequest {
+ string cluster_id = 1;
+ string keyspace = 2;
+ string name = 3;
+}
+
+message StartWorkflowRequest {
+ string cluster_id = 1;
+ string keyspace = 2;
+ string workflow = 3;
+}
+
+message StopWorkflowRequest {
+ string cluster_id = 1;
+ string keyspace = 2;
+ string workflow = 3;
+}
+
message GetWorkflowsRequest {
repeated string cluster_ids = 1;
// ActiveOnly specifies whether to return workflows that are currently
@@ -672,6 +749,24 @@ message LaunchSchemaMigrationRequest {
vtctldata.LaunchSchemaMigrationRequest request = 2;
}
+message MaterializeCreateRequest {
+ string cluster_id = 1;
+ // TableSettings is a JSON string defining what tables to materialize using
+ // what select statements.
+ string table_settings = 2;
+ vtctldata.MaterializeCreateRequest request = 3;
+}
+
+message MoveTablesCompleteRequest {
+ string cluster_id = 1;
+ vtctldata.MoveTablesCompleteRequest request = 2;
+}
+
+message MoveTablesCreateRequest {
+ string cluster_id = 1;
+ vtctldata.MoveTablesCreateRequest request = 2;
+}
+
message PingTabletRequest {
// Unique (per cluster) tablet alias of the standard form: "$cell-$uid"
topodata.TabletAlias alias = 1;
@@ -870,6 +965,11 @@ message RunHealthCheckResponse {
Cluster cluster = 2;
}
+message ReshardCreateRequest {
+ string cluster_id = 1;
+ vtctldata.ReshardCreateRequest request = 2;
+}
+
message SetReadOnlyRequest {
topodata.TabletAlias alias = 1;
repeated string cluster_ids = 2;
@@ -960,6 +1060,34 @@ message ValidateVersionShardRequest {
string shard = 3;
}
+message VDiffCreateRequest {
+ string cluster_id = 1;
+ vtctldata.VDiffCreateRequest request = 2;
+}
+
+message VDiffShowRequest {
+ string cluster_id = 1;
+ vtctldata.VDiffShowRequest request = 2;
+}
+
+message VDiffProgress {
+ double percentage = 1;
+ string eta = 2;
+}
+
+message VDiffShardReport {
+ string state = 1;
+ int64 rows_compared = 2;
+ bool has_mismatch = 3;
+ string started_at = 4;
+ string completed_at = 5;
+ VDiffProgress progress = 6;
+}
+
+message VDiffShowResponse {
+ map shard_report = 1;
+}
+
message VTExplainRequest {
string cluster = 1;
string keyspace = 2;
diff --git a/proto/vtctldata.proto b/proto/vtctldata.proto
index 869e50a23df..b1e5fb215bc 100644
--- a/proto/vtctldata.proto
+++ b/proto/vtctldata.proto
@@ -98,6 +98,9 @@ message MaterializeSettings {
tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 15;
bool atomic_copy = 16;
WorkflowOptions workflow_options = 17;
+
+ // ReferenceTables is set to a csv list of tables, if the materialization is for reference tables.
+ repeated string reference_tables = 18;
}
/* Data types for VtctldServer */
@@ -206,14 +209,25 @@ message Shard {
topodata.Shard shard = 3;
}
+enum ShardedAutoIncrementHandling {
+ LEAVE = 0;
+ REMOVE = 1;
+ REPLACE = 2;
+}
+
message WorkflowOptions {
string tenant_id = 1;
// Remove auto_increment clauses on tables when moving them to a sharded
- // keyspace.
- bool strip_sharded_auto_increment = 2;
+ // keyspace and optionally replace them with vschema AutoIncrement
+ // definitions.
+ ShardedAutoIncrementHandling sharded_auto_increment_handling = 2;
// Shards on which vreplication streams in the target keyspace are created for this workflow and to which the data
// from the source will be vreplicated.
repeated string shards = 3;
+ map config = 4;
+ // Where to create any related schema and vschema objects such as
+ // sequence tables.
+ string global_keyspace = 5;
}
// TODO: comment the hell out of this.
@@ -445,6 +459,8 @@ message BackupRequest {
// UpgradeSafe indicates if the backup should be taken with innodb_fast_shutdown=0
// so that it's a backup that can be used for an upgrade.
bool upgrade_safe = 5;
+ // BackupEngine specifies if we want to use a particular backup engine for this backup request
+ optional string backup_engine = 6;
}
message BackupResponse {
@@ -481,6 +497,17 @@ message CancelSchemaMigrationResponse {
map rows_affected_by_shard = 1;
}
+message ChangeTabletTagsRequest {
+ topodata.TabletAlias tablet_alias = 1;
+ map tags = 2;
+ bool replace = 3;
+}
+
+message ChangeTabletTagsResponse {
+ map before_tags = 1;
+ map after_tags = 2;
+}
+
message ChangeTabletTypeRequest {
topodata.TabletAlias tablet_alias = 1;
topodata.TabletType db_type = 2;
@@ -680,6 +707,9 @@ message EmergencyReparentShardRequest {
// WaitForAllTablets makes ERS wait for a response from all the tablets before proceeding.
// Useful when all the tablets are up and reachable.
bool wait_for_all_tablets = 7;
+ // ExpectedPrimary is the optional alias we expect to be the current primary in order for
+ // the reparent operation to succeed.
+ topodata.TabletAlias expected_primary = 8;
}
message EmergencyReparentShardResponse {
@@ -1101,6 +1131,41 @@ message TopologyCell {
int64 version = 5;
}
+message GetUnresolvedTransactionsRequest {
+ string keyspace = 1;
+ int64 abandon_age = 2; // in seconds
+}
+
+message GetUnresolvedTransactionsResponse {
+ repeated query.TransactionMetadata transactions = 1;
+}
+
+message GetTransactionInfoRequest {
+ string dtid = 1;
+}
+
+message ShardTransactionState {
+ string shard = 1;
+ string state = 2;
+ string message = 3;
+ int64 time_created = 4;
+ repeated string statements = 5;
+}
+
+message GetTransactionInfoResponse {
+ query.TransactionMetadata metadata = 1;
+ repeated ShardTransactionState shard_states = 2;
+}
+
+
+message ConcludeTransactionRequest {
+ string dtid = 1;
+ repeated query.Target participants = 2;
+}
+
+message ConcludeTransactionResponse {
+}
+
message GetVSchemaRequest {
string keyspace = 1;
}
@@ -1359,6 +1424,9 @@ message PlannedReparentShardRequest {
vttime.Duration tolerable_replication_lag = 6;
// AllowCrossCellPromotion allows cross cell promotion,
bool allow_cross_cell_promotion = 7;
+ // ExpectedPrimary is the optional alias we expect to be the current primary in order for
+ // the reparent operation to succeed.
+ topodata.TabletAlias expected_primary = 8;
}
message PlannedReparentShardResponse {
@@ -1525,6 +1593,8 @@ message ReshardCreateRequest {
bool defer_secondary_keys = 11;
// Start the workflow after creating it.
bool auto_start = 12;
+ WorkflowOptions workflow_options = 13;
+
}
message RestoreFromBackupRequest {
@@ -1541,6 +1611,8 @@ message RestoreFromBackupRequest {
// RestoreToTimestamp, if given, requested an inremental restore up to (and excluding) the given timestamp.
// RestoreToTimestamp and RestoreToPos are mutually exclusive.
vttime.Time restore_to_timestamp = 5;
+ // AllowedBackupEngines, if present will filter out any backups taken with engines not included in the list
+ repeated string allowed_backup_engines = 6;
}
message RestoreFromBackupResponse {
@@ -1842,26 +1914,86 @@ message ValidateVSchemaResponse {
}
message VDiffCreateRequest {
+ // The name of the workflow that we're diffing tables for.
string workflow = 1;
+ // The keyspace where the vreplication workflow is running.
string target_keyspace = 2;
+ // A unique identifier for the vdiff.
+ // If empty, a new UUID will be generated.
string uuid = 3;
+ // The cells to look for source tablets in.
+ // If empty, all cells are used.
repeated string source_cells = 4;
+ // The cells to look for target tablets in.
+ // If empty, all cells are used.
repeated string target_cells = 5;
+ // The tablet types to use when searching for tablets to use when streaming
+ // results.
+ // A default value of "replica,rdonly,primary" is used by the tablet picker.
repeated topodata.TabletType tablet_types = 6;
+ // When performing source tablet selection, look for candidates in the type
+ // order as they are listed in the tablet_types value (or the default of
+ // "replica,rdonly,primary" that the tablet picker uses).
+ // The default is ANY (0) and you can use INORDER (1) to e.g. ensure that a
+ // primary tablet is only used if there are no available replica or rdonly
+ // tablets.
tabletmanagerdata.TabletSelectionPreference tablet_selection_preference = 7;
+ // The tables to compare. If empty, all tables in the workflow are compared.
repeated string tables = 8;
+ // The maximum number of rows to compare for each table on each shard.
+ // The default is a max int64 value: 2^63 - 1 or 9,223,372,036,854,775,807.
int64 limit = 9;
+ // How long to wait for the relevant vreplication stream(s) to catch up when
+ // attempting to setup the table snapshots on the source and target to use for
+ // the diff on each shard.
+ // The default is 30s.
vttime.Duration filtered_replication_wait_time = 10;
+ // Include the MySQL query used for the diff in the report that is stored on
+ // each shard primary tablet in the _vt.vdiff_table records.
bool debug_query = 11;
+ // Only show the Primary Key columns in any row diff output. You would
+ // typically want to use this if you set the max_sample_rows very high.
bool only_p_ks = 12;
+ // Update the table statistics, using ANALYZE TABLE, on each table involved
+ // in the vdiff during initialization on each target shard. This will ensure
+ // that progress estimates are as accurate as possible -- but it does involve
+ // locks and can potentially impact query processing on the target keyspace.
bool update_table_stats = 13;
+ // If there are collation differences between the source and target, you can
+ // have rows that are identical but simply returned in a different order from
+ // MySQL. We will do a second pass to compare the rows for any actual
+ // differences in this case and this flag allows you to control the resources
+ // used for this operation.
+ // The default is 0, comparing no extra rows.
int64 max_extra_rows_to_compare = 14;
+ // Wait for the vdiff to complete before returning (making the call synchronous
+ // vs asynchronous by default).
bool wait = 15;
+ // When wait is true, this is how frequently the vdiff progress will be shown.
vttime.Duration wait_update_interval = 16;
+ // Automatically retry the vdiff if we encounter an error. This should almost
+ // always be set to true (default is false).
bool auto_retry = 17;
+ // Include additional information in the vdiff report that is produced and
+ // stored on each target shard primary's _vt sidecar database.
bool verbose = 18;
+ // The maximum number of rows to include in the row diff report (when
+ // differences are found) for each table on each shard.
+ // The default is 0, which will include no sample rows that differed.
int64 max_report_sample_rows = 19;
+ // The maximum time that a diff of a single table can run on each target shard
+ // before it is stopped and then later resumed from where we left off. This
+ // can be helpful in limiting the impact of holding open that large transaction
+ // where we scan up to every row in the table.
+ // The default is 0 or no limit.
vttime.Duration max_diff_duration = 20;
+ // At what length should we truncate the column values in the row diff report
+ // generated for each table on each shard when differences are detected.
+ // The default is 0, meaning do not truncate.
+ int64 row_diff_column_truncate_at = 21;
+ // Auto start the vdiff after creating it.
+ // The default is true if no value is specified.
+ optional bool auto_start = 22;
}
message VDiffCreateResponse {
@@ -1884,6 +2016,7 @@ message VDiffResumeRequest {
string workflow = 1;
string target_keyspace = 2;
string uuid = 3;
+ repeated string target_shards = 4;
}
message VDiffResumeResponse {
@@ -1905,6 +2038,7 @@ message VDiffStopRequest {
string workflow = 1;
string target_keyspace = 2;
string uuid = 3;
+ repeated string target_shards = 4;
}
message VDiffStopResponse {
@@ -1916,6 +2050,10 @@ message WorkflowDeleteRequest {
bool keep_data = 3;
bool keep_routing_rules = 4;
repeated string shards = 5;
+ // The max records to delete from the moved tables when cleaning
+ // up the migrated data. This is only used with multi-tenant
+ // MoveTables migrations.
+ int64 delete_batch_size = 6;
}
message WorkflowDeleteResponse {
@@ -1972,6 +2110,7 @@ message WorkflowSwitchTrafficRequest {
bool dry_run = 9;
bool initialize_target_sequences = 10;
repeated string shards = 11;
+ bool force = 12;
}
message WorkflowSwitchTrafficResponse {
diff --git a/proto/vtctlservice.proto b/proto/vtctlservice.proto
index 672374038b5..3133d4fb4e1 100644
--- a/proto/vtctlservice.proto
+++ b/proto/vtctlservice.proto
@@ -58,6 +58,8 @@ service Vtctld {
rpc BackupShard(vtctldata.BackupShardRequest) returns (stream vtctldata.BackupResponse) {};
// CancelSchemaMigration cancels one or all migrations, terminating any running ones as needed.
rpc CancelSchemaMigration(vtctldata.CancelSchemaMigrationRequest) returns (vtctldata.CancelSchemaMigrationResponse) {};
+ // ChangeTabletTags changes the tags of the specified tablet, if possible.
+ rpc ChangeTabletTags(vtctldata.ChangeTabletTagsRequest) returns (vtctldata.ChangeTabletTagsResponse) {};
// ChangeTabletType changes the db type for the specified tablet, if possible.
// This is used primarily to arrange replicas, and it will not convert a
// primary. For that, use InitShardPrimary.
@@ -70,6 +72,8 @@ service Vtctld {
rpc CleanupSchemaMigration(vtctldata.CleanupSchemaMigrationRequest) returns (vtctldata.CleanupSchemaMigrationResponse) {};
// CompleteSchemaMigration completes one or all migrations executed with --postpone-completion.
rpc CompleteSchemaMigration(vtctldata.CompleteSchemaMigrationRequest) returns (vtctldata.CompleteSchemaMigrationResponse) {};
+ // CompleteSchemaMigration completes one or all migrations executed with --postpone-completion.
+ rpc ConcludeTransaction(vtctldata.ConcludeTransactionRequest) returns (vtctldata.ConcludeTransactionResponse) {};
// CreateKeyspace creates the specified keyspace in the topology. For a
// SNAPSHOT keyspace, the request must specify the name of a base keyspace,
// as well as a snapshot time.
@@ -169,6 +173,10 @@ service Vtctld {
rpc GetThrottlerStatus(vtctldata.GetThrottlerStatusRequest) returns (vtctldata.GetThrottlerStatusResponse) {};
// GetTopologyPath returns the topology cell at a given path.
rpc GetTopologyPath(vtctldata.GetTopologyPathRequest) returns (vtctldata.GetTopologyPathResponse) {};
+ // GetTransactionInfo reads a given transactions information.
+ rpc GetTransactionInfo(vtctldata.GetTransactionInfoRequest) returns (vtctldata.GetTransactionInfoResponse) {};
+ // GetTransactions returns the unresolved transactions for the request.
+ rpc GetUnresolvedTransactions(vtctldata.GetUnresolvedTransactionsRequest) returns (vtctldata.GetUnresolvedTransactionsResponse) {};
// GetVersion returns the version of a tablet from its debug vars.
rpc GetVersion(vtctldata.GetVersionRequest) returns (vtctldata.GetVersionResponse) {};
// GetVSchema returns the vschema for a keyspace.
diff --git a/proto/vtgate.proto b/proto/vtgate.proto
index 0d8781bcd61..aadf211f0a2 100644
--- a/proto/vtgate.proto
+++ b/proto/vtgate.proto
@@ -76,6 +76,8 @@ message Session {
// reserved connection if a dedicated connection is needed
int64 reserved_id = 4;
bool vindex_only = 5;
+ // rows_affected tracks if any query has modified the rows.
+ bool rows_affected = 6;
}
// shard_sessions keep track of per-shard transaction info.
repeated ShardSession shard_sessions = 2;
@@ -302,6 +304,10 @@ message VStreamFlags {
string cells = 4;
string cell_preference = 5;
string tablet_order = 6;
+ // When set, all new row events from the `heartbeat` table, for all shards, in the sidecardb will be streamed.
+ bool stream_keyspace_heartbeats = 7;
+ // Include reshard journal events in the stream.
+ bool include_reshard_journal_events = 8;
}
// VStreamRequest is the payload for VStream.
diff --git a/test.go b/test.go
index 360b231e889..95d62af892f 100755
--- a/test.go
+++ b/test.go
@@ -77,7 +77,7 @@ For example:
// Flags
var (
flavor = flag.String("flavor", "mysql80", "comma-separated bootstrap flavor(s) to run against (when using Docker mode). Available flavors: all,"+flavors)
- bootstrapVersion = flag.String("bootstrap-version", "34", "the version identifier to use for the docker images")
+ bootstrapVersion = flag.String("bootstrap-version", "39", "the version identifier to use for the docker images")
runCount = flag.Int("runs", 1, "run each test this many times")
retryMax = flag.Int("retry", 3, "max number of retries, to detect flaky tests")
logPass = flag.Bool("log-pass", false, "log test output even if it passes")
@@ -112,7 +112,7 @@ const (
configFileName = "test/config.json"
// List of flavors for which a bootstrap Docker image is available.
- flavors = "mysql57,mysql80,percona,percona57,percona80"
+ flavors = "mysql80,percona80"
)
// Config is the overall object serialized in test/config.json.
diff --git a/test/ci_workflow_gen.go b/test/ci_workflow_gen.go
index 09e2fbb6900..b24db1154fb 100644
--- a/test/ci_workflow_gen.go
+++ b/test/ci_workflow_gen.go
@@ -75,6 +75,7 @@ var (
"xb_backup",
"backup_pitr",
"backup_pitr_xtrabackup",
+ "backup_pitr_mysqlshell",
"21",
"mysql_server_vault",
"vstream",
@@ -102,6 +103,7 @@ var (
"vtgate_vindex_heavy",
"vtgate_vschema",
"vtgate_queries",
+ "vtgate_plantests",
"vtgate_schema_tracker",
"vtgate_foreignkey_stress",
"vtorc",
@@ -115,7 +117,8 @@ var (
"vreplication_v2",
"vreplication_partial_movetables_and_materialize",
"vreplication_foreign_key_stress",
- "vreplication_migrate_vdiff2_convert_tz",
+ "vreplication_migrate",
+ "vreplication_vtctldclient_vdiff2_movetables_tz",
"vreplication_multi_tenant",
"schemadiff_vrepl",
"topo_connection_cache",
@@ -152,7 +155,11 @@ var (
"onlineddl_vrepl_stress_suite",
"onlineddl_vrepl_suite",
"vreplication_basic",
- "vreplication_migrate_vdiff2_convert_tz",
+ "vreplication_migrate",
+ "vreplication_vtctldclient_vdiff2_movetables_tz",
+ }
+ clusterRequiringMinio = []string{
+ "21",
}
)
@@ -171,6 +178,7 @@ type clusterTest struct {
EnableBinlogTransactionCompression bool
PartialKeyspace bool
Cores16 bool
+ NeedsMinio bool
}
type vitessTesterTest struct {
@@ -283,6 +291,13 @@ func generateClusterWorkflows(list []string, tpl string) {
break
}
}
+ minioClusters := canonnizeList(clusterRequiringMinio)
+ for _, minioCluster := range minioClusters {
+ if minioCluster == cluster {
+ test.NeedsMinio = true
+ break
+ }
+ }
if mysqlVersion == mysql57 {
test.Platform = string(mysql57)
}
diff --git a/test/config.json b/test/config.json
index 49f77e1b7fb..da0026f0125 100644
--- a/test/config.json
+++ b/test/config.json
@@ -109,6 +109,15 @@
"RetryMax": 1,
"Tags": []
},
+ "backup_pitr_mysqlshell": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/backup/pitr_mysqlshell", "-timeout", "30m"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "backup_pitr_mysqlshell",
+ "RetryMax": 1,
+ "Tags": []
+ },
"backup": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/backup/vtctlbackup", "-timeout", "30m"],
@@ -127,6 +136,15 @@
"RetryMax": 1,
"Tags": []
},
+ "backup_s3": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/backup/s3", "-timeout", "30m"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "21",
+ "RetryMax": 1,
+ "Tags": []
+ },
"backup_only": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/backup/vtbackup", "-timeout", "20m"],
@@ -331,17 +349,6 @@
"RetryMax": 1,
"Tags": []
},
- "pitr": {
- "File": "unused.go",
- "Args": ["vitess.io/vitess/go/test/endtoend/recovery/pitr"],
- "Command": [],
- "Manual": false,
- "Shard": "10",
- "RetryMax": 1,
- "Tags": [
- "site_test"
- ]
- },
"recovery": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/recovery/unshardedrecovery"],
@@ -842,9 +849,27 @@
"RetryMax": 1,
"Tags": []
},
- "vtgate_transaction_twopc_fuzzer": {
+ "vtgate_transaction_twopc_metric": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/transaction/twopc/metric"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "vtgate_transaction",
+ "RetryMax": 1,
+ "Tags": []
+ },
+ "vtgate_transaction_twopc_stress": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/transaction/twopc/stress"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "vtgate_transaction",
+ "RetryMax": 1,
+ "Tags": []
+ },
+ "vtgate_transaction_twopc_fuzz": {
"File": "unused.go",
- "Args": ["vitess.io/vitess/go/test/endtoend/transaction/twopc/fuzzer"],
+ "Args": ["vitess.io/vitess/go/test/endtoend/transaction/twopc/fuzz"],
"Command": [],
"Manual": false,
"Shard": "vtgate_transaction",
@@ -860,6 +885,15 @@
"RetryMax": 1,
"Tags": []
},
+ "vtgate_plantests": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/vtgate/plan_tests"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "vtgate_plantests",
+ "RetryMax": 1,
+ "Tags": []
+ },
"vtgate_unsharded": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/vtgate/unsharded"],
@@ -1042,7 +1076,7 @@
},
"vreplication_materialize": {
"File": "unused.go",
- "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestMaterialize"],
+ "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "Materialize"],
"Command": [],
"Manual": false,
"Shard": "vreplication_partial_movetables_and_materialize",
@@ -1211,6 +1245,17 @@
"RetryMax": 1,
"Tags": []
},
+ "vtop_example": {
+ "File": "",
+ "Args": [],
+ "Command": [
+ "test/vtop_example.sh"
+ ],
+ "Manual": false,
+ "Shard": "",
+ "RetryMax": 1,
+ "Tags": []
+ },
"vtorc_primary_failure": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/vtorc/primaryfailure"],
@@ -1301,21 +1346,12 @@
"RetryMax": 1,
"Tags": []
},
- "vreplication_vtctldclient_cli": {
- "File": "unused.go",
- "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestVtctldclientCLI", "-timeout", "20m"],
- "Command": [],
- "Manual": false,
- "Shard": "vreplication_migrate_vdiff2_convert_tz",
- "RetryMax": 1,
- "Tags": []
- },
"vreplication_vtctl_migrate": {
"File": "unused.go",
"Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestVtctlMigrate", "-timeout", "30m"],
"Command": [],
"Manual": false,
- "Shard": "vreplication_migrate_vdiff2_convert_tz",
+ "Shard": "vreplication_migrate",
"RetryMax": 1,
"Tags": []
},
@@ -1324,7 +1360,7 @@
"Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestVtctldMigrate", "-timeout", "30m"],
"Command": [],
"Manual": false,
- "Shard": "vreplication_migrate_vdiff2_convert_tz",
+ "Shard": "vreplication_migrate",
"RetryMax": 1,
"Tags": []
},
@@ -1333,7 +1369,16 @@
"Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestVDiff2", "-timeout", "30m"],
"Command": [],
"Manual": false,
- "Shard": "vreplication_migrate_vdiff2_convert_tz",
+ "Shard": "vreplication_vtctldclient_vdiff2_movetables_tz",
+ "RetryMax": 1,
+ "Tags": []
+ },
+ "vreplication_vtctldclient_cli": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestVtctldclientCLI", "-timeout", "20m"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "vreplication_vtctldclient_vdiff2_movetables_tz",
"RetryMax": 1,
"Tags": []
},
@@ -1342,7 +1387,25 @@
"Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestMoveTablesTZ"],
"Command": [],
"Manual": false,
- "Shard": "vreplication_migrate_vdiff2_convert_tz",
+ "Shard": "vreplication_vtctldclient_vdiff2_movetables_tz",
+ "RetryMax": 1,
+ "Tags": []
+ },
+ "loopkup_index": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/vreplication", "-run", "TestLookupIndex"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "vreplication_vtctldclient_vdiff2_movetables_tz",
+ "RetryMax": 1,
+ "Tags": []
+ },
+ "vtadmin": {
+ "File": "unused.go",
+ "Args": ["vitess.io/vitess/go/test/endtoend/vtadmin"],
+ "Command": [],
+ "Manual": false,
+ "Shard": "15",
"RetryMax": 1,
"Tags": []
},
diff --git a/test/templates/cluster_endtoend_test.tpl b/test/templates/cluster_endtoend_test.tpl
index 332cb67fedc..8d0a2f650b5 100644
--- a/test/templates/cluster_endtoend_test.tpl
+++ b/test/templates/cluster_endtoend_test.tpl
@@ -14,7 +14,7 @@ env:
jobs:
build:
name: Run endtoend tests on {{.Name}}
- runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1{{else}}gh-hosted-runners-4cores-1{{end}}
+ runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1-24.04{{else}}ubuntu-24.04{{end}}
steps:
- name: Skip CI
@@ -56,11 +56,11 @@ jobs:
- name: Check out code
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: actions/checkout@v4
+ uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7
- name: Check for changes in relevant files
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: dorny/paths-filter@v3.0.1
+ uses: dorny/paths-filter@ebc4d7e9ebcb0b1eb21480bb8f43113e996ac77a # v3.0.1
id: changes
with:
token: ''
@@ -85,13 +85,13 @@ jobs:
- name: Set up Go
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-go@v5
+ uses: actions/setup-go@0a12ed9d6a96ab950c8f026ed9f722fe0da7ef32 # v5.0.2
with:
- go-version: 1.22.5
+ go-version-file: go.mod
- name: Set up python
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-python@v5
+ uses: actions/setup-python@39cd14951b08e74b54015e9e001cdefcf80e669f # v5.1.1
- name: Tune the OS
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
@@ -117,19 +117,28 @@ jobs:
sudo apt-get -qq update
# Install everything else we need, and configure
- sudo apt-get -qq install -y percona-server-server percona-server-client make unzip g++ etcd git wget eatmydata xz-utils libncurses5
+ sudo apt-get -qq install -y percona-server-server percona-server-client make unzip g++ etcd-client etcd-server git wget eatmydata xz-utils libncurses6
{{else}}
# Get key to latest MySQL repo
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys A8D3785C
# Setup MySQL 8.0
- wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.32-1_all.deb
+ wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.33-1_all.deb
echo mysql-apt-config mysql-apt-config/select-server select mysql-8.0 | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
sudo apt-get -qq update
+
+ # We have to install this old version of libaio1 in case we end up testing with MySQL 5.7. See also:
+ # https://bugs.launchpad.net/ubuntu/+source/libaio/+bug/2067501
+ curl -L -O http://mirrors.kernel.org/ubuntu/pool/main/liba/libaio/libaio1_0.3.112-13build1_amd64.deb
+ sudo dpkg -i libaio1_0.3.112-13build1_amd64.deb
+ # libtinfo5 is also needed for older MySQL 5.7 builds.
+ curl -L -O http://mirrors.kernel.org/ubuntu/pool/universe/n/ncurses/libtinfo5_6.3-2ubuntu0.1_amd64.deb
+ sudo dpkg -i libtinfo5_6.3-2ubuntu0.1_amd64.deb
+
# Install everything else we need, and configure
- sudo apt-get -qq install -y mysql-server mysql-client make unzip g++ etcd curl git wget eatmydata xz-utils libncurses5
+ sudo apt-get -qq install -y mysql-server mysql-shell mysql-client make unzip g++ etcd-client etcd-server curl git wget eatmydata xz-utils libncurses6
{{end}}
@@ -148,6 +157,15 @@ jobs:
{{end}}
+ {{if .NeedsMinio }}
+ - name: Install Minio
+ if: steps.skip-workflow.outputs.skip-workflow == 'false'
+ run: |
+ wget https://dl.min.io/server/minio/release/linux-amd64/minio
+ chmod +x minio
+ mv minio /usr/local/bin
+ {{end}}
+
{{if .MakeTools}}
- name: Installing zookeeper and consul
@@ -222,7 +240,7 @@ jobs:
- name: Test Summary
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && always()
- uses: test-summary/action@v2
+ uses: test-summary/action@31493c76ec9e7aa675f1585d3ed6f1da69269a86 # v2.4
with:
paths: "report.xml"
- show: "fail, skip"
+ show: "fail"
diff --git a/test/templates/cluster_endtoend_test_docker.tpl b/test/templates/cluster_endtoend_test_docker.tpl
index 2b63e6d3516..f7e8aa2c1d8 100644
--- a/test/templates/cluster_endtoend_test_docker.tpl
+++ b/test/templates/cluster_endtoend_test_docker.tpl
@@ -6,7 +6,7 @@ permissions: read-all
jobs:
build:
name: Run endtoend tests on {{.Name}}
- runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1{{else}}gh-hosted-runners-4cores-1{{end}}
+ runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1-24.04{{else}}ubuntu-24.04{{end}}
steps:
- name: Skip CI
@@ -28,11 +28,11 @@ jobs:
- name: Check out code
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: actions/checkout@v4
+ uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7
- name: Check for changes in relevant files
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: dorny/paths-filter@v3.0.1
+ uses: dorny/paths-filter@ebc4d7e9ebcb0b1eb21480bb8f43113e996ac77a # v3.0.1
id: changes
with:
token: ''
@@ -54,9 +54,9 @@ jobs:
- name: Set up Go
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-go@v5
+ uses: actions/setup-go@0a12ed9d6a96ab950c8f026ed9f722fe0da7ef32 # v5.0.2
with:
- go-version: 1.22.5
+ go-version-file: go.mod
- name: Tune the OS
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
diff --git a/test/templates/cluster_endtoend_test_mysql57.tpl b/test/templates/cluster_endtoend_test_mysql57.tpl
index a29698da472..f4152c939b0 100644
--- a/test/templates/cluster_endtoend_test_mysql57.tpl
+++ b/test/templates/cluster_endtoend_test_mysql57.tpl
@@ -19,7 +19,7 @@ env:
jobs:
build:
name: Run endtoend tests on {{.Name}}
- runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1{{else}}gh-hosted-runners-4cores-1{{end}}
+ runs-on: {{if .Cores16}}gh-hosted-runners-16cores-1-24.04{{else}}ubuntu-24.04{{end}}
steps:
- name: Skip CI
@@ -61,11 +61,11 @@ jobs:
- name: Check out code
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: actions/checkout@v4
+ uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7
- name: Check for changes in relevant files
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: dorny/paths-filter@v3.0.1
+ uses: dorny/paths-filter@ebc4d7e9ebcb0b1eb21480bb8f43113e996ac77a # v3.0.1
id: changes
with:
token: ''
@@ -90,13 +90,13 @@ jobs:
- name: Set up Go
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-go@v5
+ uses: actions/setup-go@0a12ed9d6a96ab950c8f026ed9f722fe0da7ef32 # v5.0.2
with:
- go-version: 1.22.5
+ go-version-file: go.mod
- name: Set up python
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-python@v5
+ uses: actions/setup-python@39cd14951b08e74b54015e9e001cdefcf80e669f # v5.1.1
- name: Tune the OS
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
@@ -126,16 +126,20 @@ jobs:
# Get key to latest MySQL repo
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys A8D3785C
- wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.32-1_all.deb
+ wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.33-1_all.deb
# Bionic packages are still compatible for Jammy since there's no MySQL 5.7
# packages for Jammy.
echo mysql-apt-config mysql-apt-config/repo-codename select bionic | sudo debconf-set-selections
echo mysql-apt-config mysql-apt-config/select-server select mysql-5.7 | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
sudo apt-get update
- sudo DEBIAN_FRONTEND="noninteractive" apt-get install -y mysql-client=5.7* mysql-community-server=5.7* mysql-server=5.7* libncurses5
+ # We have to install this old version of libaio1. See also:
+ # https://bugs.launchpad.net/ubuntu/+source/libaio/+bug/2067501
+ curl -L -O http://mirrors.kernel.org/ubuntu/pool/main/liba/libaio/libaio1_0.3.112-13build1_amd64.deb
+ sudo dpkg -i libaio1_0.3.112-13build1_amd64.deb
+ sudo DEBIAN_FRONTEND="noninteractive" apt-get install -y mysql-client=5.7* mysql-community-server=5.7* mysql-server=5.7* libncurses6
- sudo apt-get install -y make unzip g++ etcd curl git wget eatmydata
+ sudo apt-get install -y make unzip g++ etcd-client etcd-server curl git wget eatmydata
sudo service mysql stop
sudo service etcd stop
@@ -228,7 +232,7 @@ jobs:
- name: Test Summary
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && always()
- uses: test-summary/action@v2
+ uses: test-summary/action@31493c76ec9e7aa675f1585d3ed6f1da69269a86 # v2.4
with:
paths: "report.xml"
- show: "fail, skip"
+ show: "fail"
diff --git a/test/templates/cluster_vitess_tester.tpl b/test/templates/cluster_vitess_tester.tpl
index 541bfd5c6a0..b8d77754ba6 100644
--- a/test/templates/cluster_vitess_tester.tpl
+++ b/test/templates/cluster_vitess_tester.tpl
@@ -14,7 +14,7 @@ env:
jobs:
build:
name: Run endtoend tests on {{.Name}}
- runs-on: gh-hosted-runners-4cores-1
+ runs-on: ubuntu-24.04
steps:
- name: Skip CI
@@ -43,11 +43,11 @@ jobs:
- name: Check out code
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: actions/checkout@v4
+ uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7
- name: Check for changes in relevant files
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: dorny/paths-filter@v3.0.1
+ uses: dorny/paths-filter@ebc4d7e9ebcb0b1eb21480bb8f43113e996ac77a # v3.0.1
id: changes
with:
token: ''
@@ -69,13 +69,13 @@ jobs:
- name: Set up Go
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-go@v5
+ uses: actions/setup-go@0a12ed9d6a96ab950c8f026ed9f722fe0da7ef32 # v5.0.2
with:
- go-version: 1.22.5
+ go-version-file: go.mod
- name: Set up python
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
- uses: actions/setup-python@v5
+ uses: actions/setup-python@39cd14951b08e74b54015e9e001cdefcf80e669f # v5.1.1
- name: Tune the OS
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true'
@@ -93,12 +93,12 @@ jobs:
# Get key to latest MySQL repo
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys A8D3785C
# Setup MySQL 8.0
- wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.32-1_all.deb
+ wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.33-1_all.deb
echo mysql-apt-config mysql-apt-config/select-server select mysql-8.0 | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
sudo apt-get -qq update
# Install everything else we need, and configure
- sudo apt-get -qq install -y mysql-server mysql-client make unzip g++ etcd curl git wget eatmydata xz-utils libncurses5
+ sudo apt-get -qq install -y mysql-server mysql-client make unzip g++ etcd-client etcd-server curl git wget eatmydata xz-utils libncurses6
sudo service mysql stop
sudo service etcd stop
@@ -110,7 +110,7 @@ jobs:
go install github.com/vitessio/go-junit-report@HEAD
# install vitess tester
- go install github.com/vitessio/vitess-tester@89dd933a9ea0e15f69ca58b9c8ea09a358762cca
+ go install github.com/vitessio/vt/go/vt@e43009309f599378504905d4b804460f47822ac5
- name: Setup launchable dependencies
if: steps.skip-workflow.outputs.is_draft == 'false' && steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && github.base_ref == 'main'
@@ -140,11 +140,11 @@ jobs:
i=1
for dir in {{.Path}}/*/; do
# We go over all the directories in the given path.
- # If there is a vschema file there, we use it, otherwise we let vitess-tester autogenerate it.
+ # If there is a vschema file there, we use it, otherwise we let vt tester autogenerate it.
if [ -f $dir/vschema.json ]; then
- vitess-tester --xunit --vschema "$dir"vschema.json $dir/*.test
+ vt tester --xunit --vschema "$dir"vschema.json $dir/*.test
else
- vitess-tester --sharded --xunit $dir/*.test
+ vt tester --sharded --xunit $dir/*.test
fi
# Number the reports by changing their file names.
mv report.xml report"$i".xml
@@ -164,7 +164,7 @@ jobs:
- name: Test Summary
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' && always()
- uses: test-summary/action@v2
+ uses: test-summary/action@31493c76ec9e7aa675f1585d3ed6f1da69269a86 # v2.4
with:
paths: "report*.xml"
- show: "fail, skip"
+ show: "fail"
diff --git a/test/templates/dockerfile.tpl b/test/templates/dockerfile.tpl
index 437971aa532..af4376d3ca9 100644
--- a/test/templates/dockerfile.tpl
+++ b/test/templates/dockerfile.tpl
@@ -1,4 +1,4 @@
-ARG bootstrap_version=34
+ARG bootstrap_version=39
ARG image="vitess/bootstrap:${bootstrap_version}-{{.Platform}}"
FROM "${image}"
diff --git a/test/templates/unit_test.tpl b/test/templates/unit_test.tpl
index 21707a89e48..3704aebac4e 100644
--- a/test/templates/unit_test.tpl
+++ b/test/templates/unit_test.tpl
@@ -14,7 +14,7 @@ env:
jobs:
test:
name: {{.Name}}
- runs-on: gh-hosted-runners-4cores-1
+ runs-on: ubuntu-24.04
steps:
- name: Skip CI
@@ -43,11 +43,11 @@ jobs:
- name: Check out code
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: actions/checkout@v4
+ uses: actions/checkout@692973e3d937129bcbf40652eb9f2f61becf3332 # v4.1.7
- name: Check for changes in relevant files
if: steps.skip-workflow.outputs.skip-workflow == 'false'
- uses: dorny/paths-filter@v3.0.1
+ uses: dorny/paths-filter@ebc4d7e9ebcb0b1eb21480bb8f43113e996ac77a # v3.0.1
id: changes
with:
token: ''
@@ -67,13 +67,13 @@ jobs:
- name: Set up Go
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true'
- uses: actions/setup-go@v5
+ uses: actions/setup-go@0a12ed9d6a96ab950c8f026ed9f722fe0da7ef32 # v5.0.2
with:
- go-version: 1.22.5
+ go-version-file: go.mod
- name: Set up python
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true'
- uses: actions/setup-python@v5
+ uses: actions/setup-python@39cd14951b08e74b54015e9e001cdefcf80e669f # v5.1.1
- name: Tune the OS
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true'
@@ -87,20 +87,20 @@ jobs:
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true'
run: |
export DEBIAN_FRONTEND="noninteractive"
- sudo apt-get -qq update
+ sudo apt-get update
# Uninstall any previously installed MySQL first
sudo systemctl stop apparmor
- sudo DEBIAN_FRONTEND="noninteractive" apt-get -qq remove -y --purge mysql-server mysql-client mysql-common
- sudo apt-get -qq -y autoremove
- sudo apt-get -qq -y autoclean
+ sudo DEBIAN_FRONTEND="noninteractive" apt-get remove -y --purge mysql-server mysql-client mysql-common
+ sudo apt-get -y autoremove
+ sudo apt-get -y autoclean
sudo deluser mysql
sudo rm -rf /var/lib/mysql
sudo rm -rf /etc/mysql
# Get key to latest MySQL repo
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys A8D3785C
- wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.32-1_all.deb
+ wget -c https://dev.mysql.com/get/mysql-apt-config_0.8.33-1_all.deb
{{if (eq .Platform "mysql57")}}
# Bionic packages are still compatible for Jammy since there's no MySQL 5.7
@@ -108,25 +108,32 @@ jobs:
echo mysql-apt-config mysql-apt-config/repo-codename select bionic | sudo debconf-set-selections
echo mysql-apt-config mysql-apt-config/select-server select mysql-5.7 | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
- sudo apt-get -qq update
- sudo DEBIAN_FRONTEND="noninteractive" apt-get -qq install -y mysql-client=5.7* mysql-community-server=5.7* mysql-server=5.7* libncurses5
+ sudo apt-get update
+ # We have to install this old version of libaio1. See also:
+ # https://bugs.launchpad.net/ubuntu/+source/libaio/+bug/2067501
+ curl -L -O http://mirrors.kernel.org/ubuntu/pool/main/liba/libaio/libaio1_0.3.112-13build1_amd64.deb
+ sudo dpkg -i libaio1_0.3.112-13build1_amd64.deb
+ # libtinfo5 is also needed for older MySQL 5.7 builds.
+ curl -L -O http://mirrors.kernel.org/ubuntu/pool/universe/n/ncurses/libtinfo5_6.3-2ubuntu0.1_amd64.deb
+ sudo dpkg -i libtinfo5_6.3-2ubuntu0.1_amd64.deb
+ sudo DEBIAN_FRONTEND="noninteractive" apt-get install -y mysql-client=5.7* mysql-community-server=5.7* mysql-server=5.7* libncurses6
{{end}}
{{if (eq .Platform "mysql80")}}
echo mysql-apt-config mysql-apt-config/select-server select mysql-8.0 | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
- sudo apt-get -qq update
- sudo DEBIAN_FRONTEND="noninteractive" apt-get -qq install -y mysql-server mysql-client
+ sudo apt-get update
+ sudo DEBIAN_FRONTEND="noninteractive" apt-get install -y mysql-server mysql-client
{{end}}
{{if (eq .Platform "mysql84")}}
echo mysql-apt-config mysql-apt-config/select-server select mysql-8.4-lts | sudo debconf-set-selections
sudo DEBIAN_FRONTEND="noninteractive" dpkg -i mysql-apt-config*
- sudo apt-get -qq update
- sudo DEBIAN_FRONTEND="noninteractive" apt-get -qq install -y mysql-server mysql-client
+ sudo apt-get update
+ sudo DEBIAN_FRONTEND="noninteractive" apt-get install -y mysql-server mysql-client
{{end}}
- sudo apt-get -qq install -y make unzip g++ curl git wget ant openjdk-11-jdk eatmydata
+ sudo apt-get install -y make unzip g++ curl git wget ant openjdk-11-jdk eatmydata
sudo service mysql stop
sudo bash -c "echo '/usr/sbin/mysqld { }' > /etc/apparmor.d/usr.sbin.mysqld" # https://bugs.launchpad.net/ubuntu/+source/mariadb-10.1/+bug/1806263
sudo ln -s /etc/apparmor.d/usr.sbin.mysqld /etc/apparmor.d/disable/
@@ -186,7 +193,7 @@ jobs:
- name: Test Summary
if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' && always()
- uses: test-summary/action@v2
+ uses: test-summary/action@31493c76ec9e7aa675f1585d3ed6f1da69269a86 # v2.4
with:
paths: "report.xml"
- show: "fail, skip"
+ show: "fail"
diff --git a/test/vtop_example.sh b/test/vtop_example.sh
index 5ff90a2be7e..c537c0f844c 100755
--- a/test/vtop_example.sh
+++ b/test/vtop_example.sh
@@ -482,11 +482,12 @@ EOF
waitForKeyspaceToBeServing customer 80- 1
}
+kind delete cluster --name kind || true
# Build the docker image for vitess/lite using the local code
docker build -f docker/lite/Dockerfile -t vitess/lite:pr .
# Build the docker image for vitess/vtadmin using the local code
-docker build -f docker/binaries/vtadmin/Dockerfile --build-arg VT_BASE_VER=pr -t vitess/vtadmin:pr .
+docker build -f docker/binaries/vtadmin/Dockerfile --build-arg VT_BASE_VER=pr -t vitess/vtadmin:pr ./docker/binaries/vtadmin
# Print the docker images available
docker image ls
diff --git a/tools/check_go_versions.sh b/tools/check_go_versions.sh
index 3549cdc10e9..846a10322f1 100755
--- a/tools/check_go_versions.sh
+++ b/tools/check_go_versions.sh
@@ -14,25 +14,9 @@ if [ -z "${GO_MOD_VERSION}" ]; then
exit 1
fi
-# ci workflows
-TPL_GO_VERSIONS="$(awk '/go-version: /{print $(NF-0)}' .github/workflows/*.yml test/templates/*.tpl | sort -u)"
-TPL_GO_VERSIONS_COUNT=$(echo "$TPL_GO_VERSIONS" | wc -l | tr -d [:space:])
-if [ "${TPL_GO_VERSIONS_COUNT}" -gt 1 ]; then
- echo -e "expected a consistent 'go-version:' in CI workflow files/templates, found versions:\n${TPL_GO_VERSIONS}"
- exit 1
-fi
-TPL_GO_VERSION="${TPL_GO_VERSIONS}"
-if [[ ! "${TPL_GO_VERSION}" =~ "${GO_MOD_VERSION}" ]]; then
- echo "expected go-version in test/templates/* to be equal to go.mod: '${TPL_GO_VERSION}' != '${GO_MOD_VERSION}'"
- exit 1
-fi
-
# docker/bootstrap/Dockerfile.common
BOOTSTRAP_GO_VERSION="$(awk -F ':' '/golang:/{print $(NF-0)}' docker/bootstrap/Dockerfile.common | cut -d- -f1)"
if [[ ! "${BOOTSTRAP_GO_VERSION}" =~ "${GO_MOD_VERSION}" ]]; then
echo "expected golang docker version in docker/bootstrap/Dockerfile.common to be equal to go.mod: '${TPL_GO_VERSION}' != '${GO_MOD_VERSION}'"
exit 1
-elif [ "${TPL_GO_VERSION}" != "${BOOTSTRAP_GO_VERSION}" ]; then
- echo "expected equal go version in CI workflow files/templates and bootstrap Dockerfile: '${TPL_GO_VERSIONS}' != '${BOOTSTRAP_GO_VERSION}'"
- exit 1
fi
diff --git a/tools/get_kubectl_kind.sh b/tools/get_kubectl_kind.sh
index 57df414fdd8..169b120aaa0 100755
--- a/tools/get_kubectl_kind.sh
+++ b/tools/get_kubectl_kind.sh
@@ -12,7 +12,7 @@ source build.env
mkdir -p "$VTROOT/bin"
cd "$VTROOT/bin"
-KUBE_VERSION="${KUBE_VERSION:-v1.21.1}"
+KUBE_VERSION="${KUBE_VERSION:-v1.31.0}"
KUBERNETES_RELEASE_URL="${KUBERNETES_RELEASE_URL:-https://dl.k8s.io}"
# Download kubectl if needed.
@@ -28,7 +28,7 @@ ln -sf "kubectl-${KUBE_VERSION}" kubectl
if ! command -v kind &> /dev/null
then
echo "Downloading kind..."
- curl -L https://kind.sigs.k8s.io/dl/v0.12.0/kind-linux-amd64 > "kind"
+ curl -L https://kind.sigs.k8s.io/dl/v0.22.0/kind-linux-amd64 > "kind"
chmod +x "kind"
echo "Installed kind"
else
diff --git a/vitess-mixin/e2e/run-forever.sh b/tools/map-shard-for-value/Makefile
old mode 100755
new mode 100644
similarity index 72%
rename from vitess-mixin/e2e/run-forever.sh
rename to tools/map-shard-for-value/Makefile
index 51e511be77a..61bc88ac0ed
--- a/vitess-mixin/e2e/run-forever.sh
+++ b/tools/map-shard-for-value/Makefile
@@ -1,21 +1,22 @@
-#!/bin/bash
-
-# Copyright 2019 The Vitess Authors.
-#
+# 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.
-while true
-do
- $@
- sleep 2
-done
+build:
+ go build map-shard-for-value.go
+
+test:
+ echo "1\n-1\n99" | go run map-shard-for-value.go --total_shards=4 --vindex=xxhash
+
+clean:
+ rm -f map-shard-for-value
diff --git a/tools/map-shard-for-value/map-shard-for-value.go b/tools/map-shard-for-value/map-shard-for-value.go
new file mode 100755
index 00000000000..18a092d1371
--- /dev/null
+++ b/tools/map-shard-for-value/map-shard-for-value.go
@@ -0,0 +1,207 @@
+/*
+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 main
+
+import (
+ "bufio"
+ "context"
+ "encoding/hex"
+ "fmt"
+ "log"
+ "os"
+ "strconv"
+ "strings"
+
+ flag "github.com/spf13/pflag"
+
+ "vitess.io/vitess/go/vt/topo"
+
+ "vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/key"
+ "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/vtgate/vindexes"
+)
+
+/*
+ * This tool reads a list of values from stdin and prints the
+ * corresponding keyspace ID and shard for each value. It uses the given vindex
+ * and shard ranges to determine the shard. The vindex is expected to be a
+ * single-column vindex. The shard ranges are specified as a comma-separated
+ * list of key ranges, example "-80,80-".
+ * If you have uniformly distributed shards, you can specify the total number
+ * of shards using the -total_shards flag, and the tool will generate the shard ranges
+ * using the same logic as the Vitess operator does (using the key.GenerateShardRanges() function).
+ *
+ * Example usage:
+ * echo "1\n2\n3" | go run shard-from-id.go -vindex=hash -shards=-80,80-
+ *
+ * Currently tested only for integer values and hash/xxhash vindexes.
+ */
+
+func mapShard(allShards []*topodata.ShardReference, ksid key.DestinationKeyspaceID) (string, error) {
+ foundShard := ""
+ addShard := func(shard string) error {
+ foundShard = shard
+ return nil
+ }
+ if err := ksid.Resolve(allShards, addShard); err != nil {
+ return "", fmt.Errorf("failed to resolve keyspace ID: %v:: %s", ksid.String(), err)
+ }
+
+ if foundShard == "" {
+ return "", fmt.Errorf("no shard found for keyspace ID: %v", ksid)
+ }
+ return foundShard, nil
+}
+
+func selectShard(vindex vindexes.Vindex, value sqltypes.Value, allShards []*topodata.ShardReference) (string, key.DestinationKeyspaceID, error) {
+ ctx := context.Background()
+
+ destinations, err := vindexes.Map(ctx, vindex, nil, [][]sqltypes.Value{{value}})
+ if err != nil {
+ return "", nil, fmt.Errorf("failed to map value to keyspace ID: %w", err)
+ }
+
+ if len(destinations) != 1 {
+ return "", nil, fmt.Errorf("unexpected number of destinations: %d", len(destinations))
+ }
+
+ ksid, ok := destinations[0].(key.DestinationKeyspaceID)
+ if !ok {
+ return "", nil, fmt.Errorf("unexpected destination type: %T", destinations[0])
+ }
+
+ foundShard, err := mapShard(allShards, ksid)
+ if err != nil {
+ return "", nil, fmt.Errorf("failed to map shard, original value %v, keyspace id %s: %w", value, ksid, err)
+ }
+ return foundShard, ksid, nil
+}
+
+func getValue(valueStr, valueType string) (sqltypes.Value, error) {
+ var value sqltypes.Value
+
+ switch valueType {
+ case "int":
+ valueInt, err := strconv.ParseInt(valueStr, 10, 64)
+ if err != nil {
+ return value, fmt.Errorf("failed to parse int value: %w", err)
+ }
+ value = sqltypes.NewInt64(valueInt)
+ case "uint":
+ valueUint, err := strconv.ParseUint(valueStr, 10, 64)
+ if err != nil {
+ return value, fmt.Errorf("failed to parse uint value: %w", err)
+ }
+ value = sqltypes.NewUint64(valueUint)
+ case "string":
+ value = sqltypes.NewVarChar(valueStr)
+ default:
+ return value, fmt.Errorf("unsupported value type: %s", valueType)
+ }
+
+ return value, nil
+}
+
+func getShardMap(shardsCSV *string) []*topodata.ShardReference {
+ var allShards []*topodata.ShardReference
+
+ for _, shard := range strings.Split(*shardsCSV, ",") {
+ _, keyRange, err := topo.ValidateShardName(shard)
+ if err != nil {
+ log.Fatalf("invalid shard range: %s", shard)
+ }
+ allShards = append(allShards, &topodata.ShardReference{
+ Name: shard,
+ KeyRange: keyRange,
+ })
+ }
+ return allShards
+}
+
+type output struct {
+ Value string
+ KeyspaceID string
+ Shard string
+}
+
+func processValues(scanner *bufio.Scanner, shardsCSV *string, vindexName string, valueType string) ([]output, error) {
+ allShards := getShardMap(shardsCSV)
+
+ vindex, err := vindexes.CreateVindex(vindexName, vindexName, nil)
+ if err != nil {
+ return nil, fmt.Errorf("failed to create vindex: %v", err)
+ }
+ var outputs []output
+ for scanner.Scan() {
+ valueStr := scanner.Text()
+ if valueStr == "" {
+ continue
+ }
+ value, err := getValue(valueStr, valueType)
+ if err != nil {
+ return nil, fmt.Errorf("failed to get value for: %v, value_type %s:: %v", valueStr, valueType, err)
+ }
+ shard, ksid, err := selectShard(vindex, value, allShards)
+ if err != nil {
+ // ignore errors so that we can go ahead with the computation for other values
+ continue
+ }
+ outputs = append(outputs, output{Value: valueStr, KeyspaceID: hex.EncodeToString(ksid), Shard: shard})
+ }
+ return outputs, nil
+}
+
+func printOutput(outputs []output) {
+ fmt.Println("value,keyspaceID,shard")
+ for _, output := range outputs {
+ fmt.Printf("%s,%s,%s\n", output.Value, output.KeyspaceID, output.Shard)
+ }
+}
+
+func main() {
+ // Explicitly configuring the logger since it was flaky in displaying logs locally without this.
+ log.SetOutput(os.Stderr)
+ log.SetFlags(log.LstdFlags)
+ log.SetPrefix("LOG: ")
+
+ vindexName := flag.String("vindex", "xxhash", "name of the vindex")
+ shardsCSV := flag.String("shards", "", "comma-separated list of shard ranges")
+ totalShards := flag.Int("total_shards", 0, "total number of uniformly distributed shards")
+ valueType := flag.String("value_type", "int", "type of the value (int, uint, or string)")
+ flag.Parse()
+
+ if *totalShards > 0 {
+ if *shardsCSV != "" {
+ log.Fatalf("cannot specify both total_shards and shards")
+ }
+ shardArr, err := key.GenerateShardRanges(*totalShards)
+ if err != nil {
+ log.Fatalf("failed to generate shard ranges: %v", err)
+ }
+ *shardsCSV = strings.Join(shardArr, ",")
+ }
+ if *shardsCSV == "" {
+ log.Fatal("shards or total_shards must be specified")
+ }
+ scanner := bufio.NewScanner(os.Stdin)
+ outputs, err := processValues(scanner, shardsCSV, *vindexName, *valueType)
+ if err != nil {
+ log.Fatalf("failed to process values: %v", err)
+ }
+ printOutput(outputs)
+}
diff --git a/tools/map-shard-for-value/map-shard-for-value.md b/tools/map-shard-for-value/map-shard-for-value.md
new file mode 100644
index 00000000000..17daf7f5fe5
--- /dev/null
+++ b/tools/map-shard-for-value/map-shard-for-value.md
@@ -0,0 +1,47 @@
+## Map Shard for Value Tool
+
+### Overview
+
+The `map-shard-for-value` tool maps a given value to a specific shard. This tool helps in determining
+which shard a particular value belongs to, based on the vindex algorithm and shard ranges.
+
+### Features
+-
+
+- Allows specifying the vindex type (e.g., `hash`, `xxhash`).
+- Allows specifying the shard list of (for uniformly distributed shard ranges) the total number of shards to generate.
+- Designed as a _filter_: Reads input values from `stdin` and outputs the corresponding shard information, so it can be
+ used to map values from a file or another program.
+
+### Usage
+
+```sh
+make build
+```
+
+```sh
+echo "1\n-1\n99" | ./map-shard-for-value --total_shards=4 --vindex=xxhash
+value,keyspaceID,shard
+1,d46405367612b4b7,c0-
+-1,d8e2a6a7c8c7623d,c0-
+99,200533312244abca,-40
+
+echo "1\n-1\n99" | ./map-shard-for-value --vindex=hash --shards="-80,80-"
+value,keyspaceID,shard
+1,166b40b44aba4bd6,-80
+-1,355550b2150e2451,-80
+99,2c40ad56f4593c47,-80
+```
+
+#### Flags
+
+- `--vindex`: Specifies the name of the vindex to use (e.g., `hash`, `xxhash`) (default `xxhash`)
+
+One (and only one) of these is required:
+
+- `--shards`: Comma-separated list of shard ranges
+- `--total_shards`: Total number of shards, only if shards are uniformly distributed
+
+Optional:
+- `--value_type`: Type of the value to map, one of int, uint, string (default `int`)
+
diff --git a/tools/map-shard-for-value/map-shard-for-value_test.go b/tools/map-shard-for-value/map-shard-for-value_test.go
new file mode 100644
index 00000000000..ca014818bb9
--- /dev/null
+++ b/tools/map-shard-for-value/map-shard-for-value_test.go
@@ -0,0 +1,90 @@
+/*
+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 main
+
+import (
+ "bufio"
+ "fmt"
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+)
+
+func TestProcess(t *testing.T) {
+ type testCase struct {
+ name string
+ shardsCSV string
+ vindexType string
+ values []int
+ valueType string
+ expected []output
+ }
+ testCases := []testCase{
+ {
+ name: "hash,2 shards",
+ shardsCSV: "-80,80-",
+ vindexType: "hash",
+ values: []int{1, 99},
+ valueType: "int",
+ expected: []output{
+ {
+ Value: "1",
+ KeyspaceID: "166b40b44aba4bd6",
+ Shard: "-80",
+ },
+ {
+ Value: "99",
+ KeyspaceID: "2c40ad56f4593c47",
+ Shard: "-80",
+ },
+ },
+ },
+ {
+ name: "xxhash,4 shards",
+ shardsCSV: "-40,40-80,80-c0,c0-",
+ vindexType: "xxhash",
+ values: []int{1, 99},
+ valueType: "int",
+ expected: []output{
+ {
+ Value: "1",
+ KeyspaceID: "d46405367612b4b7",
+ Shard: "c0-",
+ },
+ {
+ Value: "99",
+ KeyspaceID: "200533312244abca",
+ Shard: "-40",
+ },
+ },
+ },
+ }
+ for _, tc := range testCases {
+ t.Run(tc.name, func(t *testing.T) {
+ var input strings.Builder
+ for _, num := range tc.values {
+ fmt.Fprintf(&input, "%d\n", num)
+ }
+ reader := strings.NewReader(input.String())
+ scanner := bufio.NewScanner(reader)
+ got, err := processValues(scanner, &tc.shardsCSV, tc.vindexType, tc.valueType)
+ require.NoError(t, err)
+ require.EqualValues(t, tc.expected, got)
+ })
+ }
+}
diff --git a/vitess-mixin/.env b/vitess-mixin/.env
deleted file mode 100644
index 2dd23ee5572..00000000000
--- a/vitess-mixin/.env
+++ /dev/null
@@ -1,15 +0,0 @@
-TOPOLOGY_FLAGS=--topo_implementation consul --topo_global_server_address consul1:8500 --topo_global_root vitess/global
-GRPC_PORT=15999
-WEB_PORT=8080
-MYSQL_PORT=15306
-
-CELL=local
-KEYSPACE=commerce
-DB=commerce
-
-EXTERNAL_DB=0
-DB_HOST=external_db_host
-DB_PORT=3306
-DB_USER=external_db_user
-DB_PASS=external_db_password
-DB_CHARSET=CHARACTER SET utf8 COLLATE utf8_general_ci
\ No newline at end of file
diff --git a/vitess-mixin/.gitignore b/vitess-mixin/.gitignore
deleted file mode 100644
index eb410a1f9e8..00000000000
--- a/vitess-mixin/.gitignore
+++ /dev/null
@@ -1,8 +0,0 @@
-.DS_Store
-*.yaml
-dashboards_out
-prometheus_alerts.yaml
-prometheus_rules.yaml
-jsonnetfile.lock.json
-vendor
-.vscode/
diff --git a/vitess-mixin/Makefile b/vitess-mixin/Makefile
deleted file mode 100644
index 267bc569312..00000000000
--- a/vitess-mixin/Makefile
+++ /dev/null
@@ -1,91 +0,0 @@
-.PHONY: dashboards_out prometheus_alerts.yaml prometheus_rules.yaml test tools all e2e e2e-dev e2e-compose-up e2e-compose-down lint fmt
-
-help: #: Show this message.
- @echo "\nAvailable Targets:\n"
- @sed -ne '/@sed/!s/#: //p' $(MAKEFILE_LIST)
-
-all: #: format all .jsonnet/.libsonnet files, generate all dashboards json, alerts and rules yaml
- @make tools
- @make clean > /dev/null
- @make fmt > /dev/null
- @make test
- @make dashboards_out > /dev/null
- @make prometheus_rules.yaml > /dev/null
-
-fmt: #: Usage make fmt
- @echo "# Formatting all .libsonnet and .jsonnet files...\n"
- @scripts/fmt.sh
- @echo "\nDone!\n"
-
-a.yaml: #: Build prometheus alerts
- @echo "# Building 'prometheus_alerts.yaml'...\n"
- @jsonnet -S lib/alerts.jsonnet > $@
- @echo "\nDone!\n"
-
-prometheus_rules.yaml: #: Build prometheus rules
- @echo "# Building 'prometheus_rules.yaml'...\n"
- @jsonnet -S lib/rules.jsonnet > $@
- @echo "\nDone!\n"
-
-dashboards_out: #: Generate Grafana Dashboards Usage: `ENV='prod' make dashboards_out`
- @echo "# Building Grafana dashboards...\n"
- @make clean > /dev/null
- @mkdir -p dashboards_out
- @[ "${ENV}" = 'dev' ] || [ "${ENV}" = 'prod' ] || ( echo -e "##ERROR\nPlease specify ENV (prod or dev)"; exit 1)
- @jsonnet -J vendor --ext-str env="${ENV}" -m dashboards_out lib/dashboards.jsonnet
- @echo "\nDone!\n"
-
-lint: #: Usage: make lint
- @echo "# Linting all .libsonnet and .jsonnet files...\n"
- @scripts/fmt.sh --check
- @echo "\nDone!\n"
- @echo "# Linting 'prometheus_rules.yaml'...\n"
- @make prometheus_rules.yaml > /dev/null
- @promtool check rules prometheus_rules.yaml
- @echo "Done!\n"
-
-clean: #: Delete generated dashboards (/dashboards_out) Usage: make clean
- @echo "# Cleaning up all generated files...\n"
- @rm -rf dashboards_out prometheus_alerts.yaml prometheus_rules.yaml > /dev/null
- @echo "\nDone!\n"
-
-test: #: Compare your .json generated dashboards local version with the origin/main version.
- @make clean > /dev/null
- @ENV=${ENV} scripts/vitess-mixin-plan.sh
- @make clean > /dev/null
-
-tools: tools.go
- @# -mod='' tells go to ignore the vendor/ directory
- @cat $^ | grep _ | awk -F'"' '{print $$2}' | xargs -I% go install -mod='' %
- @jb install
- @jb update https://github.com/grafana/grafonnet-lib/grafonnet > /dev/null 2>&1
-
-
-E2E_GRAFANA_VERSION ?= 7.3.6
-
-e2e: #: Run all Grafana end-to-end tests.
- GRAFANA_VERSION=${E2E_GRAFANA_VERSION} \
- docker-compose -f e2e/docker-compose.yml up \
- --abort-on-container-exit \
- --exit-code-from e2e \
- --remove-orphans
-
-e2e-dev: #: Run Grafana e2e tests in Cypress test runner.
- GRAFANA_VERSION=${E2E_GRAFANA_VERSION} \
- DISPLAY=$$(ipconfig getifaddr en0):0 \
- docker-compose -f e2e/docker-compose.dev.yml up \
- --abort-on-container-exit \
- --exit-code-from e2e \
- --remove-orphans
-
-e2e-compose-up: #: Run Grafana e2e environment spining Prometheus and Vitess in the backend.
- GRAFANA_VERSION=${E2E_GRAFANA_VERSION} \
- COMPOSE_HTTP_TIMEOUT=200 \
- DISPLAY=$$(ipconfig getifaddr en0):0 \
- docker-compose -f e2e/docker-compose.vt.yml up \
- --remove-orphans
-
-e2e-compose-down: #: Clean docker compose resources
- @docker-compose -f e2e/docker-compose.vt.yml down -v --remove-orphans
- @rm e2e/grafana/provisioning/dashboards/*.json
- @rm e2e/prometheus/prometheus_rules.yaml
diff --git a/vitess-mixin/README.md b/vitess-mixin/README.md
deleted file mode 100644
index 695a943295c..00000000000
--- a/vitess-mixin/README.md
+++ /dev/null
@@ -1,41 +0,0 @@
-# (Beta) Monitoring Mixin for Vitess
-
-A set of Grafana dashboards, Prometheus rules and alerts for Vitess, packaged together in a reusable and extensible bundle.
-
-## 🔁 Prerequisites
-
-1. Go (programming language)
- - Install binaries using the official [installer](https://golang.org/dl/)
- - Ensure `GOPATH` variable is set in your system. See instructions [here](https://golang.org/doc/install#install). Here below there's a sample config:
-
- ```shell
- export GOPATH=$HOME/go
- export PATH="$GOPATH/bin:/usr/local/go/bin:$PATH"
- ```
-
-1. Install the go tools: `make tools`, `jb`, `jsonnet`, `jsonnetfmt`, and `promtool` should now be in `$GOPATH/bin`.
-
-1. Install the dependencies by running: `jb install`
-
-## ℹ️ How-to
-
-Customize `config.libsonnet` based on your setup. Example: specify the `dataSource` name (default to `Prometheus_Vitess`). You can then generate:
-
-- Prometheus alerts: `$ make prometheus_alerts.yaml`
-(Note: This files is empty because the current version of the mixin uses Grafana Alerts)
-
-- Prometheus rules: `$ make prometheus_rules.yaml`
-
-- Grafana dashboard: `$ ENV='prod' make dashboards_out` (Supported environments are `dev` and `prod`).
-
-The `prometheus_alerts.yaml` and `prometheus_rules.yaml` file then need to passed to your Prometheus server, and the files in `dashboards_out` need to be imported into you Grafana server.
-
-## 👩💻 Development
-
-If you want to contribute please read [Vitess mixin quickstart guide](vitess-mixin-quickstart.md)
-
-## 📚 Useful links & further learning
-
-- For more information about monitoring mixins, see this [design doc](https://docs.google.com/document/d/1A9xvzwqnFVSOZ5fD3blKODXfsat5fg6ZhnKu9LK3lB4/edit#).
-- For more motivation, see
-"[The RED Method: How to instrument your services](https://kccncna17.sched.com/event/CU8K/the-red-method-how-to-instrument-your-services-b-tom-wilkie-kausal?iframe=no&w=100%&sidebar=yes&bg=no)" talk from CloudNativeCon Austin.
diff --git a/vitess-mixin/alerts/alerts.libsonnet b/vitess-mixin/alerts/alerts.libsonnet
deleted file mode 100644
index 732a682f8b6..00000000000
--- a/vitess-mixin/alerts/alerts.libsonnet
+++ /dev/null
@@ -1,5 +0,0 @@
-{
- prometheusAlerts+:: {
-
- },
-}
diff --git a/vitess-mixin/config.libsonnet b/vitess-mixin/config.libsonnet
deleted file mode 100644
index 8c8f69ca0ee..00000000000
--- a/vitess-mixin/config.libsonnet
+++ /dev/null
@@ -1,74 +0,0 @@
-{
- _config+:: {
-
- // Selectors are inserted between {} in Prometheus queries.
- vtctldSelector: 'job="vitess-vtctld"',
- vtgateSelector: 'job="vitess-vtgate"',
- vttabletSelector: 'job="vitess-vttablet"',
- vtgateNodeSelector: 'job="node-exporter-vitess-vtgate"',
- mysqlSelector: 'job="mysql"',
- defaultTimeFrom: 'now-30m',
- vttabletMountpoint: '/mnt',
-
- // Datasource to use
- dataSource: 'Prometheus',
- nodeDataSource: 'Prometheus',
-
- // Default config for the Grafana dashboards in the Vitess Mixin
- grafanaDashboardMetadataDefault: {
- dashboardNameSuffix: '(auto-generated)',
- dashboardAlertPrefix: 'alerts',
- dashboardTags: ['vitess-mixin'],
- },
-
- dashborardLinks: {
- title: 'vitess-mixin',
- tags: ['vitess-mixin'],
- keepTime: true,
- includeVars: false,
- },
-
- // Grafana dashboard IDs are necessary for stable links for dashboards
- grafanaDashboardMetadata: {
-
- local defaultDashboard = {
- environments: ['dev', 'prod'],
- time_from: $._config.defaultTimeFrom,
- },
-
- // Overview
- clusterOverview+: defaultDashboard {
- uid: 'vitess-cluster-overview',
- title: 'cluster - overview %(dashboardNameSuffix)s' % $._config.grafanaDashboardMetadataDefault,
- description: 'General cluster overview',
- dashboardTags: $._config.grafanaDashboardMetadataDefault.dashboardTags + ['overview', 'cluster'],
- },
- vtgateOverview+: defaultDashboard {
- uid: 'vitess-vtgate-overview',
- title: 'vtgate - overview %(dashboardNameSuffix)s' % $._config.grafanaDashboardMetadataDefault,
- description: 'General vtgate overview',
- dashboardTags: $._config.grafanaDashboardMetadataDefault.dashboardTags + ['overview', 'vtgate'],
- },
-
- // Host View
- vttabletHostView+: defaultDashboard {
- uid: 'vitess-vttablet-host-view',
- title: 'vttablet - host view %(dashboardNameSuffix)s' % $._config.grafanaDashboardMetadataDefault,
- description: 'Detailed vttablet host view',
- dashboardTags: $._config.grafanaDashboardMetadataDefault.dashboardTags + ['vttablet', 'host'],
- },
- vtgateHostView+: defaultDashboard {
- uid: 'vitess-vtgate-host-view',
- title: 'vtgate - host view %(dashboardNameSuffix)s' % $._config.grafanaDashboardMetadataDefault,
- description: 'Detailed vtgate view by host',
- dashboardTags: $._config.grafanaDashboardMetadataDefault.dashboardTags + ['vtgate', 'host'],
- },
- },
- },
-
- os: import 'dashboards/resources/config/os_config.libsonnet',
- vttablet: import 'dashboards/resources/config/vttablet_config.libsonnet',
- vtgate: import 'dashboards/resources/config/vtgate_config.libsonnet',
- mysql: import 'dashboards/resources/config/mysql_config.libsonnet',
- row: import 'dashboards/resources/config/row_config.libsonnet',
-}
diff --git a/vitess-mixin/dashboards/dashboards.libsonnet b/vitess-mixin/dashboards/dashboards.libsonnet
deleted file mode 100644
index 5840e4b5228..00000000000
--- a/vitess-mixin/dashboards/dashboards.libsonnet
+++ /dev/null
@@ -1,5 +0,0 @@
-(import 'layouts/cluster_overview.libsonnet') +
-(import 'layouts/vtgate_host_view.libsonnet') +
-(import 'layouts/vtgate_overview.libsonnet') +
-(import 'layouts/vttablet_host_view.libsonnet') +
-(import 'defaults.libsonnet')
diff --git a/vitess-mixin/dashboards/defaults.libsonnet b/vitess-mixin/dashboards/defaults.libsonnet
deleted file mode 100644
index d94de1fab09..00000000000
--- a/vitess-mixin/dashboards/defaults.libsonnet
+++ /dev/null
@@ -1,24 +0,0 @@
-{
- local grafanaDashboards = super.grafanaDashboards,
-
- grafanaDashboards:: {
- [filename]: grafanaDashboards[filename] {
- // Modify tooltip to only show a single value
- rows: [
- row {
- panels: [
- panel {
- tooltip+: {
- shared: false,
- },
- }
- for panel in super.panels
- ],
- }
- for row in super.rows
- ],
-
- }
- for filename in std.objectFields(grafanaDashboards)
- },
-}
diff --git a/vitess-mixin/dashboards/layouts/cluster_overview.libsonnet b/vitess-mixin/dashboards/layouts/cluster_overview.libsonnet
deleted file mode 100644
index 6db716c982c..00000000000
--- a/vitess-mixin/dashboards/layouts/cluster_overview.libsonnet
+++ /dev/null
@@ -1,56 +0,0 @@
-local helpers = import '../resources/grafonnet/helpers/helpers.libsonnet';
-local panels = import '../resources/grafonnet/panels.libsonnet';
-local rows = import '../resources/grafonnet/rows.libsonnet';
-local singlestats = import '../resources/grafonnet/singlestats.libsonnet';
-local templates = import '../resources/grafonnet/templates.libsonnet';
-local texts = import '../resources/grafonnet/texts.libsonnet';
-
-local config = import '../../config.libsonnet';
-
-{
- grafanaDashboards+:: {
- 'cluster_overview.json':
-
- helpers.dashboard.getDashboard(config._config.grafanaDashboardMetadata.clusterOverview)
- .addTemplates([
- templates.interval,
- ])
- .addLink(helpers.default.getDashboardLink(config._config.dashborardLinks))
- .addPanels([
- texts.clusterOverview { gridPos: { h: 3, w: 24, x: 0, y: 0 } },
- singlestats.vtgateSuccessRate { gridPos: { h: 4, w: 4, x: 0, y: 3 } },
- singlestats.vttabletQuerySuccess { gridPos: { h: 4, w: 4, x: 4, y: 3 } },
- helpers.vtgate.getSingleStat(config.vtgate.singlestats.vtgateQueryLatencyP99) { gridPos: { h: 4, w: 4, x: 8, y: 3 } },
- helpers.vtgate.getSingleStat(config.vtgate.singlestats.vtgateQPS) { gridPos: { h: 2, w: 4, x: 12, y: 3 } },
- helpers.vttablet.getSingleStat(config.vttablet.singlestats.vttabletQPS) { gridPos: { h: 2, w: 4, x: 12, y: 5 } },
- singlestats.mysqlQPS { gridPos: { h: 2, w: 4, x: 16, y: 3 } },
- singlestats.keyspaceCount { gridPos: { h: 2, w: 2, x: 16, y: 3 } },
- singlestats.shardCount { gridPos: { h: 2, w: 2, x: 18, y: 3 } },
- singlestats.vtgateUp { gridPos: { h: 2, w: 2, x: 20, y: 3 } },
- singlestats.vtctldUp { gridPos: { h: 2, w: 2, x: 20, y: 5 } },
- singlestats.vttabletUp { gridPos: { h: 2, w: 2, x: 22, y: 3 } },
-
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequests) { gridPos: { h: 6, w: 8, x: 0, y: 7 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRate) { gridPos: { h: 6, w: 8, x: 8, y: 7 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99) { gridPos: { h: 6, w: 8, x: 16, y: 7 } },
-
- rows.RED { gridPos: { h: 1, w: 24, x: 0, y: 13 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequestsByKeyspace) { gridPos: { h: 8, w: 8, x: 0, y: 14 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRateByKeyspace) { gridPos: { h: 8, w: 8, x: 8, y: 14 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99ByKeyspace) { gridPos: { h: 8, w: 8, x: 16, y: 14 } },
-
- rows.tabletsQueries { gridPos: { h: 1, w: 24, x: 0, y: 22 } },
- helpers.vttablet.getPanel(config.vttablet.panels.countServingTablets) { gridPos: { h: 8, w: 8, x: 0, y: 23 } },
- helpers.mysql.getPanel(config.mysql.panels.mysqlSlowQueries) { gridPos: { h: 8, w: 8, x: 8, y: 23 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryTransactionKilled) { gridPos: { h: 8, w: 8, x: 16, y: 23 } },
-
- rows.serviceRestart { gridPos: { h: 1, w: 24, x: 0, y: 31 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRestart) { gridPos: { h: 8, w: 8, x: 0, y: 32 } },
- panels.vtctldRestart { gridPos: { h: 8, w: 8, x: 8, y: 32 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRestart) { gridPos: { h: 8, w: 8, x: 16, y: 32 } },
-
- helpers.mysql.getPanel(config.mysql.panels.mysqlRestart) { gridPos: { h: 8, w: 8, x: 16, y: 40 } },
- ]),
-
- },
-}
diff --git a/vitess-mixin/dashboards/layouts/vtgate_host_view.libsonnet b/vitess-mixin/dashboards/layouts/vtgate_host_view.libsonnet
deleted file mode 100644
index 139999e09fc..00000000000
--- a/vitess-mixin/dashboards/layouts/vtgate_host_view.libsonnet
+++ /dev/null
@@ -1,49 +0,0 @@
-local heatmaps = import '../resources/grafonnet/heatmaps.libsonnet';
-local helpers = import '../resources/grafonnet/helpers/helpers.libsonnet';
-local rows = import '../resources/grafonnet/rows.libsonnet';
-local singlestats = import '../resources/grafonnet/singlestats.libsonnet';
-local templates = import '../resources/grafonnet/templates.libsonnet';
-local texts = import '../resources/grafonnet/texts.libsonnet';
-
-local config = import '../../config.libsonnet';
-
-// TODO: add connections info
-
-{
- grafanaDashboards+:: {
- 'vtgate_host_view.json':
-
- helpers.dashboard.getDashboard(config._config.grafanaDashboardMetadata.vtgateHostView)
- .addTemplates([
- templates.hostVtgate,
- ])
- .addLink(helpers.default.getDashboardLink(config._config.dashborardLinks))
- .addPanels([
- texts.vtgateHost { gridPos: { h: 3, w: 24, x: 0, y: 0 } },
- rows.RED { gridPos: { h: 1, w: 24, x: 0, y: 4 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequestsByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 5 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRateByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 5 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99ByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 5 } },
- rows.REDByTabletType.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequestsByInstanceDBType) { gridPos: { h: 7, w: 8, x: 0, y: 9 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRateByInstanceDBType) { gridPos: { h: 7, w: 8, x: 8, y: 9 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99ByInstanceDBType) { gridPos: { h: 7, w: 8, x: 16, y: 9 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 8 } },
- rows.errors.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorsByInstanceKeyspace) { gridPos: { h: 7, w: 8, x: 0, y: 17 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorsByInstanceCode) { gridPos: { h: 7, w: 8, x: 8, y: 17 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 16 } },
- rows.duration.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationAVGByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 25 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP50ByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 25 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP95ByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 25 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 24 } },
- rows.OS.addPanels([
- helpers.os.getPanel(config.os.panels.CPUUsageByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 33 } },
- helpers.os.getPanel(config.os.panels.MemoryUsageByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 33 } },
- helpers.os.getPanel(config.os.panels.NetworkUsageByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 33 } },
- helpers.os.getPanel(config.os.panels.TCPRetransmissionsByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 40 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 32 } },
- ]),
- },
-}
diff --git a/vitess-mixin/dashboards/layouts/vtgate_overview.libsonnet b/vitess-mixin/dashboards/layouts/vtgate_overview.libsonnet
deleted file mode 100644
index b2c14eb8563..00000000000
--- a/vitess-mixin/dashboards/layouts/vtgate_overview.libsonnet
+++ /dev/null
@@ -1,58 +0,0 @@
-local heatmaps = import '../resources/grafonnet/heatmaps.libsonnet';
-local helpers = import '../resources/grafonnet/helpers/helpers.libsonnet';
-local rows = import '../resources/grafonnet/rows.libsonnet';
-local singlestats = import '../resources/grafonnet/singlestats.libsonnet';
-local templates = import '../resources/grafonnet/templates.libsonnet';
-local texts = import '../resources/grafonnet/texts.libsonnet';
-
-local config = import '../../config.libsonnet';
-
-// TODO: add connections info
-
-{
- grafanaDashboards+:: {
- 'vtgate_overview.json':
-
- helpers.dashboard.getDashboard(config._config.grafanaDashboardMetadata.vtgateOverview)
- .addLink(helpers.default.getDashboardLink(config._config.dashborardLinks))
- .addPanels([
- texts.vtgateOverview { gridPos: { h: 3, w: 24, x: 0, y: 0 } },
-
- rows.RED { gridPos: { h: 1, w: 24, x: 0, y: 4 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequests) { gridPos: { h: 7, w: 8, x: 0, y: 5 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRate) { gridPos: { h: 7, w: 8, x: 8, y: 5 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99) { gridPos: { h: 7, w: 8, x: 16, y: 5 } },
-
- rows.REDByKeyspace.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequestsByKeyspace) { gridPos: { h: 7, w: 8, x: 0, y: 13 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRateByKeyspace) { gridPos: { h: 7, w: 8, x: 8, y: 13 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99ByKeyspace) { gridPos: { h: 7, w: 8, x: 16, y: 13 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 12 } },
-
- rows.REDByTabletType.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateRequestsByDBType) { gridPos: { h: 7, w: 8, x: 0, y: 21 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorRateByDBType) { gridPos: { h: 7, w: 8, x: 8, y: 21 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP99ByDBType) { gridPos: { h: 7, w: 8, x: 16, y: 21 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 20 } },
-
- rows.errors.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorsByCode) { gridPos: { h: 7, w: 8, x: 0, y: 29 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorsByOperation) { gridPos: { h: 7, w: 8, x: 8, y: 29 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateErrorsByDbtype) { gridPos: { h: 7, w: 8, x: 16, y: 29 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 28 } },
-
- rows.duration.addPanels([
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationAVG) { gridPos: { h: 7, w: 8, x: 0, y: 37 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP50) { gridPos: { h: 7, w: 8, x: 8, y: 37 } },
- helpers.vtgate.getPanel(config.vtgate.panels.vtgateDurationP95) { gridPos: { h: 7, w: 8, x: 16, y: 37 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 36 } },
-
- rows.OS.addPanels([
- helpers.os.getPanel(config.os.panels.CPUUsage) { gridPos: { h: 7, w: 8, x: 0, y: 45 } },
- helpers.os.getPanel(config.os.panels.MemoryUsage) { gridPos: { h: 7, w: 8, x: 8, y: 45 } },
- helpers.os.getPanel(config.os.panels.NetworkUsage) { gridPos: { h: 7, w: 8, x: 16, y: 45 } },
- helpers.os.getPanel(config.os.panels.TCPRetransmissions) { gridPos: { h: 7, w: 8, x: 16, y: 52 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 44 } },
- ]),
- },
-}
diff --git a/vitess-mixin/dashboards/layouts/vttablet_host_view.libsonnet b/vitess-mixin/dashboards/layouts/vttablet_host_view.libsonnet
deleted file mode 100644
index 7148deb44ee..00000000000
--- a/vitess-mixin/dashboards/layouts/vttablet_host_view.libsonnet
+++ /dev/null
@@ -1,103 +0,0 @@
-local heatmaps = import '../resources/grafonnet/heatmaps.libsonnet';
-local helpers = import '../resources/grafonnet/helpers/helpers.libsonnet';
-local rows = import '../resources/grafonnet/rows.libsonnet';
-local templates = import '../resources/grafonnet/templates.libsonnet';
-local texts = import '../resources/grafonnet/texts.libsonnet';
-
-local config = import '../../config.libsonnet';
-local rows_helper = helpers.default;
-
-{
- grafanaDashboards+:: {
- 'vttablet_host_view.json':
-
- helpers.dashboard.getDashboard(config._config.grafanaDashboardMetadata.vttabletHostView)
- .addTemplates(
- [
- templates.hostVttablet,
- ]
- ).addLink(helpers.default.getDashboardLink(config._config.dashborardLinks))
- .addPanels(
- [
- texts.vttabletHost { gridPos: { h: 3, w: 24, x: 0, y: 0 } },
-
- rows.RED { gridPos: { h: 1, w: 24, x: 0, y: 4 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRequestsByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 5 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletErrorRateByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 5 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryDurationP99ByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 5 } },
-
- rows.REDByPlanType.addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRequestsByPlanType) { gridPos: { h: 7, w: 8, x: 0, y: 13 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletErrorRateByPlanFilteredByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 13 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryDurationP99ByPlan) { gridPos: { h: 7, w: 8, x: 16, y: 13 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 12 } },
-
- rows.REDByTable.addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRequestsByTableFilteredByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 29 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletErrorRateByTableFilteredByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 29 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 28 } },
-
- rows.rowsReturned.addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRowsReturnedByTableFilteredByInstance) { gridPos: { h: 7, w: 12, x: 0, y: 37 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletRowsReturnedByPlansFilterByInstance) { gridPos: { h: 7, w: 12, x: 12, y: 37 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 36 } },
-
- rows_helper.getRow(config.row.queryErrors).addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueriesKilled) { gridPos: { h: 7, w: 8, x: 0, y: 45 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryErrorsByType) { gridPos: { h: 7, w: 8, x: 8, y: 45 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 44 } },
-
- rows.vitessQueryPool.addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryPoolAvailableConnections) { gridPos: { h: 7, w: 8, x: 0, y: 52 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryPoolActiveConnections) { gridPos: { h: 7, w: 8, x: 8, y: 52 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryPoolIddleClosedRate) { gridPos: { h: 7, w: 8, x: 16, y: 52 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryPoolWaitCount) { gridPos: { h: 7, w: 8, x: 0, y: 59 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryPoolAvgWaitTime) { gridPos: { h: 7, w: 8, x: 8, y: 59 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 51 } },
-
- rows.vitessTransactionPool.addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionPoolAvailableConnections) { gridPos: { h: 7, w: 8, x: 0, y: 67 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionPoolActiveConnections) { gridPos: { h: 7, w: 8, x: 8, y: 67 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionPoolIddleClosedRate) { gridPos: { h: 7, w: 8, x: 16, y: 67 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionPoolWaitCount) { gridPos: { h: 7, w: 8, x: 0, y: 74 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionPoolAvgWaitTime) { gridPos: { h: 7, w: 8, x: 8, y: 74 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 66 } },
-
- rows_helper.getRow(config.row.vitessTimings).addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryDurationAvgByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 82 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryDurationP50ByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 82 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletQueryDurationP95ByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 82 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionDurationAvgByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 89 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionDurationP50ByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 89 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletTransactionDurationP95ByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 89 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vtgateToVtTabletCallTimeAvgFilteredByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 86 } },
- heatmaps.vttabletQueryTimeDistribution { gridPos: { h: 7, w: 16, x: 8, y: 86 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 81 } },
-
- rows.mysql.addPanels([
- helpers.mysql.getPanel(config.mysql.panels.mysqlSlowQueriesByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 94 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 93 } },
-
- rows_helper.getRow(config.row.mysqlTimings).addPanels([
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletMysqlTimeAvgFilteredByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 102 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletMysqlExecTimeP50FilterebyInstance) { gridPos: { h: 7, w: 8, x: 8, y: 102 } },
- helpers.vttablet.getPanel(config.vttablet.panels.vttabletMysqlExecTimeP95FilterebyInstance) { gridPos: { h: 7, w: 8, x: 16, y: 102 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 101 } },
-
- rows.OS.addPanels([
- helpers.os.getPanel(config.os.panels.CPUUsageByInstance) { gridPos: { h: 7, w: 8, x: 0, y: 110 } },
- helpers.os.getPanel(config.os.panels.MemoryUsageByInstance) { gridPos: { h: 7, w: 8, x: 8, y: 110 } },
- helpers.os.getPanel(config.os.panels.DiskUsageByInstance) { gridPos: { h: 7, w: 8, x: 16, y: 110 } },
- helpers.os.getPanel(config.os.panels.NetworkTxByInstance) { gridPos: { h: 7, w: 12, x: 0, y: 117 } },
- helpers.os.getPanel(config.os.panels.NetworkRxByInstance) { gridPos: { h: 7, w: 12, x: 12, y: 117 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 109 } },
-
- rows_helper.getRow(config.row.misc).addPanels([
- helpers.os.getPanel(config.vttablet.panels.vttabletGarbageCollectionCount) { gridPos: { h: 7, w: 8, x: 0, y: 125 } },
- helpers.os.getPanel(config.vttablet.panels.vttabletGarbageCollectionDuration) { gridPos: { h: 7, w: 8, x: 8, y: 125 } },
- helpers.os.getPanel(config.vttablet.panels.vttabletGarbageCollectionDurationQuantiles) { gridPos: { h: 7, w: 8, x: 16, y: 125 } },
- ]) { gridPos: { h: 1, w: 24, x: 0, y: 124 } },
- ],
- ),
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/configuration_templates.libsonnet b/vitess-mixin/dashboards/resources/config/configuration_templates.libsonnet
deleted file mode 100644
index 6f1f85da0a3..00000000000
--- a/vitess-mixin/dashboards/resources/config/configuration_templates.libsonnet
+++ /dev/null
@@ -1,125 +0,0 @@
-local config = import '../../../config.libsonnet';
-
-{
- //Override default_panel values with custom configuration
- prometheus_vitess: {
- panel: {
-
- datasource: {
- datasource: '%(dataSource)s' % config._config,
- },
-
- default+: self.datasource {
- fill: 0,
- legend_alignAsTable: true,
- legend_values: true,
- min: 0,
- shared_tooltip: false,
- },
-
- legend_max+: self.default {
- legend_max: true,
- },
-
- legend_min_max_avg+: self.default {
- legend_max: true,
- legend_min: true,
- legend_avg: true,
- },
-
- legend_min_max_current+: self.default {
- legend_max: true,
- legend_min: true,
- legend_current: true,
- },
-
- null_as_zeros+: self.default {
- nullPointMode: 'null as zero',
- },
-
- mysql_default+: self.default {
- legend_max: true,
- legend_sort: 'max',
- },
-
- orc_default+: self.default {
- legend_current: true,
- legend_rightSide: true,
- legend_sort: 'current',
- legend_sortDesc: false,
- pointradius: 5,
- sort: 'none',
- },
-
- vitess_vttablet_host_view+: self.mysql_default {
- legend_current: true,
- legend_min: true,
- legend_sort: 'current',
- legend_sortDesc: true,
- },
-
- go_gc_seconds+: self.legend_max {
- format: 's',
- },
-
- go_gc_ops+: self.legend_max {
- format: 'ops',
- },
-
- mysql_timings+: self.legend_min_max_avg {
- legend_sort: 'max',
- legend_sortDesc: true,
- format: 's',
- },
-
- vtgate_to_vttablet_calls: self.legend_min_max_avg + self.null_as_zeros {
- format: 's',
- legend_sortDesc: true,
- legend_sort: 'max',
- },
-
- },
- },
-
- prometheus_node: {
- panel: {
- default: {
- datasource: '%(nodeDataSource)s' % config._config,
- fill: 0,
- legend_alignAsTable: true,
- legend_current: true,
- legend_sort: 'current',
- legend_sortDesc: true,
- legend_values: true,
- min: 0,
- sort: 'decreasing',
- },
-
- percent_panel: self.default {
- format: 'percentunit',
- legend_min: true,
- legend_max: true,
- },
-
- null_as_zeros+: self.default {
- nullPointMode: 'null as zero',
- },
-
- vttablet_host_view: self.percent_panel {
- legend_avg: true,
- legend_current: false,
- legend_sort: 'max',
- },
-
- performance_analysis_short: self.percent_panel {
- format: 'short',
- legend_min: false,
- legend_max: false,
- },
-
- performance_analysis_seconds+: self.performance_analysis_short {
- format: 's',
- },
- },
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/mysql_config.libsonnet b/vitess-mixin/dashboards/resources/config/mysql_config.libsonnet
deleted file mode 100644
index 543e3565902..00000000000
--- a/vitess-mixin/dashboards/resources/config/mysql_config.libsonnet
+++ /dev/null
@@ -1,87 +0,0 @@
-local config = import '../../../config.libsonnet';
-local configuration_templates = import './configuration_templates.libsonnet';
-local vitess_ct = configuration_templates.prometheus_vitess;
-
-// TODO: move local template variables and fields to ./configuration_templates.libsonnet.
-{
- // TODO: add description for each panel.
- panels: {
-
- local panel_template = vitess_ct.panel.mysql_default,
- local vttablet_host_view_panel_template = vitess_ct.panel.vitess_vttablet_host_view,
-
- // TODO Create a recording rule for the prometheus target.
- mysqlRestart: panel_template {
- title: 'MySQL (by keyspace/shard)',
- bars: true,
- format: 'short',
- legend_sort: 'max',
- legend_sortDesc: false,
- lines: false,
- sort: 'increasing',
- targets: [
- {
- expr: |||
- count by (keyspace, shard) (
- idelta (
- mysql_global_status_uptime{
- %(mysqlSelector)s
- }[5m]
- ) < 0
- )
- ||| % config._config,
- legendFormat: '{{keyspace}}/{{shard}}',
- },
- ],
- },
-
- // TODO Create a recording rule for the prometheus target.
- mysqlSlowQueries: panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Slow queries',
- format: 'cps',
- legend_min: true,
- legend_current: true,
- legend_sort: 'current',
- legend_sortDesc: true,
- sort: 'decreasing',
- nullPointMode: 'null as zero',
- targets: [
- {
- expr: |||
- sum (
- rate(
- mysql_global_status_slow_queries{
- %(mysqlSelector)s
- }[$interval]
- )
- )
- ||| % config._config,
- legendFormat: 'Slow Queries',
- },
- ],
- },
-
- // TODO Create a recording rule for the prometheus target.
- mysqlSlowQueriesByInstance: vttablet_host_view_panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Slow Queries',
- format: 'ops',
- targets: [
- {
- expr: |||
- sum by(instance)(
- rate(
- mysql_global_status_slow_queries{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/os_config.libsonnet b/vitess-mixin/dashboards/resources/config/os_config.libsonnet
deleted file mode 100644
index 8451311cb1b..00000000000
--- a/vitess-mixin/dashboards/resources/config/os_config.libsonnet
+++ /dev/null
@@ -1,384 +0,0 @@
-/** This is a configuration file containing metadata for OS (Prometheus Node) grafana resources. */
-
-local config = import '../../../config.libsonnet';
-local configuration_templates = import './configuration_templates.libsonnet';
-local node_ct = configuration_templates.prometheus_node;
-
-// TODO: move local template variables and fields to ./configuration_templates.libsonnet.
-{
- // TODO: add description for each panel.
- panels: {
-
- local vtgate_panel_template = node_ct.panel.percent_panel,
- local vttablet_host_view_panel_template = node_ct.panel.vttablet_host_view,
-
- CPUUsage:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'CPU Usage',
- targets: [
- {
- expr:
- |||
- 1 -
- avg (
- rate(
- node_cpu_seconds_total{
- %(vtgateNodeSelector)s,
- mode="idle"
- }[1m]
- )
- )
- ||| % config._config,
- legendFormat: 'cpu usage',
- },
- ],
- },
-
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- CPUUsageByInstance:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'CPU Usage',
- targets: [
- {
- expr:
- |||
- 1 -
- avg by (instance)(
- rate(
- node_cpu_seconds_total{
- instance=~"$host",
- mode="idle"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- MemoryUsage:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Memory Usage',
- targets: [
- {
- expr:
- |||
- 1 -
- sum (
- node_memory_MemAvailable_bytes{
- %(vtgateNodeSelector)s
- }
- )
- /
- sum (
- node_memory_MemTotal_bytes{
- %(vtgateNodeSelector)s
- }
- )
- ||| % config._config,
- legendFormat: 'Memory Usage',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- MemoryUsageByInstance:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Memory Usage',
- targets: [
- {
- expr:
- |||
- 1 -
- sum by (instance)(
- node_memory_MemAvailable_bytes{
- instance=~"$host"
- }
- )
- /
- sum by (instance)(
- node_memory_MemTotal_bytes{
- instance=~"$host"
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- TCPRetransmissions:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'TCP Retransmissions',
- targets: [
- {
- expr:
- |||
- sum (
- rate(
- node_netstat_Tcp_RetransSegs{
- %(vtgateNodeSelector)s
- }[1m]
- )
- )
- /
- sum (
- rate(
- node_netstat_Tcp_OutSegs{
- %(vtgateNodeSelector)s
- }[1m]
- )
- )
- ||| % config._config,
- legendFormat: 'TCP retransmissions',
- },
- ],
- },
-
- TCPRetransmissionsByInstance:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'TCP Retransmissions',
- targets: [
- {
- expr:
- |||
- sum by (instance) (
- rate(
- node_netstat_Tcp_RetransSegs{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by (instance) (
- rate(
- node_netstat_Tcp_OutSegs{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- NetworkUsage:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Network Usage',
- format: 'bps',
- min: null,
- seriesOverrides: [
- {
- alias: '/egress .*/',
- transform: 'negative-Y',
- },
- ],
- targets: [
- {
- expr:
- |||
- sum (
- rate(
- node_network_receive_bytes_total{
- %(vtgateNodeSelector)s
- }[5m]
- )
- )
- * 8
- ||| % config._config,
- legendFormat: 'ingress',
- },
- {
- expr:
- |||
- sum (
- rate(
- node_network_transmit_bytes_total{
- %(vtgateNodeSelector)s
- }[5m]
- )
- )
- * 8
- ||| % config._config,
- legendFormat: 'egress',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- NetworkUsageByInstance:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Network Usage',
- format: 'Bps',
- min: null,
- seriesOverrides: [
- {
- alias: '/egress .*/',
- transform: 'negative-Y',
- },
- ],
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_receive_bytes_total{
- instance=~"$host"
- }[5m]
- )
- )
- ||| % config._config,
- legendFormat: 'ingress - {{instance}}',
- },
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_transmit_bytes_total{
- instance=~"$host"
- }[5m]
- )
- )
- ||| % config._config,
- legendFormat: 'egress - {{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- NetworkUsageByInstanceFilteredByShardKeyspace:
- vttablet_host_view_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Network Usage',
- format: 'bps',
- min: null,
- seriesOverrides: [
- {
- alias: '/egress .*/',
- transform: 'negative-Y',
- },
- ],
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_receive_bytes_total{
- keyspace="$keyspace",
- shard=~"$shard",
- instance=~"$host"
- }[5m]
- )
- )
- * 8
- |||,
- legendFormat: 'ingress - {{instance}}',
- },
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_transmit_bytes_total{
- keyspace="$keyspace",
- shard=~"$shard",
- instance=~"$host"
- }[5m]
- )
- )
- * 8
- |||,
- legendFormat: 'egress - {{instance}}',
- },
- ],
- },
-
- NetworkRxByInstance:
- vttablet_host_view_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Network Rx Bytes',
- format: 'bps',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_receive_bytes_total{
- instance=~"$host"
- }[1m]
- )
- )
- * 8
- |||,
- legendFormat: 'ingress - {{instance}}',
- },
- ],
- },
-
- NetworkTxByInstance:
- vttablet_host_view_panel_template
- + node_ct.panel.null_as_zeros {
- title: 'Network Tx Bytes',
- format: 'bps',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- node_network_transmit_bytes_total{
- instance=~"$host"
- }[1m]
- )
- )
- * 8
- |||,
- legendFormat: 'egress - {{instance}}',
- },
- ],
- },
-
- DiskUsageByInstance:
- vtgate_panel_template
- + node_ct.panel.null_as_zeros {
- title: '/mnt disk free',
- min: null,
- targets: [
- {
- expr:
- |||
- avg by(instance)(
- node_filesystem_avail_bytes{
- instance=~"$host",
- mountpoint="%(vttabletMountpoint)s"
- }
- /
- node_filesystem_size_bytes{
- instance=~"$host",
- mountpoint="%(vttabletMountpoint)s"
- }
- )
- ||| % config._config,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/row_config.libsonnet b/vitess-mixin/dashboards/resources/config/row_config.libsonnet
deleted file mode 100644
index 2fb6d6cb355..00000000000
--- a/vitess-mixin/dashboards/resources/config/row_config.libsonnet
+++ /dev/null
@@ -1,18 +0,0 @@
-{
- misc:: {
- title: 'Misc',
- collapse: true,
- },
- mysqlTimings: {
- title: 'MySQL Timings',
- collapse: true,
- },
- queryErrors:: {
- title: 'Queries/Errors',
- collapse: true,
- },
- vitessTimings:: {
- title: 'Vitess Timings',
- collapse: true,
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/vtgate_config.libsonnet b/vitess-mixin/dashboards/resources/config/vtgate_config.libsonnet
deleted file mode 100644
index acbc7e19efb..00000000000
--- a/vitess-mixin/dashboards/resources/config/vtgate_config.libsonnet
+++ /dev/null
@@ -1,765 +0,0 @@
-/** This is a configuration file containing metadata for vtgate grafana resources. */
-
-local config = import '../../../config.libsonnet';
-local configuration_templates = import './configuration_templates.libsonnet';
-local vitess_ct = configuration_templates.prometheus_vitess;
-
-// TODO: move local template variables and fields to ./configuration_templates.libsonnet.
-{
- // ____ _
- // | _ \ __ _ _ __ ___| |___
- // | |_) / _` | '_ \ / _ \ / __|
- // | __/ (_| | | | | __/ \__ \
- // |_| \__,_|_| |_|\___|_|___/
-
- // TODO: add description for each panel.
- panels: {
-
- //Override default_panel values with custom configuration
- local panel_template = {
- datasource: '%(dataSource)s' % config._config,
- format: 'rps',
- fill: 0,
- legend_values: true,
- legend_alignAsTable: true,
- legend_min: true,
- legend_max: true,
- legend_current: true,
- legend_sort: 'current',
- legend_sortDesc: true,
- min: 0,
- sort: 'decreasing',
- },
-
- local garbage_collector_panel_template = panel_template {
- format: 's',
- legend_sort: 'max',
- },
-
- vtgateRequests:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests',
- fill: 1,
- targets: [
- {
- expr: |||
- sum (
- vitess_mixin:vtgate_api_count:rate1m
- )
- |||,
- legendFormat: 'Requests',
- },
- ],
- },
-
- vtgateRequestsByKeyspace:
- panel_template +
- vitess_ct.panel.null_as_zeros {
- title: 'Requests (by keyspace)',
- targets: [
- {
- expr: |||
- sum by(keyspace)(
- vitess_mixin:vtgate_api_count_by_keyspace:rate1m
- )
- ||| % config._config,
- legendFormat: '{{keyspace}}',
- },
- ],
- },
-
-
- vtgateRequestsByDBType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests (by db_type)',
- targets: [
- {
- expr: |||
- sum by (db_type)(
- vitess_mixin:vtgate_api_count_by_db_type:rate1m
- )
- |||,
- legendFormat: '{{db_type}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vtgateRequestsByInstanceDBType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests (by db_type)',
- fill: 0,
- targets: [
- {
- expr: |||
- sum by (instance, db_type)(
- rate(
- vtgate_api_count{
- instance=~"$host",
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}} - {{db_type}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS TARGET
- vtgateRequestsByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests',
- fill: 0,
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vtgate_api_count{
- instance=~'$host'
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
-
- vtgateErrorRate:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate',
- format: 'percentunit',
- fill: 1,
- aliasColors: {
- 'Error rate': '#F2495C',
- },
- targets: [
- {
- expr: |||
- sum (
- vitess_mixin:vtgate_api_error_counts:rate1m)
- /
- sum (
- vitess_mixin:vtgate_api_count:rate1m)
- |||,
- legendFormat: 'Error rate',
- },
- ],
- },
-
- vtgateErrorRateByKeyspace:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate (by keyspace)',
- format: 'percentunit',
- targets: [
- {
- expr: |||
- sum by(keyspace)(
- vitess_mixin:vtgate_api_error_counts_by_keyspace:rate1m)
- /
- sum by(keyspace)(
- vitess_mixin:vtgate_api_count_by_keyspace:rate1m)
- |||,
- legendFormat: '{{keyspace}}',
- },
- ],
- },
-
- vtgateErrorRateByDBType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate (by db_type)',
- format: 'percentunit',
- targets: [
- {
- expr: |||
- sum by (db_type)(
- vitess_mixin:vtgate_api_error_counts_by_db_type:rate1m
- )
- /
- sum by (db_type)(
- vitess_mixin:vtgate_api_count_by_db_type:rate1m
- )
- ||| % config._config,
- legendFormat: '{{db_type}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS TARGET
- vtgateErrorRateByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate',
- fill: 0,
- format: 'percentunit',
- nullPointMode: 'null as zero',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- vtgate_api_error_counts[1m]
- ) > 0
- )
- /
- sum by(instance)(
- rate(
- vtgate_api_count[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO Create RECORDING RULES FOR THESE PROM TARGETS
- vtgateErrorRateByInstanceDBType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate (by db_type)',
- fill: 0,
- format: 'percentunit',
- targets: [
- {
- expr:
- |||
- sum by(instance, db_type)(
- rate(vtgate_api_error_counts{
- instance=~"$host"
- }[1m]
- ) > 0
- )
- /
- sum by(instance, db_type)(
- rate(
- vtgate_api_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}} - {{db_type}}',
- intervalFactor: 1,
- },
- ],
- },
-
- vtgateDurationP99:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Duration 99th quantile',
- fill: 1,
- format: 's',
- aliasColors: {
- Duration: '#5794F2',
- },
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.99,
- sum by(le)(
- vitess_mixin:vtgate_api_bucket:rate1m
- )
- )
- |||,
- legendFormat: 'Duration',
- },
- ],
- },
-
- vtgateDurationP99ByKeyspace:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Duration 99th quantile (by keyspace)',
- format: 's',
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.99,
- sum by(keyspace,le)(
- vitess_mixin:vtgate_api_bucket_by_keyspace:rate1m
- )
- )
- ||| % config._config,
- legendFormat: '{{keyspace}}',
- },
- ],
- },
-
- local vtgateDurationTemplate =
- panel_template
- + vitess_ct.panel.null_as_zeros {
- fill: 1,
- format: 's',
- aliasColors: {
- Duration: '#5794F2',
- },
- },
-
- //TODO crete a recording rule for this prometheus vitess target
- vtgateDurationP99ByInstance: vtgateDurationTemplate {
- title: 'Duration 99th quantile',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.99,
- sum by(instance,le)(
- rate(
- vtgate_api_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO crete a recording rule for this prometheus vitess target
- vtgateDurationP99ByInstanceDBType: vtgateDurationTemplate {
- title: 'Duration 99th quantile (by db_type)',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.99,
- sum by(instance,db_type,le)(
- rate(
- vtgate_api_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}} - {{db_type}}',
- intervalFactor: 1,
- },
- ],
- },
-
- vtgateDurationP50: vtgateDurationTemplate {
- title: 'Duration 50th quantile',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.50,
- sum by(le)(
- vitess_mixin:vtgate_api_bucket:rate1m
- )
- )
- |||,
- legendFormat: 'Duration p50',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vtgateDurationP50ByInstance: vtgateDurationTemplate {
- title: 'Duration 50th quantile',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.50,
- sum by(instance, le)(
- rate(
- vtgate_api_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vtgateDurationP95: vtgateDurationTemplate {
- title: 'Duration 95th quantile',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.95,
- sum by(le)(
- vitess_mixin:vtgate_api_bucket:rate1m
- )
- )
- |||,
- legendFormat: 'Duration p95',
- },
- ],
- },
-
- vtgateDurationP95ByInstance: vtgateDurationTemplate {
- title: 'Duration 95th quantile',
- fill: 0,
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.95,
- sum by(instance, le)(
- rate(
- vtgate_api_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO write a recording rule for this prometheus_vitess target
- // only vtgate_api_sum requires a rule. Use 1m interval instead of 5m.
- vtgateDurationAVG: vtgateDurationTemplate {
- title: 'Duration (Avg)',
- fill: 0,
- targets: [
- {
- expr: |||
- sum (
- rate(
- vtgate_api_sum[5m]
- )
- )
- /
- sum (
- rate(
- vtgate_api_count[5m]
- )
- )
- |||,
- legendFormat: 'Avg Latency',
- },
- ],
- },
-
- //TODO write a recording rule for this prometheus_vitess target
- vtgateDurationAVGByInstance: vtgateDurationTemplate {
- title: 'Duration (Avg)',
- fill: 0,
- targets: [
- {
- expr: |||
- sum by (instance)(
- rate(
- vtgate_api_sum{
- instance=~"$host"
- }[5m]
- )
- )
- /
- sum by (instance)(
- rate(
- vtgate_api_count{
- instance=~"$host"
- }[5m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vtgateDurationP99ByDBType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Duration 99th quantile (by db_type)',
- format: 's',
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.99,
- sum by (db_type, le)(
- vitess_mixin:vtgate_api_bucket_by_db_type:rate1m
- )
- )
- |||,
- legendFormat: '{{db_type}}',
- },
- ],
- },
-
- vtgateErrorsByCode:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Errors (by code)',
- format: 'cps',
- targets: [
- {
- expr: |||
- sum by (code)(
- vitess_mixin:vtgate_api_error_counts_by_code:rate1m
- )
- |||,
- legendFormat: '{{code}}',
- },
- ],
-
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vtgateErrorsByInstanceCode:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Errors (by code)',
- format: 'cps',
- targets: [
- {
- expr: |||
- sum by (instance,code)(
- rate(
- vtgate_api_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}} - {{code}}',
- },
- ],
-
- },
-
- vtgateErrorsByOperation:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Errors (by operation)',
- format: 'cps',
- targets: [
- {
- expr: |||
- sum by (operation)(
- vitess_mixin:vtgate_api_error_counts_by_operation:rate1m
- )
- |||,
- legendFormat: '{{operation}}',
- },
- ],
-
- },
-
- vtgateErrorsByDbtype:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Errors (by db_type)',
- format: 'cps',
- targets: [
- {
- expr: |||
- sum by (db_type)(
- vitess_mixin:vtgate_api_error_counts_by_db_type:rate1m
- )
- |||,
- legendFormat: '{{db_type}}',
- },
- ],
-
- },
-
- //TODO CREATE RECORDING RULE FOR THIS PROM TARGET
- vtgateErrorsByInstanceKeyspace:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Errors (by keyspace)',
- format: 'cps',
- targets: [
- {
- expr: |||
- sum by (instance,keyspace)(
- rate(
- vtgate_api_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{keyspace}}',
- intervalFactor: 1,
- },
- ],
- },
-
- vtgateRestart: {
- title: 'vtgate',
- bars: true,
- datasource: '%(dataSource)s' % config._config,
- fill: 0,
- format: 'short',
- legend_alignAsTable: true,
- legend_current: false,
- legend_max: true,
- legend_min: false,
- legend_sort: 'max',
- legend_sortDesc: false,
- legend_values: true,
- lines: false,
- min: 0,
- shared_tooltip: false,
- sort: 'increasing',
- targets: [
- {
- expr: |||
- sum by (instance)(
- vitess_mixin:process_start_time_seconds_by_instance_job:sum5m{
- %(vtgateSelector)s
- }
- )
- ||| % config._config,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO crete a recording rule for this prometheus vitess target
- vtgateGarbageCollectionCount: garbage_collector_panel_template {
- title: 'GC Count',
- format: 'ops',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- go_gc_duration_seconds_count{
- %(vtgateSelector)s
- }[1m]
- )
- )
- ||| % config._config,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
- //TODO crete a recording rule for this prometheus vitess target
- vtgateGarbageCollectionDuration: garbage_collector_panel_template {
- title: 'GC Duration total per second',
- description: 'A summary of the pause duration of garbage collection cycles',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- go_gc_duration_seconds_count{
- %(vtgateSelector)s
- }[1m]
- )
- )
- ||| % config._config,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
- //TODO crete a recording rule for this prometheus vitess target
- vtgateGarbageCollectionDurationQuantiles: garbage_collector_panel_template {
- title: 'GC Duration quantiles',
- targets: [
- {
- expr:
- |||
- sum by(quantile)(
- rate(
- go_gc_duration_seconds{
- %(vtgateSelector)s
- }[1m]
- )
- )
- ||| % config._config,
- legendFormat: 'p{{quantile}}',
- intervalFactor: 1,
- },
- ],
- },
- },
-
- // _ _ _ _
- // ___(_)_ __ __ _| | ___ ___| |_ __ _| |_ ___
- // / __| | '_ \ / _` | |/ _ \/ __| __/ _` | __/ __|
- // \__ \ | | | | (_| | | __/\__ \ || (_| | |_\__ \
- // |___/_|_| |_|\__, |_|\___||___/\__\__,_|\__|___/
- // |___/
-
- //TODO move default configurations to helper code (vttablet_helper)
- singlestats: {
-
- vtgateQPS: {
- title: 'QPS - vtgate',
- datasource: '%(dataSource)s' % config._config,
- format: 'short',
- valueFontSize: '70%',
- valueName: 'current',
- sparklineFull: true,
- sparklineShow: true,
- target:
- {
- expr: |||
- sum (
- vitess_mixin:vtgate_api_count:rate1m
- )
- |||,
- intervalFactor: 1,
- },
- },
-
- vtgateQueryLatencyP99: {
- title: 'Query latency p99',
- datasource: '%(dataSource)s' % config._config,
- colorBackground: true,
- decimals: 2,
- format: 'ms',
- valueFontSize: '70%',
- valueName: 'current',
- thresholds: '30,50',
- target:
- {
- expr: |||
- 1000 * histogram_quantile(
- 0.99,
- sum by(le)(
- vitess_mixin:vtgate_api_bucket:rate1m
- )
- )
- |||,
- instant: true,
- intervalFactor: 1,
- },
- },
- },
-}
diff --git a/vitess-mixin/dashboards/resources/config/vttablet_config.libsonnet b/vitess-mixin/dashboards/resources/config/vttablet_config.libsonnet
deleted file mode 100644
index 4dbd8a989de..00000000000
--- a/vitess-mixin/dashboards/resources/config/vttablet_config.libsonnet
+++ /dev/null
@@ -1,1073 +0,0 @@
-/** This is a configuration file containing metadata for vttablet grafana resources. */
-
-local config = import '../../../config.libsonnet';
-local configuration_templates = import './configuration_templates.libsonnet';
-local vitess_ct = configuration_templates.prometheus_vitess;
-
-// TODO: move local template variables to ./configurations_templates.libsonnet.
-{
- panels: {
- // ____ _
- // | _ \ __ _ _ __ ___| |___
- // | |_) / _` | '_ \ / _ \ / __|
- // | __/ (_| | | | | __/ \__ \
- // |_| \__,_|_| |_|\___|_|___/
-
- // TODO: add description for each panel.
-
- //Override default_panel values with custom configuration
- local vttablet_queries_killed = vitess_ct.panel.legend_min_max_avg + vitess_ct.panel.null_as_zeros,
- local vttablet_query_errors_by_type = vitess_ct.panel.legend_min_max_avg + vitess_ct.panel.null_as_zeros,
-
- local panel_template = vitess_ct.panel.legend_min_max_current {
- legend_sort: 'current',
- legend_sortDesc: true,
- shared_tooltip: true,
- sort: 'decreasing',
- },
-
- local vttablet_host_view_panel_template = panel_template {
- legend_sort: 'avg',
- legend_avg: true,
- legend_current: false,
- },
-
- //TODO Create a recording rule.
- countServingTablets:
- panel_template {
- title: '# of serving tablets',
- legend_sortDesc: false,
- shared_tooltip: false,
- sort: 'increasing',
- targets: [
- {
- expr:
- |||
- count(
- vttablet_tablet_server_state{
- %(vttabletSelector)s,
- name="SERVING"
- }
- )
- |||
- % config._config,
- legendFormat: 'SERVING',
- },
- ],
- },
-
- vttabletRequestsByTable:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests (by table)',
- format: 'rps',
- targets: [
- {
- expr:
- |||
- sum by (table)(
- vitess_mixin:vttablet_query_counts_by_keyspace_table:rate1m{
- table=~"$table"
- }
- )
- or
- vector(0)
- |||,
- legendFormat: '{{table}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMEHTEUS TARGET
- vttabletRequestsByPlanType:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests (by plan type)',
- format: 'ops',
- nullPointMode: 'null as zero',
- targets: [
- {
- expr:
- |||
- sum by (plan_type)(
- rate(
- vttablet_queries_count{
- instance=~"$host"
- } [1m]
- )
- )
- |||,
- legendFormat: '{{plan_type}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vttabletRequestsByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests',
- format: 'ops',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- nullPointMode: 'null as zero',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_query_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vttabletRequestsByTableFilteredByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Requests (by table)',
- format: 'ops',
- nullPointMode: 'null as zero',
- targets: [
- {
- expr:
- |||
- sum by (table)(
- rate(
- vttablet_query_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{table}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vttabletErrorRateByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate',
- format: 'percentunit',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- /
- (
- sum by (instance)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- +
- sum by (instance)(
- rate(
- vttablet_query_counts{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vttabletErrorRateByPlanFilteredByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate (by plan type)',
- format: 'percentunit',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- targets: [
- {
- expr:
- |||
- sum by (plan)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- /
- (
- sum by (plan)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- +
- sum by (plan)(
- rate(
- vttablet_query_counts{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{plan}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROM TARGET
- vttabletErrorRateByTableFilteredByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Error rate (by table)',
- format: 'percentunit',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- targets: [
- {
- expr:
- |||
- sum by (table)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- /
- (
- sum by (table)(
- rate(
- vttablet_query_error_counts{
- instance=~"$host"
- }[1m]
- )
- )
- +
- sum by (table)(
- rate(
- vttablet_query_counts{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{table}}',
- },
- ],
- },
-
- vttabletRowsReturnedByTableFilteredByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Rows Returned (by table)',
- targets: [
- {
- expr:
- |||
- sum by (table) (
- rate(
- vttablet_query_row_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{table}}',
- },
- ],
- },
-
- vttabletRowsReturnedByPlansFilterByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Rows Returned (by plan)',
- targets: [
- {
- expr:
- |||
- sum by (plan) (
- rate(
- vttablet_query_row_counts{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{plan}}',
- },
- ],
- },
-
- //TODO DEDUPLICATE LEGEND CONFIGURATION FOR QUERY DURATION PANELS
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryDurationAvgByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Query Duration (avg)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- vttablet_queries_sum{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by(instance)(
- rate(
- vttablet_queries_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryDurationP50ByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Query Duration (p50)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.50,sum by(instance,le)(
- rate(
- vttablet_queries_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryDurationP95ByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Query Duration (p95)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.95,sum by(instance,le)(
- rate(
- vttablet_queries_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryDurationP99ByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Duration (p99)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.99,sum by(instance,le)(
- rate(
- vttablet_queries_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryDurationP99ByPlan:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Duration p99 (by plan type)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'avg',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.99,sum by(plan_type,le)(
- rate(
- vttablet_queries_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{plan_type}}',
- },
- ],
- },
-
- //TODO DEDUPLICATE LEGEND CONFIGURATION FOR TRANSACTION DURATION PANELS
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionDurationAvgByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Transaction Duration (avg)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- vttablet_transactions_sum{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by(instance)(
- rate(
- vttablet_transactions_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionDurationP50ByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Transaction Duration (p50)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.50,sum by(instance,le)(
- rate(
- vttablet_transactions_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionDurationP95ByInstance:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Transaction Duration (p95)',
- format: 's',
- legend_current: false,
- legend_avg: true,
- legend_sort: 'max',
- targets: [
- {
- expr:
- |||
- histogram_quantile(
- 0.95,sum by(instance,le)(
- rate(
- vttablet_transactions_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vttabletQueryTransactionKilled:
- panel_template
- + vitess_ct.panel.null_as_zeros {
- title: 'Query/Transaction killed',
- format: 'cps',
- legend_alignAsTable: true,
- shared_tooltip: false,
- targets: [
- {
- expr:
- |||
- sum (
- vitess_mixin:vttablet_kills:rate1m
- )
- |||,
- legendFormat: 'Killed',
- },
- ],
- },
-
- vttabletRestart: {
- title: 'vttablet',
- bars: true,
- datasource: '%(dataSource)s' % config._config,
- fill: 0,
- format: 'short',
- legend_values: true,
- legend_alignAsTable: true,
- legend_max: true,
- legend_sort: 'max',
- legend_sortDesc: false,
- lines: false,
- min: 0,
- shared_tooltip: false,
- sort: 'increasing',
- targets: [
- {
- expr:
- |||
- sum by (instance) (
- vitess_mixin:process_start_time_seconds_by_instance_job:sum5m{
- %(vttabletSelector)s
- }
- )
- ||| % config._config,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryPoolAvailableConnections: vttablet_host_view_panel_template {
- title: 'Available Connections',
- description: 'number of available connections in the pool in real-time',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- vttablet_conn_pool_available{
- instance=~'$host'
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryPoolActiveConnections: vttablet_host_view_panel_template {
- title: 'Active Connections',
- description: 'count of in use connections to mysql',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by(instance) (
- vttablet_conn_pool_active{
- instance=~'$host'
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryPoolIddleClosedRate: vttablet_host_view_panel_template {
- title: 'Idle Closed Rate',
- description: 'rate of closing connections due to the idle timeout',
- format: 'ops',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_conn_pool_idle_closed{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryPoolWaitCount: vttablet_host_view_panel_template {
- title: 'Wait count',
- description: 'WaitCount will give you how often the transaction pool gets full that causes new transactions to wait.',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_conn_pool_wait_count{
- instance=~'$host'
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryPoolAvgWaitTime: vttablet_host_view_panel_template {
- title: 'Avg wait time',
- format: 's',
- description: 'WaitTime/WaitCount will tell you the average wait time.',
- targets: [
- {
- expr:
- |||
- sum by (instance) (
- rate(
- vttablet_conn_pool_wait_time{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by (instance) (
- rate(
- vttablet_conn_pool_wait_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vttabletQueriesKilled: vttablet_queries_killed {
- title: 'Queries Killed',
- description: |||
- Kills reports the queries and transactions killed by VTTablet due to timeout.
- It’s a very important variable to look at during outages.
- |||,
- targets: [
- {
- expr: |||
- sum by (instance)(
- vitess_mixin:vttablet_kills_by_instance:rate1m{
- instance=~"$host"
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- vttabletQueryErrorsByType: vttablet_query_errors_by_type {
- title: 'Query errors (by error code)',
- description: '',
- targets: [
- {
- expr: |||
- sum by (error_code)(
- vitess_mixin:vttablet_errors:rate1m{
- instance=~"$host"
- }
- )
- |||,
- legendFormat: 'ErrorCode: {{error_code}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionPoolAvailableConnections: vttablet_host_view_panel_template {
- title: 'Available Connections',
- description: 'number of available connections in the pool',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- vttablet_transaction_pool_available{
- instance=~'$host'
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionPoolActiveConnections: vttablet_host_view_panel_template {
- title: 'Active Connections',
- description: 'Number of connections actually open to mysql',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by(instance) (
- vttablet_transaction_pool_active{
- instance=~'$host'
- }
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionPoolIddleClosedRate: vttablet_host_view_panel_template {
- title: 'Idle Closed Rate',
- description: 'Rate of closing connections due to the idle timeout',
- format: 'ops',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_transaction_pool_idle_closed{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionPoolWaitCount: vttablet_host_view_panel_template {
- title: 'Wait count',
- description: 'WaitCount will give you how often the transaction pool gets full that causes new transactions to wait.',
- format: 'short',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vttablet_transaction_pool_wait_count{
- instance=~'$host'
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletTransactionPoolAvgWaitTime: vttablet_host_view_panel_template {
- title: 'Avg wait time',
- format: 's',
- description: 'WaitTime/WaitCount will tell you the average wait time.',
- targets: [
- {
- expr:
- |||
- sum by (instance) (
- rate(
- vttablet_transaction_pool_wait_time{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by (instance) (
- rate(
- vttablet_transaction_pool_wait_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletGarbageCollectionCount: vitess_ct.panel.go_gc_ops {
- title: 'GC Count',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- go_gc_duration_seconds_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletGarbageCollectionDuration: vitess_ct.panel.go_gc_seconds {
- title: 'GC Duration total per second',
- description: 'A summary of the pause duration of garbage collection cycles',
- targets: [
- {
- expr:
- |||
- sum by(instance)(
- rate(
- go_gc_duration_seconds_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- intervalFactor: 1,
- },
- ],
- },
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletGarbageCollectionDurationQuantiles: vitess_ct.panel.go_gc_seconds {
- title: 'GC Duration quantiles (all hosts)',
- targets: [
- {
- expr:
- |||
- sum by(quantile)(
- rate(
- go_gc_duration_seconds{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: 'p{{quantile}}',
- intervalFactor: 1,
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletMysqlTimeAvgFilteredByInstance: vitess_ct.panel.mysql_timings {
- title: 'MySQL time (avg)',
- targets: [
- {
- expr:
- |||
- sum by (instance) (
- rate(
- vttablet_mysql_sum{
- instance=~"$host"
- }[1m]
- )
- )
- /
- sum by (instance) (
- rate(
- vttablet_mysql_count{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletMysqlExecTimeP50FilterebyInstance: vitess_ct.panel.mysql_timings {
- title: 'MySQL Exec Time P50',
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.50,
- sum by (le, instance) (
- rate(
- vttablet_mysql_bucket{
- operation="Exec",
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletMysqlExecTimeP95FilterebyInstance: vitess_ct.panel.mysql_timings {
- title: 'MySQL Exec Time P95',
- targets: [
- {
- expr: |||
- histogram_quantile(
- 0.95,
- sum by (le, instance) (
- rate(
- vttablet_mysql_bucket{
- operation="Exec",
- instance=~"$host"
- }[1m]
- )
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vtgateToVtTabletCallTimeAvgFilteredByInstance: vitess_ct.panel.vtgate_to_vttablet_calls {
- title: 'VtGate -> VtTablet Call Time (avg)',
- targets: [
- {
- expr:
- |||
- sum by (instance)(
- rate(
- vtgate_vttablet_call_sum[1m]
- )
- )
- /
- sum by (instance)(
- rate(
- vtgate_vttablet_call_count[1m]
- )
- )
- |||,
- legendFormat: '{{instance}}',
- },
- ],
- },
-
- },
-
- singlestats: {
- // _ _ _ _
- // ___(_)_ __ __ _| | ___ ___| |_ __ _| |_ ___
- // / __| | '_ \ / _` | |/ _ \/ __| __/ _` | __/ __|
- // \__ \ | | | | (_| | | __/\__ \ || (_| | |_\__ \
- // |___/_|_| |_|\__, |_|\___||___/\__\__,_|\__|___/
- // |___/
-
- vttabletQPS: {
- title: 'QPS - vttablet',
- datasource: '%(dataSource)s' % config._config,
- format: 'short',
- valueFontSize: '70%',
- valueName: 'current',
- sparklineFull: true,
- sparklineShow: true,
- target:
- {
- expr: |||
- sum (
- vitess_mixin:vttablet_query_counts:rate1m
- )
- |||,
- intervalFactor: 1,
- },
- },
- },
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/heatmaps.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/heatmaps.libsonnet
deleted file mode 100644
index 9a6e5f41527..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/heatmaps.libsonnet
+++ /dev/null
@@ -1,36 +0,0 @@
-// Re-cyclable components for heatmap resources
-local config = import '../../../config.libsonnet';
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-
-local heatmap = grafana.heatmapPanel;
-local prometheus = grafana.prometheus;
-{
- //TODO move to resources/vttablet
- //TODO CREATE A RECORDING RULE FOR THIS PROMETHEUS TARGET
- vttabletQueryTimeDistribution::
- heatmap.new(
- title='Query Time Distribution (Heatmap)',
- description='Shows a heatmap of the histogram bucketing of the time per read query.',
- datasource='%(dataSource)s' % config._config,
- dataFormat='tsbuckets',
- yAxis_format='s',
- color_cardColor='#FF9830',
- color_exponent=0.3,
- color_mode='opacity',
- yAxis_decimals=0,
- ).addTarget(
- prometheus.target(
- |||
- sum by (le) (
- rate(
- vttablet_queries_bucket{
- instance=~"$host"
- }[1m]
- )
- )
- |||,
- format='heatmap',
- legendFormat='{{le}}'
- )
- ),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/dashboard_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/dashboard_helper.libsonnet
deleted file mode 100644
index d998cc65540..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/dashboard_helper.libsonnet
+++ /dev/null
@@ -1,16 +0,0 @@
-/** This is a helper library to load grafonnet dashboards using the mixin metadata stored in `config.libshonnet` */
-
-local grafonnet_helper = import 'grafonnet_helper.libsonnet';
-
-{
- /**
- * Builds a dashboard using grafonnet and the configuration from `config.libsonnet`
- *
- * @name dashboard_helper.getDashboard
- *
- * @param config The dashboard configuration from mixin config file.
- * @return A new graphPanel with the configuration specified in `config.libsonnet`
- *
- */
- getDashboard(config):: grafonnet_helper.getDashboard(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/grafonnet_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/grafonnet_helper.libsonnet
deleted file mode 100644
index 2ff41c6a06f..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/grafonnet_helper.libsonnet
+++ /dev/null
@@ -1,376 +0,0 @@
-/**
- * This is a helper library to generate grafana resources reading the configuration from a config file.
- */
-
-local grafana = import '../../../../vendor/grafonnet/grafana.libsonnet';
-local prometheus = grafana.prometheus;
-local alert_condition = grafana.alertCondition;
-
-// _ _ _
-// __ _ _ __ _ __ ___ | |_ __ _| |_(_) ___ _ __
-// / _` | '_ \| '_ \ / _ \| __/ _` | __| |/ _ \| '_ \
-// | (_| | | | | | | | (_) | || (_| | |_| | (_) | | | |
-// \__,_|_| |_|_| |_|\___/ \__\__,_|\__|_|\___/|_| |_|
-
-local getAnnotation(config) = if (config.name == 'default') then
- grafana.annotation.default
-else
- // TODO when the properties are supported by grafonnet use the lib constructor
- // instead of using composition
- grafana.annotation.datasource(
- config.name,
- config.datasource
- ) +
- config.properties;
-
-// _ _ _ _
-// __| | __ _ ___| |__ | |__ ___ __ _ _ __ __| |
-// / _` |/ _` / __| '_ \| '_ \ / _ \ / _` | '__/ _` |
-// | (_| | (_| \__ \ | | | |_) | (_) | (_| | | | (_| |
-// \__,_|\__,_|___/_| |_|_.__/ \___/ \__,_|_| \__,_|
-
-local getDashboard(config) = grafana.dashboard.new(
- title=config.title,
- description=config.description,
- uid=config.uid,
- time_from=config.time_from,
- tags=(config.dashboardTags),
- editable=true,
- graphTooltip='shared_crosshair',
-) + {
- environments:: config.environments,
-};
-
-// _
-// _ __ __ _ _ __ ___| |
-// | '_ \ / _` | '_ \ / _ \ |
-// | |_) | (_| | | | | __/ |
-// | .__/ \__,_|_| |_|\___|_|
-// |_|
-// The default panel contains all the parameters that we want to override.
-// https://github.com/grafana/grafonnet-lib/blob/master/grafonnet/graph_panel.libsonnet
-
-local default_panel = {
- title: '',
- aliasColors: {},
- bars: false,
- decimals: null,
- description: null,
- format: 'short',
- fill: 1,
- legend_alignAsTable: false,
- legend_avg: false,
- legend_current: false,
- legend_hideZero: null,
- legend_max: false,
- legend_min: false,
- legend_rightSide: false,
- legend_sort: null,
- legend_sortDesc: null,
- legend_values: false,
- lines: true,
- linewidth: 1,
- max: null,
- min: null,
- points: false,
- pointradius: 5,
- nullPointMode: 'null',
- shared_tooltip: true,
- sort: 0,
- thresholds: [],
-};
-
-local default_prometheus_target = {
- format: 'time_series',
- instant: null,
- intervalFactor: 2,
- legendFormat: '',
-};
-
-local default_alert_condition = {
- evaluatorParams: [],
- evaluatorType: 'gt',
- operatorType: 'and',
- queryRefId: 'A',
- queryTimeEnd: 'now',
- queryTimeStart: '5m',
- reducerParams: [],
- reducerType: 'avg',
-};
-
-local getConditions(config) =
- if std.objectHas(config.alert[std.extVar('env')], 'conditions') then
- //reducerType is a grafonnet field value. This asserts the config is not legacy
- if std.objectHas(config.alert[std.extVar('env')].conditions[0], 'reducerType')
- then
- local x = std.map(
- function(c) default_alert_condition + c
- , config.alert[std.extVar('env')].conditions
- );
- std.map(
- function(c)
- alert_condition.new(
- evaluatorParams=c.evaluatorParams,
- evaluatorType=c.evaluatorType,
- operatorType=c.operatorType,
- queryRefId=c.queryRefId,
- queryTimeEnd=c.queryTimeEnd,
- queryTimeStart=c.queryTimeStart,
- reducerParams=c.reducerParams,
- reducerType=c.reducerType,
- )
- , x
- )
- else
- //Legacy config files include calls to grafonnet.alert_condition.new()
- //TODO update legacy config files to use alert conditions in json format,
- config.alert[std.extVar('env')].conditions
- else [];
-
-local getTargets(config) =
- if std.objectHas(config, 'targets') then
- if config.datasource != null &&
- std.startsWith(config.datasource, 'Prometheus') &&
- std.objectHas(config.targets[0], 'expr')
- then
- local x = std.map(
- function(t) default_prometheus_target + t
- , config.targets
- );
- std.map(
- function(t)
- prometheus.target(
- t.expr,
- legendFormat=t.legendFormat,
- instant=t.instant,
- intervalFactor=t.intervalFactor,
- format=t.format
- )
- , x
- )
- else
- //When the datasource is not prometheus(elastic, graphite) config file
- //include calls to graphite.target() and elasticsearch.target().
- //see webapp_config.lisonnet
- //TODO Update this method to decouple grafonnet code from the configuration files.
- //Legacy configuration files include prometheus.target() calls.
- //TODO update legacy config files to use {'expr':'Prom query' ...} format,
- config.targets
- else [];
-
-// This method overriddes grafonnet graphPanel defaults with the values in the config file .
-// https://github.com/grafana/grafonnet-lib/blob/master/grafonnet/graph_panel.libsonnet
-// TODO: When grapPanel supports either addLinks (https://github.com/grafana/grafonnet-lib/pull/278)
-// we should add the links there instead of composing the `options` field.
-local initPanel(config) =
- grafana.graphPanel.new(
- title=config.title,
- aliasColors=config.aliasColors,
- bars=config.bars,
- datasource=config.datasource,
- decimals=config.decimals,
- description=config.description,
- fill=config.fill,
- format=config.format,
- legend_alignAsTable=config.legend_alignAsTable,
- legend_avg=config.legend_avg,
- legend_rightSide=config.legend_rightSide,
- legend_hideZero=config.legend_hideZero,
- legend_min=config.legend_min,
- legend_max=config.legend_max,
- legend_current=config.legend_current,
- legend_sort=config.legend_sort,
- legend_sortDesc=config.legend_sortDesc,
- legend_values=config.legend_values,
- lines=config.lines,
- linewidth=config.linewidth,
- max=config.max,
- min=config.min,
- points=config.points,
- pointradius=config.pointradius,
- nullPointMode=config.nullPointMode,
- shared_tooltip=config.shared_tooltip,
- sort=config.sort,
- thresholds=config.thresholds,
- ).addTargets(
- getTargets(config)
- ) +
- {
- [if std.objectHas(config, 'options')
- then 'options']:
- config.options,
- };
-
-local getPanel(c) =
- if std.objectHas(c, 'alert') then
- local config = default_panel + c;
- local panel = initPanel(config).addAlert(
- config.alert.name,
- executionErrorState=config.alert.executionErrorState,
- forDuration=config.alert.forDuration,
- frequency=config.alert.frequency,
- message=config.alert.message,
- noDataState=config.alert.noDataState,
- notifications=config.alert[std.extVar('env')].notifications,
- ).addConditions(
- getConditions(config)
- );
- if std.objectHas(config, 'seriesOverrides') then
- local it = panel;
- std.foldl(function(p, o) p.addSeriesOverride(o), config.seriesOverrides, it)
- else
- panel
- else
- (local config = default_panel + c;
- local panel = initPanel(config);
- if std.objectHas(config, 'seriesOverrides') then
- local it = panel;
- std.foldl(function(p, o) p.addSeriesOverride(o), config.seriesOverrides, it)
- else
- panel);
-
-// _ __ _____ __
-// | '__/ _ \ \ /\ / /
-// | | | (_) \ V V /
-// |_| \___/ \_/\_/
-
-local row_default = {
- title: '',
- height: null,
- collapse: false,
- repeat: null,
- showTitle: null,
- titleSize: 'h6',
-};
-
-local getRow(c) =
- local config = row_default + c;
- grafana.row.new(
- title=config.title,
- height=config.height,
- collapse=config.collapse,
- repeat=config.repeat,
- showTitle=config.showTitle,
- titleSize=config.titleSize
- );
-
-// _ _ _ _
-// ___(_)_ __ __ _| | ___ ___| |_ __ _| |_
-// / __| | '_ \ / _` | |/ _ \/ __| __/ _` | __|
-// \__ \ | | | | (_| | | __/\__ \ || (_| | |_
-// |___/_|_| |_|\__, |_|\___||___/\__\__,_|\__|
-// |___/
-//The default value should include all the parameters that are overridden by the objects that extend the default.
-//Default values match grafonnet defaults > https://github.com/grafana/grafonnet-lib/blob/master/grafonnet/singlestat.libsonnet
-
-local default_singlestat = {
- colors: [
- '#299c46',
- 'rgba(237, 129, 40, 0.89)',
- '#d44a3a',
- ],
- colorBackground: false,
- decimals: null,
- format: 'none',
- valueFontSize: '80%',
- valueName: 'avg',
- sparklineFull: false,
- sparklineShow: false,
- thresholds: '',
-};
-
-local initSingleStat(config) = grafana.singlestat.new(
- title=config.title,
- datasource=config.datasource,
- colors=config.colors,
- colorBackground=config.colorBackground,
- decimals=config.decimals,
- format=config.format,
- valueFontSize=config.valueFontSize,
- valueName=config.valueName,
- sparklineFull=config.sparklineFull,
- sparklineShow=config.sparklineShow,
- thresholds=config.thresholds,
-);
-
-local getSingleStat(c) = if std.objectHas(c, 'target')
-then
- local config = default_singlestat + c;
- local tc = default_prometheus_target + config.target;
- local t = prometheus.target(
- tc.expr,
- legendFormat=tc.legendFormat,
- instant=tc.instant,
- intervalFactor=tc.intervalFactor,
- format=tc.format
- );
- initSingleStat(config).addTarget(t)
-else
- local config = default_singlestat + c;
- initSingleStat(config);
-
-// _ _ _
-// | |_ ___ _ __ ___ _ __ | | __ _| |_ ___
-// | __/ _ \ '_ ` _ \| '_ \| |/ _` | __/ _ \
-// | || __/ | | | | | |_) | | (_| | || __/
-// \__\___|_| |_| |_| .__/|_|\__,_|\__\___|
-// |_|
-// default values from https://github.com/grafana/grafonnet-lib/blob/master/grafonnet/template.libsonnet
-local template_default = {
- label: null,
- allValues: null,
- tagValuesQuery: '',
- current: null,
- hide: '',
- regex: '',
- refresh: 'never',
- includeAll: false,
- multi: false,
- sort: 0,
-};
-local getTemplate(c) =
- local config = template_default + c;
- grafana.template.new(
- name=config.name,
- datasource=config.datasource,
- query=config.query,
- label=config.label,
- current=config.current,
- regex=config.regex,
- refresh=config.refresh,
- sort=config.sort,
- );
-
-// _ _ _ _ _ _ _
-// __| | __ _ ___| |__ | |__ ___ __ _ _ __ __| | | (_)_ __ | | __
-// / _` |/ _` / __| '_ \| '_ \ / _ \ / _` | '__/ _` | | | | '_ \| |/ /
-// | (_| | (_| \__ \ | | | |_) | (_) | (_| | | | (_| | | | | | | | <
-// \__,_|\__,_|___/_| |_|_.__/ \___/ \__,_|_| \__,_| |_|_|_| |_|_|\_\
-local link_default = {
- asDropdown: true,
- includeVars: false,
- keepTime: false,
- icon: 'external link',
- url: '',
- targetBlank: false,
- type: 'dashboards',
-};
-
-local getDashboardLink(c) =
- local config = link_default + c;
- grafana.link.dashboards(
- title=config.title,
- tags=config.tags,
- keepTime=config.keepTime,
- includeVars=config.includeVars,
- );
-
-{
- getAnnotation(config):: getAnnotation(config),
- getDashboard(config):: getDashboard(config),
- getPanel(config):: getPanel(config),
- getRow(config):: getRow(config),
- getSingleStat(config):: getSingleStat(config),
- getTemplate(config):: getTemplate(config),
- getDashboardLink(config):: getDashboardLink(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/helpers.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/helpers.libsonnet
deleted file mode 100644
index a6d74985dd3..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/helpers.libsonnet
+++ /dev/null
@@ -1,9 +0,0 @@
-// TODO deduplicate helper code. Same/similar functions are used by vtgate, vttablet and orchestrator helpers
-{
- dashboard:: import 'dashboard_helper.libsonnet',
- default:: import 'grafonnet_helper.libsonnet',
- mysql:: import 'mysql_helper.libsonnet',
- os:: import 'os_helper.libsonnet',
- vtgate:: import 'vtgate_helper.libsonnet',
- vttablet:: import 'vttablet_helper.libsonnet',
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/mysql_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/mysql_helper.libsonnet
deleted file mode 100644
index 3d0b1ff6a44..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/mysql_helper.libsonnet
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * This is a helper library to generate os resources reading the configuration from mysql_config.libsonnet
- */
-
-local grafonnet_helper = import './grafonnet_helper.libsonnet';
-
-/**
- * Builds grapPanel using grafonnet and the configuration from `mysql_config.libsonnet`
- *
- * @name mysql_helper.getPanel
- *
- * @param `config`: The panel configuration from mysql_config file.
- *
- * @return A new graphPanel with the configuration specified in `mysql_config.libsonnet`
- *
- */
-{
- getPanel(config):: grafonnet_helper.getPanel(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/os_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/os_helper.libsonnet
deleted file mode 100644
index 2580d54e43a..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/os_helper.libsonnet
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * This is a helper library to generate os resources reading the configuration from os_config.libsonnet
- */
-
-local grafonnet_helper = import './grafonnet_helper.libsonnet';
-
-{
- /**
- * Builds grapPanel using grafonnet and the configuration from `os_config.libsonnet`
- *
- * @name os_helper.getPanel
- *
- * @param `config`: The panel configuration from os_config file.
- *
- * @return A new graphPanel with the configuration specified in `os_config.libsonnet`
- *
- */
- getPanel(config):: grafonnet_helper.getPanel(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/vtgate_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/vtgate_helper.libsonnet
deleted file mode 100644
index 2997979edf4..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/vtgate_helper.libsonnet
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * This is a helper library to generate vtgate resources reading the configuration from vtgate_config.libsonnet
- */
-
-local grafonnet_helper = import './grafonnet_helper.libsonnet';
-
-{
- /**
- * Builds grapPanel using grafonnet and the configuration from `vtgate_config.libsonnet`
- *
- * @name vtgate_helper.getPanel
- *
- * @param `config`: The panel configuration from vtgate_config file.
- *
- * @return A new graphPanel with the configuration specified in `vtgate_config.libsonnet`
- *
- */
- getPanel(config):: grafonnet_helper.getPanel(config),
-
- /**
- * Builds a singlestat using grafonnet and the configuration from `vtgate_config.libsonnet`
- *
- * @name vtgate_helper.getSingleStat
- *
- * @param `config`: The singlestat configuration from vtgate_config file.
- *
- * @return A new singlestat with the configuration specified in `vtgate_config.libsonnet`
- *
- */
- getSingleStat(config):: grafonnet_helper.getSingleStat(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/helpers/vttablet_helper.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/helpers/vttablet_helper.libsonnet
deleted file mode 100644
index 2613de0f094..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/helpers/vttablet_helper.libsonnet
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * This is a helper library to generate vttablet resources reading the configuration from `vttablet_config.libsonnet`
- */
-local grafonnet_helper = import './grafonnet_helper.libsonnet';
-
-{
- /**
- * Builds grapPanel using grafonnet and the configuration from `vttablet_config.libsonnet`
- *
- * @name vttablet_helper.getPanel
- *
- * @param `config`: The panel configuration from vttablet_config file.
- *
- * @return A new graphPanel with the configuration specified in `vttablet_config.libsonnet`
- *
- */
- getPanel(config):: grafonnet_helper.getPanel(config),
- /**
- * Builds singlestat using grafonnet and the configuration from `vttablet_config.libsonnet`
- *
- * @name vttablet_helper.getPanel
- *
- * @param `config`: The singlestat configuration from vttablet_config file.
- *
- * @return A new singlestat with the configuration specified in `vttablet_config.libsonnet`
- *
- */
- getSingleStat(config):: grafonnet_helper.getSingleStat(config),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/panels.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/panels.libsonnet
deleted file mode 100644
index 40c20a2cb10..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/panels.libsonnet
+++ /dev/null
@@ -1,65 +0,0 @@
-// Re-cyclable components for panel resources
-local config = import '../../../config.libsonnet';
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-
-local graphPanel = grafana.graphPanel;
-local prometheus = grafana.prometheus;
-
-// TODO: add description for each panel.
-// TODO: create a _helper _config file for each group [vtctld],
-{
- // _ _ _ _
- // __ _| |_ ___| |_| | __| |
- // \ \ / / __/ __| __| |/ _` |
- // \ V /| || (__| |_| | (_| |
- // \_/ \__\___|\__|_|\__,_|
- //
-
- // _
- // _ __ ___ (_)___ ___
- // | '_ ` _ \| / __|/ __|
- // | | | | | | \__ \ (__
- // |_| |_| |_|_|___/\___|
- //
- local default_notification_config = {
- prod+: {
- notifications: [
- { uid: 'alerts-vitess' },
- { uid: 'pagerduty-vitess' },
- ],
- },
- dev+: {
- notifications: [
- { uid: 'alerts-vitess-dev' },
- ],
- },
- },
-
- vtctldRestart::
- graphPanel.new(
- 'vtctld',
- bars=true,
- datasource='%(dataSource)s' % config._config,
- fill=0,
- format='short',
- legend_values=true,
- legend_alignAsTable=true,
- legend_max=true,
- legend_sort='max',
- legend_sortDesc=false,
- lines=false,
- min=0,
- shared_tooltip=false,
- sort='increasing',
- )
- .addTarget(prometheus.target(
- |||
- sum by (instance) (
- vitess_mixin:process_start_time_seconds_by_instance_job:sum5m{
- %(vtctldSelector)s
- }
- ) > 0
- ||| % config._config,
- legendFormat='{{instance}}'
- )),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/rows.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/rows.libsonnet
deleted file mode 100644
index 54173c2a3f1..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/rows.libsonnet
+++ /dev/null
@@ -1,182 +0,0 @@
-// Re-cyclable components for row resources
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-local row = grafana.row;
-
-//TODO move all rows to config/row_config.libsonnet and update the layouts to use grafonnet_helper.getRow()
-{
- connection::
- row.new(
- title='Connection',
- ),
-
- cpu::
- row.new(
- title='CPU',
- collapse=true,
- ),
-
- duration::
- row.new(
- title='Duration',
- collapse=true,
- ),
-
- errorsRowsReturned::
- row.new(
- title='Errors / Rows returned',
- ),
-
- errors::
- row.new(
- title='Errors',
- collapse=true,
- ),
-
- healthcheck::
- row.new(
- title='Healthcheck',
- collapse=true,
- ),
-
- tabletsQueries::
- row.new(
- title='Tablets/Queries',
- ),
-
- mysql::
- row.new(
- title='MySQL',
- collapse=true,
- ),
-
- misc::
- row.new(
- title='Misc',
- collapse=true,
- ),
-
- networkingTCP::
- row.new(
- title='Networking TCP',
- collapse=true,
- ),
-
- networkNIC::
- row.new(
- title='Network NIC',
- collapse=true,
- ),
-
- OS::
- row.new(
- title='OS',
- collapse=true,
- ),
-
- processes::
- row.new(
- title='Processes',
- collapse=true,
- ),
-
- queryTimings::
- row.new(
- // as we don't have timings by table (yet!)
- title="Query/Transaction timings (table filter doesn't apply)",
- collapse=true,
- ),
-
- query::
- row.new(
- title='Query',
- ),
-
- RED::
- row.new(
- title='RED - Requests / Error rate / Duration',
- ),
-
- REDByKeyspace::
- row.new(
- title='RED (by keyspace)',
- collapse=true
- ),
-
- REDByTabletType::
- row.new(
- title='RED (by tablet type)',
- collapse=true
- ),
-
- REDByPlanType::
- row.new(
- title='RED (by plan type)',
- collapse=true
- ),
-
- REDByShard::
- row.new(
- title='RED (by shard)',
- collapse=true
- ),
-
- REDByTable::
- row.new(
- title='RED (by table)',
- collapse=true
- ),
-
-
- rowsReturned::
- row.new(
- title='Rows returned',
- collapse=true,
- ),
-
- serviceRestart::
- row.new(
- title='Service restart',
- ),
-
- storage::
- row.new(
- title='Storage',
- collapse=true,
- ),
-
- topLevel::
- row.new(
- title='Top level',
- ),
-
- topologyWatcher::
- row.new(
- title='Topology watcher',
- collapse=true,
- ),
-
- vitessQueryPool::
- row.new(
- title='Vitess - Query pool',
- collapse=true,
- ),
-
- vitessTransactionPool::
- row.new(
- title='Vitess - Transaction pool',
- collapse=true,
- ),
-
- vtgate::
- row.new(
- title='vtgate - Requests (by table / by plan / by keyspace )',
- collapse=true,
- ),
-
- vttablet::
- row.new(
- title='vttablet - Requests (by table / by plan / by keyspace )',
- collapse=true,
- ),
-
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/singlestats.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/singlestats.libsonnet
deleted file mode 100644
index e3930e382d8..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/singlestats.libsonnet
+++ /dev/null
@@ -1,221 +0,0 @@
-// Re-cyclable components for singlestat resources
-local config = import '../../../config.libsonnet';
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-
-local singlestat = grafana.singlestat;
-local prometheus = grafana.prometheus;
-
-{
- //TODO move to resource to use vtgate_config/vtgate_helper
- vtgateSuccessRate::
- singlestat.new(
- 'Query success - vtgate',
- datasource='%(dataSource)s' % config._config,
- colorBackground=true,
- decimals=4,
- format='percent',
- colors=[
- '#d44a3a',
- 'rgba(237, 129, 40, 0.89)',
- '#299c46',
- ],
- valueFontSize='70%',
- valueName='current',
- thresholds='0.99,0.999',
- )
- .addTarget(
- prometheus.target(
- |||
- 100
- -
- sum(
- rate(
- vtgate_api_error_counts{
- %(vtgateSelector)s
- }[$interval]
- ) OR vector(0)
- )
- /
- sum(
- rate(
- vtgate_api_count{
- %(vtgateSelector)s
- }[$interval]
- )
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
- //TODO move to resource to use vtgate_config/vtgate_helper
- vtgateUp::
- singlestat.new(
- 'vtgate',
- datasource='%(dataSource)s' % config._config,
- valueFontSize='50%',
- valueName='current',
- )
- .addTarget(
- prometheus.target(
- |||
- sum(
- up{
- %(vtgateSelector)s
- }
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
- //TODO move to resource to use vttablet_config/vttablet_helper
- vttabletQuerySuccess::
- singlestat.new(
- 'Query success - vttablet',
- datasource='%(dataSource)s' % config._config,
- colorBackground=true,
- decimals=4,
- format='percent',
- colors=[
- '#d44a3a',
- 'rgba(237, 129, 40, 0.89)',
- '#299c46',
- ],
- valueFontSize='70%',
- valueName='current',
- thresholds='0.99,0.999',
- )
- .addTarget(
- prometheus.target(
- |||
- 100
- -
- (
- sum (
- vitess_mixin:vttablet_errors:rate1m
- )
- /
- sum (
- vitess_mixin:vttablet_query_counts:rate1m
- )
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
- //TODO move to resource to use vttablet_config/vttablet_helper
- vttabletUp::
- singlestat.new(
- 'vttablet',
- datasource='%(dataSource)s' % config._config,
- valueFontSize='50%',
- valueName='current',
- )
- .addTarget(
- prometheus.target(
- |||
- sum(
- up{
- %(vttabletSelector)s
- }
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
-
- //TODO move to resource to use vttablet_config/vttablet_helper
- keyspaceCount::
- singlestat.new(
- 'keyspace',
- description='count of keyspaces with active queries',
- datasource='%(dataSource)s' % config._config,
- valueFontSize='50%',
- valueName='current',
- )
- .addTarget(
- prometheus.target(
- |||
- count(
- count by (keyspace)(
- vtgate_vttablet_call_count{
- }
- )
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
- //TODO move to resource to use vttablet_config/vttablet_helper
- shardCount::
- singlestat.new(
- 'shard',
- datasource='%(dataSource)s' % config._config,
- valueFontSize='50%',
- valueName='current',
- )
- .addTarget(
- prometheus.target(
- |||
- count(
- count by(shard)(
- vttablet_tablet_state{
- %(vttabletSelector)s
- }
- )
- )
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-
- mysqlQPS::
- singlestat.new(
- 'QPS - MySQL',
- datasource='%(dataSource)s' % config._config,
- format='short',
- valueFontSize='70%',
- valueName='current',
- sparklineFull=true,
- sparklineShow=true,
- )
- .addTarget(
- prometheus.target(
- |||
- sum (
- vitess_mixin:mysql_global_status_queries:rate1m
- )
- |||,
- intervalFactor=1
- )
- ),
-
- vtctldUp::
- singlestat.new(
- 'vtctld',
- datasource='%(dataSource)s' % config._config,
- valueFontSize='50%',
- valueName='current',
- )
- .addTarget(
- prometheus.target(
- |||
- sum(
- up{
- %(vtctldSelector)s})
- ||| % config._config,
- instant=true,
- intervalFactor=1
- )
- ),
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/templates.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/templates.libsonnet
deleted file mode 100644
index a32b71283c4..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/templates.libsonnet
+++ /dev/null
@@ -1,51 +0,0 @@
-// Re-cyclable components for template resources
-local config = import '../../../config.libsonnet';
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-
-local template = grafana.template;
-{
- interval:: template.interval(
- name='interval',
- label='Interval',
- query='auto,1m,5m,10m,30m,1h,6h,12h',
- current='auto',
- auto_min='1m'
- ),
-
- table::
- template.new(
- 'table',
- '%(dataSource)s' % config._config,
- 'query_result(sum by(table)(vitess_mixin:vtgate_queries_processed_by_table:rate1m{keyspace="$keyspace"}))',
- regex='.*table="(.*)".*',
- label='Table',
- refresh='time',
- includeAll=true,
- sort=1,
- allValues='.*',
- ),
-
- hostVtgate::
- template.new(
- 'host',
- '%(dataSource)s' % config._config,
- 'label_values(vtgate_build_number, instance)',
- label='Host(s)',
- refresh='time',
- multi=true,
- allValues='.*',
- ),
-
- hostVttablet::
- template.new(
- 'host',
- '%(dataSource)s' % config._config,
- 'label_values(vttablet_build_number{}, instance)',
- label='Host(s)',
- refresh='time',
- multi=true,
- allValues='.*',
- sort=1
- ),
-
-}
diff --git a/vitess-mixin/dashboards/resources/grafonnet/texts.libsonnet b/vitess-mixin/dashboards/resources/grafonnet/texts.libsonnet
deleted file mode 100644
index 44478b7a13f..00000000000
--- a/vitess-mixin/dashboards/resources/grafonnet/texts.libsonnet
+++ /dev/null
@@ -1,65 +0,0 @@
-// Re-cyclable components for text resources
-local config = import '../../../config.libsonnet';
-local grafana = import '../../../vendor/grafonnet/grafana.libsonnet';
-
-local text = grafana.text;
-
-// TODO: figure out how to make emoji work in jsonnet. They are not correctly handled
-{
-
- local footnote =
- |||
- This Dasboard has been automatically generated using vitess-mixin.
- If you want to contribute please visit [https://github.com/vitess/vitess-mixin](https://github.com/vitessio/vitess/tree/main/vitess-mixin)!
- |||,
-
- local notes = {
- footnote: footnote,
- },
-
- clusterOverview::
- text.new(
- '',
- mode='markdown',
- content=|||
- #### Cluster overview
-
- This is a general overview of the Vitess clusters.
-
- %(footnote)s
- ||| % notes
- ),
-
- vtgateOverview::
- text.new(
- '',
- mode='markdown',
- content=|||
- #### vtgate overview
-
- This is a general overview of the vtgate tier.
-
- %(footnote)s
- ||| % notes
- ),
-
- vtgateHost::
- text.new(
- '',
- mode='markdown',
- content=|||
- #### vtgate host
- %s
- ||| % footnote,
- ),
-
- vttabletHost::
- text.new(
- '',
- mode='markdown',
- content=|||
- #### vttablet host
- %s
- ||| % footnote,
- ),
-}
diff --git a/vitess-mixin/e2e/.env b/vitess-mixin/e2e/.env
deleted file mode 100644
index 00e3ed4bf0a..00000000000
--- a/vitess-mixin/e2e/.env
+++ /dev/null
@@ -1,18 +0,0 @@
-TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500 -topo_global_root vitess/global
-GRPC_PORT=15999
-WEB_PORT=8080
-MYSQL_PORT=15306
-
-CELL=local
-KEYSPACE=commerce
-DB=commerce
-
-EXTERNAL_DB=0
-DB_HOST=external_db_host
-DB_PORT=3306
-DB_USER=external_db_user
-DB_PASS=external_db_password
-DB_CHARSET=CHARACTER SET utf8 COLLATE utf8_general_ci
-
-GRAFANA_VERSION=7.3.6
-DISPLAY=
\ No newline at end of file
diff --git a/vitess-mixin/e2e/Dockerfile b/vitess-mixin/e2e/Dockerfile
deleted file mode 100644
index cf4a4a9f85e..00000000000
--- a/vitess-mixin/e2e/Dockerfile
+++ /dev/null
@@ -1,18 +0,0 @@
-ARG CYPRESS_IMAGE
-
-FROM $CYPRESS_IMAGE
-WORKDIR /e2e
-
-# dependencies will be installed only if the package files change
-COPY package.json .
-COPY package-lock.json .
-
-# by setting CI environment variable we switch the Cypress install messages
-# to small "started / finished" and avoid 1000s of lines of progress messages
-# https://github.com/cypress-io/cypress/issues/1243
-ENV CI=1
-RUN npm ci
-# verify that Cypress has been installed correctly.
-# running this command separately from "cypress run" will also cache its result
-# to avoid verifying again when running the tests
-RUN npx cypress verify
diff --git a/vitess-mixin/e2e/backups/.gitignore b/vitess-mixin/e2e/backups/.gitignore
deleted file mode 100644
index 35bc8010eb9..00000000000
--- a/vitess-mixin/e2e/backups/.gitignore
+++ /dev/null
@@ -1,4 +0,0 @@
-# ignore all files in this dir...
-*
-# ... except for this one.
-!.gitignore
diff --git a/vitess-mixin/e2e/config/init_db.sql b/vitess-mixin/e2e/config/init_db.sql
deleted file mode 100644
index 6059bbf7ca6..00000000000
--- a/vitess-mixin/e2e/config/init_db.sql
+++ /dev/null
@@ -1,79 +0,0 @@
-# This file is executed immediately after mysql_install_db,
-# to initialize a fresh data directory.
-###############################################################################
-# Equivalent of mysql_secure_installation
-###############################################################################
-# We need to ensure that super_read_only is disabled so that we can execute
-# these commands. Note that disabling it does NOT disable read_only.
-# We save the current value so that we only re-enable it at the end if it was
-# enabled before.
-SET @original_super_read_only=IF(@@global.super_read_only=1, 'ON', 'OFF');
-SET GLOBAL super_read_only='OFF';
-# Changes during the init db should not make it to the binlog.
-# They could potentially create errant transactions on replicas.
-SET sql_log_bin = 0;
-# Remove anonymous users & disable remote root access (only allow UNIX socket).
-DROP USER IF EXISTS ''@'%', ''@'localhost', 'root'@'%';
-# Remove test database.
-DROP DATABASE IF EXISTS test;
-###############################################################################
-# Vitess defaults
-###############################################################################
-# Vitess-internal database.
-CREATE DATABASE IF NOT EXISTS _vt;
-# Note that definitions of local_metadata and shard_metadata should be the same
-# as in production which is defined in go/vt/mysqlctl/metadata_tables.go.
-CREATE TABLE IF NOT EXISTS _vt.local_metadata (
- name VARCHAR(255) NOT NULL,
- value VARCHAR(255) NOT NULL,
- db_name VARBINARY(255) NOT NULL,
- PRIMARY KEY (db_name, name)
- ) ENGINE=InnoDB;
-CREATE TABLE IF NOT EXISTS _vt.shard_metadata (
- name VARCHAR(255) NOT NULL,
- value MEDIUMBLOB NOT NULL,
- db_name VARBINARY(255) NOT NULL,
- PRIMARY KEY (db_name, name)
- ) ENGINE=InnoDB;
-# Admin user with all privileges.
-CREATE USER 'vt_dba'@'localhost';
-GRANT ALL ON *.* TO 'vt_dba'@'localhost';
-GRANT GRANT OPTION ON *.* TO 'vt_dba'@'localhost';
-# User for app traffic, with global read-write access.
-CREATE USER 'vt_app'@'localhost';
-GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE,
- REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES,
- LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW,
- SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER
- ON *.* TO 'vt_app'@'localhost';
-# User for app debug traffic, with global read access.
-CREATE USER 'vt_appdebug'@'localhost';
-GRANT SELECT, SHOW DATABASES, PROCESS ON *.* TO 'vt_appdebug'@'localhost';
-# User for administrative operations that need to be executed as non-SUPER.
-# Same permissions as vt_app here.
-CREATE USER 'vt_allprivs'@'localhost';
-GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE,
- REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES,
- LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW,
- SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER
- ON *.* TO 'vt_allprivs'@'localhost';
-# User for slave replication connections.
-# TODO: Should we set a password on this since it allows remote connections?
-CREATE USER 'vt_repl'@'%';
-GRANT REPLICATION SLAVE ON *.* TO 'vt_repl'@'%';
-# User for Vitess filtered replication (binlog player).
-# Same permissions as vt_app.
-CREATE USER 'vt_filtered'@'localhost';
-GRANT SELECT, INSERT, UPDATE, DELETE, CREATE, DROP, RELOAD, PROCESS, FILE,
- REFERENCES, INDEX, ALTER, SHOW DATABASES, CREATE TEMPORARY TABLES,
- LOCK TABLES, EXECUTE, REPLICATION SLAVE, REPLICATION CLIENT, CREATE VIEW,
- SHOW VIEW, CREATE ROUTINE, ALTER ROUTINE, CREATE USER, EVENT, TRIGGER
- ON *.* TO 'vt_filtered'@'localhost';
-
-RESET SLAVE ALL;
-RESET MASTER;
-# custom sql is used to add custom scripts like creating users/passwords. We use it in our tests
-# {{custom_sql}}
-
-# We need to set super_read_only back to what it was before
-SET GLOBAL super_read_only=IFNULL(@original_super_read_only, 'ON');
diff --git a/vitess-mixin/e2e/cypress.json b/vitess-mixin/e2e/cypress.json
deleted file mode 100644
index c1ffae0a972..00000000000
--- a/vitess-mixin/e2e/cypress.json
+++ /dev/null
@@ -1,4 +0,0 @@
-{
- "viewportWidth": 1200,
- "viewportHeight": 900
-}
diff --git a/vitess-mixin/e2e/cypress/integration/cluster_overview_spec.js b/vitess-mixin/e2e/cypress/integration/cluster_overview_spec.js
deleted file mode 100644
index 5bb40a68c3b..00000000000
--- a/vitess-mixin/e2e/cypress/integration/cluster_overview_spec.js
+++ /dev/null
@@ -1,56 +0,0 @@
-const fs = require('fs')
-
-describe('vitess-mixin: Cluster Overview Dashboard Test', function () {
-
- let panelTitles = []
-
- before(function () {
- cy.readFile('./test/cluster_overview.json').then((data) => {
- cy.createDashboard(data)
- })
- })
- it('renders cluster overview dashboard', function () {
- cy.visit('/d/vitess-cluster-overview/cluster-overview-auto-generated')
- })
- //TEMPLATES
- it('contains the Interval Template', function () {
- cy.get('.submenu-controls').contains('Interval');
- })
- //LINKS
- it('contains vitess-mixin Dashboard links dropdown', function () {
- cy.get('.submenu-controls').get('.gf-form').contains('vitess-mixin')
- })
- // INIT PANELS
- it('checks all panels in the cluster overview dashboard exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(22)
- expect(titles).to.deep.eq([
- '',
- 'Query success - vtgate',
- 'Query success - vttablet',
- 'Query latency p99',
- 'QPS - vtgate',
- 'QPS - MySQL',
- 'vtgate',
- 'vttablet',
- 'QPS - vttablet',
- 'keyspace',
- 'shard',
- 'vtctld',
- 'Requests',
- 'Error rate',
- 'Duration 99th quantile',
- 'Requests (by keyspace)',
- 'Error rate (by keyspace)',
- 'Duration 99th quantile (by keyspace)',
- '# of serving tablets',
- 'Slow queries',
- 'Query/Transaction killed'
- ])
- })
- })
-})
diff --git a/vitess-mixin/e2e/cypress/integration/vtgate_host_view.js b/vitess-mixin/e2e/cypress/integration/vtgate_host_view.js
deleted file mode 100644
index 7cbbcdb0e9a..00000000000
--- a/vitess-mixin/e2e/cypress/integration/vtgate_host_view.js
+++ /dev/null
@@ -1,174 +0,0 @@
-const fs = require('fs')
-
-describe('vitess-mixin: Vtgate Host View Dashboard Test', function() {
-
- let panelTitles = []
-
- before(function() {
- cy.readFile('./test/vtgate_host_view.json').then((data) => {
- cy.createDashboard(data)
- })
- })
- it('renders vtgate hostview dashboard', function() {
- cy.visit('/d/vitess-vtgate-host-view/vtgate - host view (auto-generated)')
- })
- //SUB-MENU CONTROLS
- it('contains the Host(s) Template', function() {
- cy.get('.submenu-controls').contains('Host(s)');
- })
- it('contains vitess-mixin Dashboard links dropdown', function() {
- cy.get('.submenu-controls').get('.gf-form').contains('vitess-mixin')
- })
- //Rows are loaded at start time
- it('contains 5 rows', function() {
- cy.get('.dashboard-row').should('have.length',5)
- })
- // RED ROW
- it('contains the RED row', function() {
- cy.get('.dashboard-row').contains('RED - Requests / Error rate / Duration')
- })
-
- it('checks all panels in the RED row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests',
- 'Error rate',
- 'Duration 99th quantile'
- ])
- })
- })
-
- it('collapses the RED row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED - Requests / Error rate / Duration').click();
- })
-
- // ROW (BY TABLET TYPE)
- it('contains the RED (by tablet type) row', function() {
- cy.get('.dashboard-row').contains('RED (by tablet type)');
- })
- it('RED (by tablet type) row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('RED (by tablet type)');
- })
- it('expands the RED (by tablet type) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by tablet type)').click();
- })
-
- it('checks all panels in the RED (by tablet type) row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests (by db_type)',
- 'Error rate (by db_type)',
- 'Duration 99th quantile (by db_type)'
- ])
- })
- })
-
- it('collapses the RED (by tablet type) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by tablet type)').click();
- cy.get('.dashboard-row--collapsed').contains('RED (by tablet type)');
- })
-
- //ERRORS ROW
- it('contains the Errors row', function() {
- cy.get('.dashboard-row').contains('Errors');
- })
- it('Errors row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('Errors');
- })
- it('expands the Errors row', function(){
- cy.get('.dashboard-row__title.pointer').contains('Errors').scrollIntoView().click();
- })
- it('checks all panels in the Errors Row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(3)
- expect(titles).to.deep.eq([
- '',
- 'Errors (by keyspace)',
- 'Errors (by code)'
- ])
- })
- })
- it('collapses the Errors row', function(){
- cy.get('.dashboard-row__title.pointer').contains('Errors').click();
- cy.get('.dashboard-row--collapsed').contains('Errors');
- })
- //DURATION ROW
- it('contains the Duration row', function() {
- cy.get('.dashboard-row').contains(/^Duration/)
- })
- it('Duration row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains(/^Duration/)
- })
- it('expands the Duration row', function(){
- cy.get('.dashboard-row__title.pointer').contains(/^Duration/).click();
- })
-
- it('checks all panels in the Duration row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Duration (Avg)',
- 'Duration 50th quantile',
- 'Duration 95th quantile'
- ])
- })
- })
-
- it('collapses the Duration row', function(){
- cy.get('.dashboard-row__title.pointer').contains(/^Duration/).click();
- cy.get('.dashboard-row--collapsed').contains(/^Duration/);
- })
- //OS ROW
- it('contains the OS row', function() {
- cy.get('.dashboard-row').contains('OS')
- })
- it('OS row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('OS')
- })
- it('expands the OS row', function(){
- cy.get('.dashboard-row__title.pointer').contains('OS').click();
- })
-
- it('checks all panels in the OS row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(5)
- expect(titles).to.deep.eq([
- '',
- 'CPU Usage',
- 'Memory Usage',
- 'Network Usage',
- 'TCP Retransmissions'
- ])
- })
- })
-
- it('collapses the OS row', function(){
- cy.get('.dashboard-row__title.pointer').contains('OS').click();
- cy.get('.dashboard-row--collapsed').contains('OS');
- })
-})
diff --git a/vitess-mixin/e2e/cypress/integration/vtgate_overview.js b/vitess-mixin/e2e/cypress/integration/vtgate_overview.js
deleted file mode 100644
index eb8122ca403..00000000000
--- a/vitess-mixin/e2e/cypress/integration/vtgate_overview.js
+++ /dev/null
@@ -1,201 +0,0 @@
-const fs = require('fs')
-
-describe('vitess-mixin: Vtgate Overview Dashboard Test', function() {
-
- let panelTitles = []
-
- before(function() {
- cy.readFile('./test/vtgate_overview.json').then((data) => {
- cy.createDashboard(data)
- })
- })
- it('renders vtgate overview dashboard', function() {
- cy.visit('/d/vitess-vtgate-overview/vtgate - overview (auto-generated)')
- })
- //SUB-MENU CONTROLS
- it('contains vitess-mixin Dashboard links dropdown', function() {
- cy.get('.submenu-controls').get('.gf-form').contains('vitess-mixin');
- })
- // RED ROW
- it('contains the RED row', function() {
- cy.get('.dashboard-row').contains('RED - Requests / Error rate / Duration');
- })
-
- it('checks all panels in the RED row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests',
- 'Error rate',
- 'Duration 99th quantile'
- ])
- })
- })
-
- it('collapses the RED row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED - Requests / Error rate / Duration').click();
- })
-
- // RED (BY KEYSPACE) ROW
- it('contains the RED (by keyspace) row', function() {
- cy.get('.dashboard-row').contains('RED (by keyspace)');
- })
- it('RED (by keyspace) row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('RED (by keyspace)');
- })
- it('expands the RED (by keyspace) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by keyspace)').click();
- })
-
- it('checks all panels in the RED (by keyspace) row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests (by keyspace)',
- 'Error rate (by keyspace)',
- 'Duration 99th quantile (by keyspace)'
- ])
- })
- })
-
- it('collapses the RED (by keyspace) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by keyspace)').click();
- })
-
- // ROW (BY TABLET TYPE)
- it('contains the RED (by tablet type) row', function() {
- cy.get('.dashboard-row').contains('RED (by tablet type)');
- })
- it('RED (by tablet type) row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('RED (by tablet type)');
- })
- it('expands the RED (by tablet type) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by tablet type)').click();
- })
-
- it('checks all panels in the RED (By tablet type) row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests (by db_type)',
- 'Error rate (by db_type)',
- 'Duration 99th quantile (by db_type)'
- ])
- })
- })
-
- it('collapses the RED (by tablet type) row', function(){
- cy.get('.dashboard-row__title.pointer').contains('RED (by tablet type)').click();
- cy.get('.dashboard-row--collapsed').contains('RED (by tablet type)');
- })
-
- //ERRORS ROW
- it('contains the Errors row', function() {
- cy.get('.dashboard-row').contains('Errors');
- })
- it('Errors row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('Errors');
- })
- it('expands the Errors row', function(){
- cy.get('.dashboard-row__title.pointer').contains('Errors').click();
- })
-
- it('checks all panels in the Errors row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Errors (by code)',
- 'Errors (by operation)',
- 'Errors (by db_type)'
- ])
- })
- })
-
- it('collapses the Errors row', function(){
- cy.get('.dashboard-row__title.pointer').contains('Errors').click();
- })
-
- //DURATION ROW
- it('contains the Duration row', function() {
- cy.get('.dashboard-row').contains(/^Duration/);
- })
- it('Duration row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains(/^Duration/);
- })
- it('expands the Duration row', function(){
- cy.get('.dashboard-row__title.pointer').contains(/^Duration/).click();
- })
-
- it('checks all panels in the Duration row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Duration (Avg)',
- 'Duration 50th quantile',
- 'Duration 95th quantile'
- ])
- })
- })
-
- it('collapses the Duration row', function(){
- cy.get('.dashboard-row__title.pointer').contains(/^Duration/).click();
- })
-
- //OS ROW
- it('contains the OS row', function() {
- cy.get('.dashboard-row').contains('OS');
- })
- it('OS row is collapsed', function() {
- cy.get('.dashboard-row--collapsed').contains('OS');
- })
- it('expands the OS row', function(){
- cy.get('.dashboard-row__title.pointer').contains('OS').click();
- })
-
- it('checks all panels in the row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(5)
- expect(titles).to.deep.eq([
- '',
- 'CPU Usage',
- 'Memory Usage',
- 'Network Usage',
- 'TCP Retransmissions'
- ])
- })
- })
-
- it('collapses the OS row', function(){
- cy.get('.dashboard-row__title.pointer').contains('OS').click();
- cy.get('.dashboard-row--collapsed').contains('OS');
- })
-})
diff --git a/vitess-mixin/e2e/cypress/integration/vttablet_host_view.js b/vitess-mixin/e2e/cypress/integration/vttablet_host_view.js
deleted file mode 100644
index f98ba803d85..00000000000
--- a/vitess-mixin/e2e/cypress/integration/vttablet_host_view.js
+++ /dev/null
@@ -1,409 +0,0 @@
-const fs = require('fs')
-
-describe('vitess-mixin: Vttablet Host View Dashboard Test', function () {
-
- let panelTitles = []
-
- before(function () {
- cy.readFile('./test/vttablet_host_view.json').then((data) => {
- cy.createDashboard(data)
- })
- })
- it('renders vttablet hostview dashboard', function () {
- cy.visit('/d/vitess-vttablet-host-view/vttablet - host view (auto-generated)')
- })
- //SUB-MENU CONTROLS
- it('contains the Host(s) Template', function () {
- cy.get('.submenu-controls').contains('Host(s)');
- })
- it('contains vitess-mixin Dashboard links dropdown', function () {
- cy.get('.submenu-controls').get('.gf-form').contains('vitess-mixin')
- })
- //All Rows are loaded at start time
- it('contains 12 rows', function () {
- cy.get('.dashboard-row').should('have.length', 12)
- })
-
- // RED ROW
- it('contains the RED row', function () {
- cy.get('.dashboard-row').contains('RED - Requests / Error rate / Duration')
- })
-
- it('checks all panels in the RED row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests',
- 'Error rate',
- 'Duration (p99)'
- ])
- })
- })
-
- it('collapses the RED row', function () {
- cy.get('.dashboard-row__title.pointer').contains('RED - Requests / Error rate / Duration').click();
- })
- // RED BY PLAN TYPE ROW
-
- it('contains the RED (by plan type) row', function () {
- cy.get('.dashboard-row').contains('RED (by plan type)')
- })
- it('RED (by plan type) row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('RED (by plan type)')
- })
- it('expands the RED (by plan type) row', function () {
- cy.get('.dashboard-row__title.pointer').contains('RED (by plan type)').click();
- })
-
- it('checks all panels in the RED (by plan type) row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'Requests (by plan type)',
- 'Error rate (by plan type)',
- 'Duration p99 (by plan type)'
- ])
- })
- })
-
- it('collapses the RED (by plan type) row', function () {
- cy.get('.dashboard-row__title.pointer').contains('RED (by plan type)').click();
- cy.get('.dashboard-row--collapsed').contains('RED (by plan type)');
- })
-
- // RED BY TABLE ROW
-
- it('contains the RED (by table) row', function () {
- cy.get('.dashboard-row').contains('RED (by table)')
- })
- it('RED (by table) row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('RED (by table)')
- })
- it('expands the RED (by table) row', function () {
- cy.get('.dashboard-row__title.pointer').contains('RED (by table)').click();
- })
-
- it('checks all panels in the RED (by table) row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(3)
- expect(titles).to.deep.eq([
- '',
- 'Requests (by table)',
- 'Error rate (by table)'
- ])
- })
- })
-
- it('collapses the RED (by table) row', function () {
- cy.get('.dashboard-row__title.pointer').contains('RED (by table)').click();
- cy.get('.dashboard-row--collapsed').contains('RED (by table)');
- })
-
- // Rows Returned
- it('contains the Rows Returned row', function () {
- cy.get('.dashboard-row').contains(/^Rows returned/)
- })
- it('Rows returned row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains(/^Rows returned/)
- })
- it('expands the Rows returned row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Rows returned/).click();
- })
-
- it('checks all panels in the Rows returned row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(3)
- expect(titles).to.deep.eq([
- '',
- 'Rows Returned (by table)',
- 'Rows Returned (by plan)'
- ])
- })
- })
-
- it('collapses the Rows returned row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Rows returned/).click();
- cy.get('.dashboard-row--collapsed').contains(/^Rows returned/);
- })
-
- // Queries/Errors
- it('contains the Queries/Errors row', function () {
- cy.get('.dashboard-row').contains(/^Queries\/Errors/)
- })
- it('Queries/Errors row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains(/^Queries\/Errors/)
- })
- it('expands the Queries/Errors row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Queries\/Errors/).click();
- })
-
- it('checks all panels in the Queries/Errors row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(3)
- expect(titles).to.deep.eq([
- '',
- 'Queries Killed',
- 'Query errors (by error code)'
- ])
- })
- })
-
- it('collapses the Queries/Errors row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Queries\/Errors/).click();
- cy.get('.dashboard-row--collapsed').contains(/^Queries\/Errors/);
- })
-
- // Vitess Query pool
- it('contains the Vitess - Query pool row', function () {
- cy.get('.dashboard-row').contains('Vitess - Query pool')
- })
- it('Vitess - Query pool row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('Vitess - Query pool')
- })
- it('expands the Vitess - Query pool row', function () {
- cy.get('.dashboard-row__title.pointer').contains('Vitess - Query pool').click();
- })
-
- it('checks all panels in the Query pool row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(6)
- expect(titles).to.deep.eq([
- '',
- 'Available Connections',
- 'Active Connections',
- 'Idle Closed Rate',
- 'Wait count',
- 'Avg wait time'
- ])
- })
- })
-
- it('collapses the Vitess - Query pool row', function () {
- cy.get('.dashboard-row__title.pointer').contains('Vitess - Query pool').click();
- cy.get('.dashboard-row--collapsed').contains('Vitess - Query pool');
- })
-
- // Vitess Transaction pool
- it('contains the Vitess - Transaction pool row', function () {
- cy.get('.dashboard-row').contains('Vitess - Transaction pool')
- })
- it('Vitess - Transaction pool row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('Vitess - Transaction pool')
- })
- it('expands the Vitess - Transaction pool row', function () {
- cy.get('.dashboard-row__title.pointer').contains('Vitess - Transaction pool').click();
- })
-
- it('checks all panels in the Transaction row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(6)
- expect(titles).to.deep.eq([
- '',
- 'Available Connections',
- 'Active Connections',
- 'Idle Closed Rate',
- 'Wait count',
- 'Avg wait time'
- ])
- })
- })
-
- it('collapses the Vitess - Transaction pool row', function () {
- cy.get('.dashboard-row__title.pointer').contains('Vitess - Transaction pool').click();
- cy.get('.dashboard-row--collapsed').contains('Vitess - Transaction pool');
- })
-
- //Vitess timings
- it('contains the Vitess Timings row', function () {
- cy.get('.dashboard-row').contains(/^Vitess Timings/)
- })
- it('Vitess Timings row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains(/^Vitess Timings/)
- })
- it('Vitess Timings row has 8 panels', function () {
- cy.get('.dashboard-row').contains(/^Vitess Timings/).find('.dashboard-row__panel_count').contains('(8 panels)')
- })
- it('expands the Vitess Timings row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Vitess Timings/).click();
- })
-
- it('checks all panels in the Vitess Timings row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(9)
- expect(titles).to.deep.eq([
- '',
- 'Query Duration (avg)',
- 'Query Duration (p50)',
- 'Query Duration (p95)',
- 'VtGate -> VtTablet Call Time (avg)',
- 'Query Time Distribution (Heatmap)',
- 'Transaction Duration (avg)',
- 'Transaction Duration (p50)',
- 'Transaction Duration (p95)'
- ])
- })
- })
-
- it('collapses the Vitess Timings row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Vitess Timings/).click();
- cy.get('.dashboard-row--collapsed').contains(/^Vitess Timings/);
- })
-
-
- //MYSQL ROW
- it('contains the MySQL row', function () {
- cy.get('.dashboard-row').contains('MySQL');
- })
- it('MySQL row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('MySQL');
- })
- it('expands the MySQL row', function () {
- cy.get('.dashboard-row__title.pointer').contains('MySQL').click();
- })
- it('checks all panels in the MySQL row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(2)
- expect(titles).to.deep.eq([
- '',
- 'Slow Queries'
- ])
- })
- })
- it('collapses the MySQL row', function () {
- cy.get('.dashboard-row__title.pointer').contains('MySQL').click();
- cy.get('.dashboard-row--collapsed').contains('MySQL');
- })
-
- //MYSQL Timings ROW
- it('contains the MySQL Timings row', function () {
- cy.get('.dashboard-row').contains(/MySQL Timings/);
- })
- it('MySQL Timings row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains(/MySQL Timings/);
- })
- it('expands the MySQL Timings row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/MySQL Timings/).click();
- })
-
- it('checks all panels in the MySQL row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'MySQL time (avg)',
- 'MySQL Exec Time P50',
- 'MySQL Exec Time P95'
- ])
- })
- })
-
- it('collapses the MySQL Timings row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/MySQL Timings/).click();
- cy.get('.dashboard-row--collapsed').contains(/MySQL Timings/);
- })
-
- //OS ROW
- it('contains the OS row', function () {
- cy.get('.dashboard-row').contains('OS');
- })
- it('OS row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains('OS');
- })
- it('expands the OS row', function () {
- cy.get('.dashboard-row__title.pointer').contains('OS').scrollIntoView().click();
- })
- it('checks all panels in the OS row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(6)
- expect(titles).to.deep.eq([
- '',
- 'CPU Usage',
- 'Memory Usage',
- '/mnt disk free',
- 'Network Tx Bytes',
- 'Network Rx Bytes'
- ])
- })
- })
- it('collapses the OS row', function () {
- cy.get('.dashboard-row__title.pointer').contains('OS').click();
- cy.get('.dashboard-row--collapsed').contains('OS');
- })
-
- //Misc ROW
- it('contains the Misc row', function () {
- cy.get('.dashboard-row').contains(/^Misc/);
- })
- it('Misc row is collapsed', function () {
- cy.get('.dashboard-row--collapsed').contains(/^Misc/);
- })
- it('expands the Misc row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Misc/).click();
- })
- it('checks all panels in the Misc row exist',function() {
- cy.get('.panel-title').should(($p) => {
- let titles = $p.map((i,el) => {
- return Cypress.$(el).text()
- })
- titles = titles.get()
- expect(titles).to.have.length(4)
- expect(titles).to.deep.eq([
- '',
- 'GC Count',
- 'GC Duration total per second',
- 'GC Duration quantiles (all hosts)'
- ])
- })
- })
- it('collapses the Misc row', function () {
- cy.get('.dashboard-row__title.pointer').contains(/^Misc/).click();
- cy.get('.dashboard-row--collapsed').contains(/^Misc/);
- })
-
-})
diff --git a/vitess-mixin/e2e/cypress/plugins/index.js b/vitess-mixin/e2e/cypress/plugins/index.js
deleted file mode 100644
index aa9918d2153..00000000000
--- a/vitess-mixin/e2e/cypress/plugins/index.js
+++ /dev/null
@@ -1,21 +0,0 @@
-///
-// ***********************************************************
-// This example plugins/index.js can be used to load plugins
-//
-// You can change the location of this file or turn off loading
-// the plugins file with the 'pluginsFile' configuration option.
-//
-// You can read more here:
-// https://on.cypress.io/plugins-guide
-// ***********************************************************
-
-// This function is called when a project is opened or re-opened (e.g. due to
-// the project's config changing)
-
-/**
- * @type {Cypress.PluginConfig}
- */
-module.exports = (on, config) => {
- // `on` is used to hook into various events Cypress emits
- // `config` is the resolved Cypress config
-}
diff --git a/vitess-mixin/e2e/cypress/support/commands.js b/vitess-mixin/e2e/cypress/support/commands.js
deleted file mode 100644
index 88dec8ed7b0..00000000000
--- a/vitess-mixin/e2e/cypress/support/commands.js
+++ /dev/null
@@ -1,33 +0,0 @@
-const http = require("http")
-
-Cypress.Commands.overwrite('visit', (orig, url, options) => {
- options = options || {}
- options.auth = {
- username: 'admin',
- password: 'admin',
- }
- return orig(url, options)
-})
-
-Cypress.Commands.add('createDashboard', function(dashboardJSON) {
-
- const payload = JSON.stringify({
- dashboard: dashboardJSON,
- overwrite: true
- })
-
- const options = {
- auth: 'admin:admin',
- hostname: 'grafana',
- port: 3000,
- path: '/api/dashboards/db',
- method: 'POST',
- headers: {
- 'Content-Type': 'application/json',
- }
- }
-
- const req = http.request(options)
- req.write(payload)
- req.end()
-})
diff --git a/vitess-mixin/e2e/cypress/support/index.js b/vitess-mixin/e2e/cypress/support/index.js
deleted file mode 100644
index 614041bb83f..00000000000
--- a/vitess-mixin/e2e/cypress/support/index.js
+++ /dev/null
@@ -1,28 +0,0 @@
-require('./commands')
-
-const fs = require('fs')
-
-// This does not use the usual Cypress.Commands.add registration so that it's
-// performed synchronously and we're able to return the panelTitles variable.
-cy.createDashboardFromUnitTests = function(testDir, uid, excludePanels=[]) {
- let panelTitles = []
- cy.readFile(testDir).then(function(str) {
- let panels = []
- for (let [i, [name, panel]] of Object.entries(Object.entries(str))) {
- if (excludePanels.includes(name)) {
- continue
- }
- panel['id'] = parseInt(i)
- panel['gridPos'] = {'w': 6, 'h': 4, 'x': i * 6 % 24 }
- panelTitles.push(panel.title)
- panels.push(panel)
- }
- let dashboardJSON = {
- 'uid': uid,
- 'title': uid,
- 'panels': panels
- }
- cy.createDashboard(dashboardJSON)
- })
- return panelTitles
-}
diff --git a/vitess-mixin/e2e/dbcli.sh b/vitess-mixin/e2e/dbcli.sh
deleted file mode 100755
index 6d56f0b2763..00000000000
--- a/vitess-mixin/e2e/dbcli.sh
+++ /dev/null
@@ -1,33 +0,0 @@
-#!/bin/bash
-
-# Copyright 2019 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.
-
-usage () {
- echo "Starts a session on a sideloaded vttablet."
- echo "Note that this is a direct MySQL connection; if you actually want to work with Vitess, connect via the vtgate with:"
- echo " mysql --port=15306 --host=127.0.0.1"
- echo
- echo "Usage: $0 []"
- echo " Don't forget the 'vt_' before the keyspace!"
-}
-
-if [ $# -lt 1 ]; then
- usage
- exit -1
-fi
-
-keyspace=${2:-vt_test_keyspace}
-long_alias=`printf "%010d" $1`
-docker-compose exec vttablet$1 mysql -uvt_dba -S /vt/vtdataroot/vt_${long_alias}/mysql.sock $keyspace
diff --git a/vitess-mixin/e2e/default_vschema.json b/vitess-mixin/e2e/default_vschema.json
deleted file mode 100644
index e0b50a66037..00000000000
--- a/vitess-mixin/e2e/default_vschema.json
+++ /dev/null
@@ -1,8 +0,0 @@
-{
- "sharded": false,
- "vindexes": {
- "hash": {
- "type": "hash"
- }
- }
-}
diff --git a/vitess-mixin/e2e/docker-compose.beginners.yml b/vitess-mixin/e2e/docker-compose.beginners.yml
deleted file mode 100644
index 46eadd57801..00000000000
--- a/vitess-mixin/e2e/docker-compose.beginners.yml
+++ /dev/null
@@ -1,312 +0,0 @@
-version: "2.1"
-services:
- consul1:
- image: consul:latest
- hostname: "consul1"
- ports:
- - "8400:8400"
- - "8500:8500"
- - "8600:8600"
- command: "agent -server -bootstrap-expect 3 -ui -disable-host-node-id -client 0.0.0.0"
- consul2:
- image: consul:latest
- hostname: "consul2"
- expose:
- - "8400"
- - "8500"
- - "8600"
- command: "agent -server -retry-join consul1 -disable-host-node-id"
- depends_on:
- - consul1
- consul3:
- image: consul:latest
- hostname: "consul3"
- expose:
- - "8400"
- - "8500"
- - "8600"
- command: "agent -server -retry-join consul1 -disable-host-node-id"
- depends_on:
- - consul1
- # This is a convenience container to quickly test vitess against an external database.
- # In practice you will point Vitess to your existing database and migrate to a Vitess managed cluster.
- external_db_host:
- build:
- context: ./external_db/mysql
- dockerfile: Dockerfile
- restart: always
- environment:
- MYSQL_ROOT_PASSWORD: ${MYSQL_ROOT_PASSWORD:-pass}
- MYSQL_DATABASE: ${DB:-commerce}
- MYSQL_USER: ${DB_USER:-external_db_user}
- MYSQL_PASSWORD: ${DB_PASS:-external_db_password}
- volumes:
- - ./external_db/mysql/:/docker-entrypoint-initdb.d/
- - ./external_db/mysql/log:/var/log/mysql
- command:
- - --server-id=1
- - --log-bin=mysql-bin
- - --gtid_mode=ON
- - --enforce_gtid_consistency
- - --general_log=1
- - --slow_query_log=1
- healthcheck:
- test: "/usr/bin/mysql --user=root --password=$${MYSQL_ROOT_PASSWORD} --execute \"SHOW DATABASES;\""
- timeout: 10s
- retries: 10
- ports:
- - "3306"
-
- vtctld:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15000:$WEB_PORT"
- - "$GRPC_PORT"
- command: ["sh", "-c", " /vt/bin/vtctld \
- $TOPOLOGY_FLAGS \
- -cell $CELL \
- -service_map 'grpc-vtctl' \
- -backup_storage_implementation file \
- -file_backup_storage_root /vt/vtdataroot/backups \
- -logtostderr=true \
- -port $WEB_PORT \
- -grpc_port $GRPC_PORT
- "]
- depends_on:
- - consul1
- - consul2
- - consul3
- depends_on:
- external_db_host:
- condition: service_healthy
-
- vtgate:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15099:$WEB_PORT"
- - "$GRPC_PORT"
- - "15306:$MYSQL_PORT"
- command: ["sh", "-c", "/vt/bin/vtgate \
- $TOPOLOGY_FLAGS \
- --logtostderr=true \
- --port $WEB_PORT \
- --grpc_port $GRPC_PORT \
- --mysql_server_port $MYSQL_PORT \
- --mysql_auth_server_impl none \
- --cell $CELL \
- --cells_to_watch $CELL \
- --tablet_types_to_wait PRIMARY,REPLICA \
- --service_map 'grpc-vtgateservice' \
- --enable_system_settings=true \
- "]
- volumes:
- - ".:/script"
- environment:
- - KEYSPACE
- - DB
- depends_on:
- - vtctld
- depends_on:
- vttablet101:
- condition: service_healthy
-
- schemaload:
- image: vitess/lite:${VITESS_TAG:-latest}
- command:
- - sh
- - -c
- - /script/schemaload.sh
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - TARGETTAB
- - SLEEPTIME
- - VSCHEMA_FILE
- - SCHEMA_FILES
- - POST_LOAD_FILE
- - EXTERNAL_DB
- volumes:
- - .:/script
- depends_on:
- vttablet101:
- condition: service_healthy
-
- vttablet100:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15100:$WEB_PORT"
- - "$GRPC_PORT"
- - "3306"
- volumes:
- - ".:/script"
- - "./backups:/vt/vtdataroot/backups"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- - ROLE=primary
- command: ["sh", "-c", "[ $$EXTERNAL_DB -eq 1 ] && /script/vttablet-up.sh 100 || exit 0"]
- depends_on:
- - vtctld
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:$$WEB_PORT/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-
- vttablet101:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15101:$WEB_PORT"
- - "$GRPC_PORT"
- - "3306"
- volumes:
- - ".:/script"
- - "./backups:/vt/vtdataroot/backups"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- - ROLE=primary
- command: ["sh", "-c", "/script/vttablet-up.sh 101"]
- depends_on:
- - vtctld
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:$$WEB_PORT/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-
- vttablet102:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15102:$WEB_PORT"
- - "$GRPC_PORT"
- - "3306"
- volumes:
- - ".:/script"
- - "./backups:/vt/vtdataroot/backups"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- command: ["sh", "-c", "/script/vttablet-up.sh 102"]
- depends_on:
- - vtctld
- - vttablet101
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:$$WEB_PORT/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-
- vttablet103:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15103:$WEB_PORT"
- - "$GRPC_PORT"
- - "3306"
- volumes:
- - ".:/script"
- - "./backups:/vt/vtdataroot/backups"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- command: ["sh", "-c", "/script/vttablet-up.sh 103"]
- depends_on:
- - vtctld
- - vttablet101
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:$$WEB_PORT/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-
- vtorc:
- image: vitess/lite:${VITESS_TAG:-latest}
- command: ["sh", "-c", "/script/vtorc-up.sh"]
- depends_on:
- - vtctld
- ports:
- - "13000:3000"
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:3000/api/status"]
- interval: 5s
- timeout: 10s
- retries: 15
-
- vreplication:
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS
- - WEB_PORT
- - GRPC_PORT
- - CELL
- - KEYSPACE
- - DB
- - EXTERNAL_DB
- - DB_PORT
- - DB_HOST
- - DB_USER
- - DB_PASS
- - DB_CHARSET
- command: ["sh", "-c", "[ $$EXTERNAL_DB -eq 1 ] && /script/externaldb_vreplication.sh || exit 0"]
- depends_on:
- - vtctld
diff --git a/vitess-mixin/e2e/docker-compose.dev.yml b/vitess-mixin/e2e/docker-compose.dev.yml
deleted file mode 100644
index e5bb4f20ef5..00000000000
--- a/vitess-mixin/e2e/docker-compose.dev.yml
+++ /dev/null
@@ -1,32 +0,0 @@
-version: '3'
-services:
- grafana:
- image: grafana/grafana:${GRAFANA_VERSION?err}
- ports:
- - "3030:3000"
- e2e:
- build:
- context: .
- args:
- CYPRESS_IMAGE: cypress/included:5.3.0
- image: grafonnet-e2e-dev
- entrypoint: cypress open --project .
- depends_on:
- - grafana
- environment:
- - CYPRESS_baseUrl=http://grafana:3000
- - CYPRESS_video=false
- - DISPLAY=${DISPLAY?err}
- volumes:
- - ./cypress:/e2e/cypress
- - ./cypress.json:/e2e/cypress.json
- - ../dashboards_out:/e2e/test
- - /tmp/.X11-unix:/tmp/.X11-unix
- deploy:
- resources:
- limits:
- memory: 2G
- reservations:
- memory: 1G
-volumes:
- prometheus-data: {}
diff --git a/vitess-mixin/e2e/docker-compose.vt.yml b/vitess-mixin/e2e/docker-compose.vt.yml
deleted file mode 100644
index 1132068f252..00000000000
--- a/vitess-mixin/e2e/docker-compose.vt.yml
+++ /dev/null
@@ -1,447 +0,0 @@
-version: '3'
-services:
- consul1:
- command: agent -server -bootstrap-expect 3 -ui -disable-host-node-id -client 0.0.0.0
- hostname: consul1
- image: consul:latest
- ports:
- - 8400:8400
- - 8500:8500
- - 8600:8600
- consul2:
- command: agent -server -retry-join consul1 -disable-host-node-id
- depends_on:
- - consul1
- expose:
- - "8400"
- - "8500"
- - "8600"
- hostname: consul2
- image: consul:latest
- consul3:
- command: agent -server -retry-join consul1 -disable-host-node-id
- depends_on:
- - consul1
- expose:
- - "8400"
- - "8500"
- - "8600"
- hostname: consul3
- image: consul:latest
- external_db_host:
- build:
- context: ./external_db/mysql
- dockerfile: Dockerfile
- command:
- - --server-id=1
- - --log-bin=mysql-bin
- - --gtid_mode=ON
- - --enforce_gtid_consistency
- - --general_log=1
- - --slow_query_log=1
- environment:
- MYSQL_DATABASE: ${DB:-commerce}
- MYSQL_PASSWORD: ${DB_PASS:-external_db_password}
- MYSQL_ROOT_PASSWORD: ${MYSQL_ROOT_PASSWORD:-pass}
- MYSQL_USER: ${DB_USER:-external_db_user}
- healthcheck:
- retries: 10
- test: /usr/bin/mysql --user=root --password=$${MYSQL_ROOT_PASSWORD} --execute
- "SHOW DATABASES;"
- timeout: 10s
- ports:
- - "3306"
- restart: always
- volumes:
- - ./external_db/mysql/:/docker-entrypoint-initdb.d/
- - ./external_db/mysql/log:/var/log/mysql
- schemaload_lookup_keyspace:
- command:
- - sh
- - -c
- - /script/schemaload.sh
- depends_on:
- vttablet301:
- condition: service_healthy
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=lookup_keyspace
- - TARGETTAB=test-0000000301
- - SLEEPTIME=15
- - VSCHEMA_FILE=lookup_keyspace_vschema.json
- - SCHEMA_FILES=lookup_keyspace_schema_file.sql
- - POST_LOAD_FILE=
- - EXTERNAL_DB=0
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - .:/script
- schemaload_test_keyspace:
- command:
- - sh
- - -c
- - /script/schemaload.sh
- depends_on:
- vttablet101:
- condition: service_healthy
- vttablet201:
- condition: service_healthy
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - TARGETTAB=test-0000000101
- - SLEEPTIME=15
- - VSCHEMA_FILE=test_keyspace_vschema.json
- - SCHEMA_FILES=test_keyspace_schema_file.sql
- - POST_LOAD_FILE=
- - EXTERNAL_DB=0
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - .:/script
- vreplication:
- command:
- - sh
- - -c
- - '[ $$EXTERNAL_DB -eq 1 ] && /script/externaldb_vreplication.sh || exit 0'
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - EXTERNAL_DB=0
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - .:/script
- vtctld:
- command:
- - sh
- - -c
- - ' /vt/bin/vtctld -topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global -cell test
- -service_map ''grpc-vtctl'' -backup_storage_implementation file -file_backup_storage_root
- /vt/vtdataroot/backups -logtostderr=true -port 8080 -grpc_port 15999 '
- depends_on:
- external_db_host:
- condition: service_healthy
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15000:8080
- - "15999"
- volumes:
- - .:/script
- vtgate:
- command:
- - sh
- - -c
- - '/script/run-forever.sh /vt/bin/vtgate --topo_implementation consul --topo_global_server_address
- consul1:8500 --topo_global_root vitess/global --logtostderr=true --port 8080 --grpc_port
- 15999 --mysql_server_port 15306 --mysql_auth_server_impl none --cell test --cells_to_watch
- test --tablet_types_to_wait PRIMARY,REPLICA,RDONLY --service_map ''grpc-vtgateservice''
- --normalize_queries=true '
- depends_on:
- - vtctld
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15099:8080
- - "15999"
- - 15306:15306
- volumes:
- - .:/script
- vtorc:
- command:
- - sh
- - -c
- - /script/vtorc-up.sh
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - EXTERNAL_DB=0
- - DB_USER=
- - DB_PASS=
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 13000:3000
- volumes:
- - .:/script
- vttablet101:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 101
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=-80
- - ROLE=primary
- - VTHOST=vttablet101
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15101:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet102:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 102
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=-80
- - ROLE=replica
- - VTHOST=vttablet102
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15102:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet201:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 201
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=80-
- - ROLE=primary
- - VTHOST=vttablet201
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15201:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet202:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 202
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=80-
- - ROLE=replica
- - VTHOST=vttablet202
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15202:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet301:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 301
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=lookup_keyspace
- - SHARD=-
- - ROLE=primary
- - VTHOST=vttablet301
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15301:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet302:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 302
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=lookup_keyspace
- - SHARD=-
- - ROLE=replica
- - VTHOST=vttablet302
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl -s --fail --show-error localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - 15302:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- prometheus:
- image: prom/prometheus:v2.21.0
- ports:
- - 9000:9090
- volumes:
- - ./prometheus:/etc/prometheus
- - ../prometheus_rules.yaml:/etc/prometheus/prometheus_rules.yaml
- - prometheus-data:/prometheus
- command: --web.enable-lifecycle --config.file=/etc/prometheus/prometheus.yml
- depends_on:
- - vtctld
- grafana:
- image: grafana/grafana:${GRAFANA_VERSION?err}
- ports:
- - "3030:3000"
- volumes:
- - ./grafana/provisioning:/etc/grafana/provisioning
- - ../dashboards_out/cluster_overview.json:/etc/grafana/provisioning/dashboards/cluster_overview.json
- - ../dashboards_out/vtgate_overview.json:/etc/grafana/provisioning/dashboards/vtgate_overview.json
- - ../dashboards_out/vtgate_host_view.json:/etc/grafana/provisioning/dashboards/vtgate_host_view.json
- - ../dashboards_out/vttablet_host_view.json:/etc/grafana/provisioning/dashboards/vttablet_host_view.json
- depends_on:
- - prometheus
- - vtgate
- - vttablet101
- node-exporter:
- image: prom/node-exporter
- volumes:
- - /proc:/host/proc:ro
- - /sys:/host/sys:ro
- - /:/rootfs:ro
- command:
- - '--path.procfs=/host/proc'
- - '--path.sysfs=/host/sys'
- - --collector.filesystem.ignored-mount-points
- - "^/(sys|proc|dev|host|etc|rootfs/var/lib/docker/containers|rootfs/var/lib/docker/overlay2|rootfs/run/docker/netns|rootfs/var/lib/docker/aufs)($$|/)"
- ports:
- - '9100:9100'
- deploy:
- mode: global
- mysqld_exporter:
- image: prom/mysqld-exporter:latest
- environment:
- - DATA_SOURCE_NAME=root:pass@(external_db_host:3306)/
- ports:
- - "9104:9104"
- depends_on:
- - external_db_host
- # TODO: ADD CYPRESS FOR AUTOMATED TESTING
-volumes:
- prometheus-data: {}
diff --git a/vitess-mixin/e2e/docker-compose.yml b/vitess-mixin/e2e/docker-compose.yml
deleted file mode 100644
index e46fbcab849..00000000000
--- a/vitess-mixin/e2e/docker-compose.yml
+++ /dev/null
@@ -1,20 +0,0 @@
-version: '3'
-services:
- grafana:
- image: grafana/grafana:${GRAFANA_VERSION?err}
- e2e:
- build:
- context: .
- args:
- CYPRESS_IMAGE: cypress/base:12
- image: grafonnet-e2e
- command: npx cypress run
- depends_on:
- - grafana
- environment:
- - CYPRESS_baseUrl=http://grafana:3000
- - CYPRESS_video=false
- volumes:
- - ./cypress:/e2e/cypress
- - ./cypress.json:/e2e/cypress.json
- - ../dashboards_out:/e2e/test
diff --git a/vitess-mixin/e2e/external_db/.env b/vitess-mixin/e2e/external_db/.env
deleted file mode 100644
index f2745ff08ef..00000000000
--- a/vitess-mixin/e2e/external_db/.env
+++ /dev/null
@@ -1,4 +0,0 @@
-MYSQL_ROOT_PASSWORD=pass
-MYSQL_USER=dbuser
-MYSQL_PASSWORD=dbpass
-MYSQL_DB=commerce
\ No newline at end of file
diff --git a/vitess-mixin/e2e/external_db/README.md b/vitess-mixin/e2e/external_db/README.md
deleted file mode 100644
index 610a9350111..00000000000
--- a/vitess-mixin/e2e/external_db/README.md
+++ /dev/null
@@ -1,113 +0,0 @@
-**This README is kept here for reference, however the parent [README](../README) contains all the information necesaary to simulate this in a better way**
-
-# Simulate external/remote database for Vitess using docker-compose
-
-This directory has a docker-compose that will bring up a mysql instance.
-You can then point your vitess cluster to it to understand how to use Vitess for your existing database
-when you cannot install Vitess on the mysql instance.
-
-First you will need to [install docker-compose](https://docs.docker.com/compose/install/).
-
-
-### Create new docker-machine
-Create a new docker-machine that will run your mysql container.
-Creating a new machine allows you to more comprehensively test the remote functionality.
-```
-vitess/examples/compose/external_db$ docker-machine create remote-db
-```
-
-Grab the docker-machine ip
-```
-vitess/examples/compose/external_db$ docker-machine ip remote-db
-192.168.99.101
-```
-
-Set the environment variables for the remote-db machine
-```
-vitess/examples/compose/external_db$ eval $(docker-machine ip remote-db)
-```
-
-### Start mysql
-Start the mysql instance
-```
-vitess/examples/compose/external_db$ docker-compose up -d
-```
-This will do the following;
-1. Starts mysql service and exposes it at `:3306`
-2. Creates a `commerce` database with `users` table
-3. Adds sample data to the users table
-4. Starts a lightweight adminer container to interact with the database accessible at `:8081`
-5. Default credentials
- ```
- MYSQL_DB: commerce
- MYSQL_USER: dbuser
- MYSQL_PASSWORD: dbpass
- MYSQL_ROOT_PASSWORD: pass
- ```
-
-### Confirm containers are up
-Run the following
-```
-vitess/examples/compose/external_db$ docker-compose ps
-```
-
-A valid response should look like below
-```sh
- Name Command State Ports
----------------------------------------------------------------------------------------------------------
-external_db_adminer_1 entrypoint.sh docker-php-e ... Up 0.0.0.0:8081->8080/tcp
-external_db_db_1 docker-entrypoint.sh mysqld Up (healthy) 0.0.0.0:3306->3306/tcp, 33060/tcp
-```
-You now have a mysql instance ready to be *migrated* to Vitess.
-
-### Start Vitess pointed to this remote database
-Head on to [vitess compose instructions](../README.md )
-
-If using docker-compose.beginners.yml, run;
-```
-vitess/examples/compose$ cp docker-compose.beginners.yml docker-compose.yml
-```
-Update your `.env` file with these;
-```
-KEYSPACE=commerce
-DB=commerce
-EXTERNAL_DB=1
-DB_HOST=
-DB_PORT=3306
-DB_USER=dbuser
-DB_PASS=dbpass
-DB_CHARSET=CHARACTER SET latin1 COLLATE latin1_swedish_ci
-```
-
-
-If using `vtcompose` command, run;
-```
-vitess/examples/compose$ go run vtcompose/vtcompose.go -keyspaceData="commerce:0:2::" -externalDbData="commerce::3306:dbuser:dbpass:CHARACTER SET latin1 COLLATE latin1_swedish_ci"
-```
-
-**Ensure you start Vitess in a different docker-machine!!**
-If not, run;
-```
-vitess/examples/compose$ docker-machine create vitess
-vitess/examples/compose$ $(docker-machine env vitess)
-```
-
-Start Vitess
-```
-vitess/examples/compose$ docker-compose up -d
-```
-
-You should now have Vitess running against your external database instance.
-
-* [Follow this guide for advanced usage](../README.md#advanced-usage "Advanced Usage" )
-* [See this for common issues](../README.md#common-errors "Common Issues" )
-
-### Migrating to Vitess
-Migrating to Vitess entirely can be done from;
-a) The Vitess Control Panel at http://:15000
-b) The `lvtctl.sh` Helper Script;
-
-The steps are same
-1. Do an EmergencyReparentShard to make a replica the new primary.
-2. Ran InitShardPrimary on the new primary.
-3. If Vitess is wrong about who the MySQL primary is, you can update it with TabletExternallyReparented
diff --git a/vitess-mixin/e2e/external_db/docker-compose.yml b/vitess-mixin/e2e/external_db/docker-compose.yml
deleted file mode 100644
index 4c414324e63..00000000000
--- a/vitess-mixin/e2e/external_db/docker-compose.yml
+++ /dev/null
@@ -1,41 +0,0 @@
-version: '2.1'
-
-volumes:
- vol-db:
-
-services:
- db:
- build:
- context: ./mysql
- dockerfile: Dockerfile
- restart: always
- environment:
- MYSQL_ROOT_PASSWORD: ${MYSQL_ROOT_PASSWORD:-pass}
- MYSQL_DATABASE: ${MYSQL_DB:-commerce}
- MYSQL_USER: ${MYSQL_USER:-dbuser}
- MYSQL_PASSWORD: ${MYSQL_PASSWORD:-dbpass}
- volumes:
- - vol-db:/var/lib/mysql
- - ./mysql/:/docker-entrypoint-initdb.d/
- - ./mysql/mysql57.cnf:/etc/mysql/conf.d/mysql57.cnf:ro
- - ./mysql/log:/var/log/mysql
- command:
- - --server-id=1
- - --log-bin=mysql-bin
- - --gtid_mode=ON
- - --enforce_gtid_consistency
- - --general_log=1
- - --slow_query_log=1
- healthcheck:
- test: "/usr/bin/mysql --user=root --password=$${MYSQL_ROOT_PASSWORD} --execute \"SHOW DATABASES;\""
- timeout: 10s
- retries: 10
- ports:
- - "3306:3306"
-
- adminer:
- image: adminer
- environment:
- ADMINER_DESIGN: rmsoft
- ports:
- - "8081:8080"
diff --git a/vitess-mixin/e2e/external_db/mysql/Dockerfile b/vitess-mixin/e2e/external_db/mysql/Dockerfile
deleted file mode 100644
index f44c63951e3..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/Dockerfile
+++ /dev/null
@@ -1,2 +0,0 @@
-FROM mysql:5.7
-COPY . /docker-entrypoint-initdb.d
\ No newline at end of file
diff --git a/vitess-mixin/e2e/external_db/mysql/commerce.sql b/vitess-mixin/e2e/external_db/mysql/commerce.sql
deleted file mode 100644
index 8154d91e7f5..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/commerce.sql
+++ /dev/null
@@ -1,20 +0,0 @@
-CREATE DATABASE IF NOT EXISTS commerce;
-USE commerce;
-DROP TABLE IF EXISTS users;
-CREATE TABLE users (
- device_id BIGINT,
- first_name VARCHAR(50),
- last_name VARCHAR(50),
- telephone BIGINT,
- gender VARCHAR(16),
- reference_id INT,
- confidence INT,
- coverage INT,
- refstart DATETIME,
- refstop DATETIME,
- qrystart DATETIME,
- qrystop DATETIME);
-
-LOAD DATA LOCAL INFILE '/docker-entrypoint-initdb.d/dataset.csv' INTO TABLE users FIELDS TERMINATED BY ',';
-
-ALTER TABLE users ADD id INT NOT NULL AUTO_INCREMENT PRIMARY KEY;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/external_db/mysql/dataset.csv b/vitess-mixin/e2e/external_db/mysql/dataset.csv
deleted file mode 100644
index f2af8d74ce7..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/dataset.csv
+++ /dev/null
@@ -1,1000 +0,0 @@
-1,Elianore,Dunbleton,867-921-5436,Female,1,4,90,2018-11-20 00:23:59,2018-02-11 11:32:09,2018-06-18 23:40:59,2018-07-14 00:59:56
-2,Isa,Gilfoyle,539-533-8647,Male,7,86,65,2018-03-22 18:52:41,2019-01-16 21:08:18,2019-01-03 10:48:54,2018-08-29 14:52:14
-3,Millicent,Jedrys,184-899-3979,Female,5,30,17,2018-06-29 14:14:44,2018-06-07 05:03:57,2018-04-08 03:56:39,2018-02-12 06:41:48
-4,Davey,Sutch,862-838-8206,Male,1,62,94,2018-11-14 06:47:18,2018-08-08 22:44:26,2018-07-12 21:09:27,2018-03-04 16:23:22
-5,Adiana,Strowger,792-848-5008,Female,6,97,75,2018-08-07 01:32:59,2018-08-06 16:19:48,2019-01-20 13:11:37,2018-09-02 22:39:39
-6,Oby,Winthrop,522-736-9711,Male,7,95,11,2018-09-29 06:49:44,2018-02-02 11:07:25,2018-08-17 12:49:31,2018-06-03 13:27:48
-7,Berte,Beldon,947-708-5622,Female,6,3,26,2018-11-21 14:55:27,2018-11-15 17:16:04,2018-01-23 06:12:07,2018-08-30 14:40:37
-8,Logan,Atack,103-849-8439,Male,1,20,86,2019-01-04 01:18:38,2018-09-20 23:59:42,2018-05-10 13:14:24,2018-09-25 10:05:29
-9,Vania,Rosenblum,302-132-8289,Female,5,85,32,2018-03-31 00:24:32,2018-07-28 14:50:39,2018-04-19 12:32:49,2018-04-03 08:31:11
-10,Giffie,Lindblad,312-429-3236,Male,1,52,65,2018-04-15 15:34:57,2018-09-03 05:54:49,2019-01-05 06:42:27,2018-11-05 21:47:08
-11,Bili,Weigh,442-992-2387,Female,5,88,97,2018-12-01 04:31:51,2018-08-22 15:52:23,2018-12-07 23:25:46,2018-08-08 07:47:35
-12,Marlin,Stair,854-643-9633,Male,1,62,61,2018-05-30 14:08:25,2018-03-02 03:34:27,2018-03-21 22:43:12,2018-06-03 18:02:22
-13,Dacey,Corradino,364-281-2170,Female,3,82,19,2018-04-10 01:42:36,2019-01-08 17:42:03,2018-05-20 04:07:33,2018-10-14 03:57:57
-14,Caresse,Santon,221-929-9690,Female,7,97,50,2018-11-20 23:15:03,2019-01-01 18:46:08,2018-04-08 05:49:39,2018-10-04 04:00:37
-15,Trixi,Westphalen,681-738-3653,Female,7,100,87,2018-07-09 18:04:01,2018-11-21 04:36:24,2018-11-25 09:43:50,2018-09-10 07:40:31
-16,Pauline,Breslauer,821-177-6696,Female,6,63,24,2018-11-02 19:56:35,2018-02-09 07:46:00,2019-01-08 03:21:02,2018-09-18 08:18:06
-17,Meridith,Briddock,716-528-7645,Female,3,72,24,2018-11-30 20:13:26,2018-04-04 00:17:34,2018-07-12 04:42:43,2018-07-13 02:53:00
-18,Cordy,Sothern,733-369-1763,Male,6,72,5,2018-08-19 02:02:51,2018-10-01 16:39:38,2019-01-18 12:58:50,2018-04-26 11:01:54
-19,Thom,Swarbrigg,566-424-7472,Male,3,56,29,2018-01-25 19:13:40,2018-09-06 20:39:48,2018-12-23 14:49:40,2018-09-19 13:03:37
-20,Amelina,Ekell,514-321-8056,Female,5,42,43,2018-12-15 14:21:28,2018-08-07 02:53:55,2018-09-30 04:29:03,2018-08-02 18:29:13
-21,Cesare,Lahy,446-382-1825,Male,2,32,10,2018-07-08 01:24:43,2018-03-24 15:25:21,2018-11-15 18:50:17,2018-03-27 19:18:16
-22,Elnora,Cheale,489-718-9700,Female,6,12,73,2018-10-26 17:34:35,2018-12-22 05:44:59,2018-05-09 01:25:04,2018-09-21 18:40:44
-23,Hadrian,Snarie,859-447-4644,Male,1,18,4,2018-08-31 12:15:30,2018-04-16 11:19:30,2018-02-28 10:55:26,2018-10-31 07:05:29
-24,Ravid,Meriott,518-484-9203,Male,1,83,12,2018-12-01 21:38:00,2018-02-12 17:25:00,2018-09-18 16:34:45,2018-09-08 17:32:17
-25,Elenore,De Gowe,180-237-8349,Female,7,75,3,2018-03-20 20:52:00,2018-12-22 19:52:58,2018-07-06 23:18:01,2018-08-12 22:02:21
-26,Juieta,Pridie,278-696-9233,Female,1,93,57,2018-09-04 18:45:44,2018-06-04 17:38:47,2018-11-16 10:31:12,2018-07-15 10:27:10
-27,Neddie,Mosedall,956-605-6537,Male,3,94,19,2018-04-09 13:54:22,2018-04-12 10:58:25,2018-08-21 10:57:27,2018-08-22 20:29:53
-28,Octavius,Fordham,797-217-3886,Male,3,32,82,2018-11-23 16:25:55,2018-11-15 12:08:34,2018-10-28 03:46:45,2018-06-22 03:16:15
-29,Edwin,Tapson,578-261-4030,Male,7,83,39,2018-12-16 01:07:34,2018-12-27 23:39:47,2018-12-03 00:30:10,2018-03-09 18:09:49
-30,Korry,Dyos,108-910-1353,Female,3,80,93,2018-06-14 09:39:58,2018-08-12 21:17:11,2018-09-14 12:22:43,2018-04-08 23:31:12
-31,Kenn,Leist,300-420-7371,Male,5,100,90,2018-11-24 01:42:36,2018-10-27 21:31:19,2019-01-05 02:32:45,2018-11-24 06:45:27
-32,Eldredge,Kemmis,451-974-5763,Male,2,44,42,2018-11-18 11:28:34,2018-02-17 08:01:53,2018-08-21 19:25:26,2018-12-23 12:29:15
-33,Suzanne,Matthiae,560-959-3441,Female,4,56,7,2018-05-04 04:19:19,2018-04-11 11:59:42,2018-02-01 16:22:38,2018-11-15 19:58:02
-34,Josh,Callendar,719-551-3561,Male,4,61,56,2018-10-12 07:20:47,2018-06-14 12:53:31,2018-07-20 09:35:33,2018-03-26 05:54:27
-35,Ian,Thoresby,392-740-5436,Male,7,77,8,2018-03-04 19:37:23,2018-08-07 02:19:40,2018-09-24 07:54:33,2018-07-09 15:36:52
-36,Cordi,Vanichkin,773-231-4509,Female,1,99,90,2018-07-02 02:43:15,2019-01-20 04:28:54,2018-09-05 05:02:38,2019-01-08 04:08:00
-37,Ariel,Le land,917-249-6942,Male,5,9,45,2018-04-29 18:48:49,2018-06-23 08:15:12,2018-01-22 01:29:55,2019-01-03 21:24:23
-38,Delila,Assard,119-780-7155,Female,3,83,70,2018-12-01 01:09:53,2018-05-22 00:23:40,2018-05-19 13:53:02,2018-02-17 03:43:18
-39,Deny,Cullrford,100-298-0840,Female,5,50,39,2018-05-09 23:21:57,2018-05-31 23:46:32,2018-12-03 09:53:47,2019-01-12 10:16:35
-40,Frannie,Sharer,410-855-0951,Male,6,72,43,2018-05-12 06:05:03,2018-04-21 17:15:14,2018-08-30 07:18:18,2018-06-20 04:51:13
-41,Jonis,Lintin,803-933-8038,Female,4,89,84,2018-09-13 20:13:23,2018-08-28 18:20:52,2018-03-13 03:07:16,2018-03-05 07:10:50
-42,Bessie,Hackett,443-436-4804,Female,4,20,36,2018-06-13 07:34:49,2018-06-01 11:23:42,2018-10-19 12:40:14,2018-02-10 12:37:05
-43,Gayel,Tabard,704-527-0263,Female,6,40,44,2019-01-04 14:41:19,2018-04-15 00:21:22,2018-12-30 07:24:38,2018-05-02 21:10:48
-44,Ardisj,Ridder,899-877-7365,Female,1,21,84,2018-06-13 21:09:11,2018-12-02 20:33:23,2018-03-25 07:43:03,2018-10-12 11:26:41
-45,Hammad,Stroband,874-825-9046,Male,3,30,83,2018-09-14 06:04:39,2018-10-23 13:35:20,2018-03-10 08:19:18,2018-07-18 20:31:55
-46,Darius,Sorel,750-599-8665,Male,7,98,76,2018-09-17 22:09:17,2018-06-02 14:07:06,2018-12-08 15:52:18,2018-05-05 12:38:37
-47,Dido,Stockin,339-433-0084,Female,1,89,59,2018-04-30 07:42:06,2018-10-18 22:27:31,2018-11-11 05:06:09,2019-01-01 14:19:17
-48,Sholom,Cobbing,828-431-0433,Male,4,53,86,2018-11-03 23:21:35,2018-10-23 13:33:47,2018-09-03 06:25:27,2019-01-18 06:26:39
-49,Melinde,Hynd,283-446-7128,Female,5,97,84,2018-07-31 06:28:22,2018-10-24 04:00:28,2018-07-28 03:47:24,2018-02-10 19:44:46
-50,Hedvige,Fontenot,534-558-8139,Female,4,94,62,2018-07-15 15:22:39,2018-03-20 18:21:30,2018-03-04 17:34:45,2018-05-25 22:41:39
-51,Killian,Gilmartin,722-114-3059,Male,1,47,8,2018-08-24 01:48:56,2018-07-21 08:24:06,2018-11-05 06:28:14,2018-05-06 13:37:33
-52,Inness,O'Cullinane,630-746-2431,Male,3,71,93,2018-06-14 22:12:56,2018-04-12 23:09:39,2018-02-02 09:42:12,2018-11-14 01:06:15
-53,Doretta,Galiford,528-456-0043,Female,3,84,95,2018-09-09 15:03:41,2018-03-30 03:54:32,2018-10-03 10:51:33,2018-12-15 22:54:24
-54,Alicia,Kemmons,695-461-8136,Female,7,16,82,2018-03-26 20:29:48,2018-03-25 15:58:54,2018-01-30 03:52:35,2018-02-11 04:14:39
-55,Brandea,Nannizzi,215-545-0363,Female,2,86,20,2018-11-05 15:18:22,2018-02-18 13:05:40,2018-11-27 14:37:52,2018-05-04 23:08:00
-56,Jordan,Parkman,473-392-8118,Male,1,41,46,2018-09-27 14:56:49,2018-04-27 06:35:47,2018-02-04 22:54:44,2018-12-06 04:53:24
-57,Bourke,Whittaker,712-907-5729,Male,2,71,6,2018-10-19 17:00:36,2018-12-02 03:28:48,2018-04-13 01:02:14,2018-04-11 23:42:42
-58,Honey,Adriano,822-890-9194,Female,4,31,95,2018-06-18 04:19:51,2018-03-12 03:05:16,2018-02-02 00:52:15,2019-01-07 13:33:51
-59,Tracie,Warrender,617-331-6980,Male,3,69,37,2018-07-09 15:19:55,2018-07-26 13:00:13,2018-03-06 19:39:06,2018-10-05 14:56:58
-60,Billy,Eadmead,556-100-5109,Female,6,16,63,2018-10-04 07:52:43,2018-07-02 07:34:18,2018-05-03 01:03:27,2018-06-03 11:04:04
-61,Jamesy,Mellenby,554-299-7370,Male,6,53,82,2018-10-25 22:03:32,2018-05-03 02:36:00,2018-01-26 21:04:03,2018-09-15 17:06:05
-62,Goober,Mawby,765-918-7080,Male,1,70,20,2018-08-05 15:03:53,2018-09-15 16:38:46,2018-03-02 01:37:07,2018-12-15 23:40:37
-63,Zach,Tompkins,798-498-7252,Male,6,45,65,2018-06-26 09:42:40,2018-04-20 17:05:50,2018-09-18 16:58:53,2018-06-24 00:57:54
-64,Augustine,Cornbill,600-318-2489,Male,7,30,38,2018-02-08 08:08:53,2018-08-14 07:05:17,2018-10-14 01:12:13,2018-09-20 19:42:49
-65,Cort,Rosingdall,915-533-3544,Male,5,81,82,2019-01-18 02:20:54,2018-03-12 08:02:19,2018-05-13 15:10:29,2018-05-16 21:38:26
-66,Broddie,Dziwisz,404-420-5799,Male,7,3,22,2018-08-30 21:01:34,2018-03-13 06:37:11,2018-04-27 16:13:37,2018-12-09 08:39:26
-67,Clarke,Goodge,499-845-2443,Male,2,14,95,2018-04-02 10:32:42,2018-04-27 00:42:32,2018-05-29 08:36:44,2018-04-01 23:40:09
-68,Madelyn,Knight,724-972-4025,Female,6,12,57,2018-04-25 04:45:31,2018-08-13 14:34:24,2018-05-25 09:58:32,2018-09-02 04:03:25
-69,Bibby,Pace,266-593-6853,Female,2,11,87,2018-04-12 22:35:20,2018-03-10 11:28:13,2018-04-12 15:22:53,2018-04-21 18:57:47
-70,Nedi,Tambling,534-778-4478,Female,5,69,95,2018-04-27 15:43:27,2018-07-27 11:55:14,2018-04-05 07:02:04,2018-08-26 01:25:05
-71,Veronica,Alejandro,208-365-2416,Female,1,22,32,2018-04-09 03:42:21,2018-03-30 02:47:59,2018-02-08 16:43:52,2018-03-10 19:51:53
-72,Cecil,Walewicz,133-144-8036,Male,3,58,23,2018-12-26 06:56:13,2018-06-08 09:09:35,2018-02-16 16:39:15,2018-12-21 00:33:53
-73,Salem,Armell,186-906-5312,Male,7,25,47,2018-10-08 00:28:48,2018-09-03 00:55:23,2018-01-31 20:13:21,2018-04-06 20:30:43
-74,Brook,Buckston,714-565-2523,Female,1,13,92,2018-02-12 13:53:51,2019-01-10 22:17:38,2018-08-03 18:27:58,2018-09-06 10:38:45
-75,Benson,MacParlan,881-685-7375,Male,7,95,39,2018-04-07 06:40:34,2018-05-10 22:04:48,2018-10-06 01:07:35,2018-12-25 18:36:48
-76,Lukas,Kittle,845-541-8229,Male,1,52,85,2018-04-14 08:08:32,2018-08-21 12:21:22,2018-11-17 23:14:34,2018-06-21 02:48:31
-77,Jessa,Claus,611-425-9914,Female,7,99,56,2018-10-22 02:50:57,2018-11-25 03:44:27,2018-06-30 15:33:40,2018-07-02 03:59:50
-78,Hieronymus,Schurig,840-630-9703,Male,6,98,77,2018-01-24 18:01:57,2018-06-05 08:22:05,2018-07-12 07:09:14,2018-07-31 08:09:20
-79,Quintilla,Risborough,367-606-2805,Female,4,15,15,2018-10-24 13:48:18,2018-12-16 11:17:17,2018-05-17 19:15:19,2018-06-22 16:46:31
-80,Arv,Billingsley,438-164-1166,Male,3,54,65,2018-07-21 13:35:44,2018-10-07 14:37:53,2018-11-16 15:38:23,2018-10-19 21:07:38
-81,Happy,Rodrigo,427-752-3843,Female,2,28,25,2018-01-28 21:31:16,2018-07-06 04:48:40,2018-02-19 10:50:55,2018-10-20 15:58:59
-82,Cos,Chalfain,609-228-7131,Male,6,12,80,2018-02-24 18:06:52,2018-07-21 03:12:50,2018-02-22 09:53:50,2018-12-04 18:31:35
-83,Dael,Crother,249-141-1807,Male,3,11,86,2018-09-09 23:04:41,2018-02-24 13:19:31,2018-11-05 15:18:33,2018-12-23 18:55:24
-84,Evin,Tettley,974-233-5051,Male,1,57,61,2018-12-26 14:24:36,2018-05-28 19:09:40,2018-09-18 10:01:10,2018-04-04 21:11:05
-85,Bethanne,Hynson,395-152-0590,Female,5,51,39,2018-11-12 01:12:14,2018-07-21 02:07:45,2019-01-06 17:16:22,2018-11-15 16:58:09
-86,Adrian,Sapson,962-473-8077,Female,4,19,63,2018-02-13 14:27:28,2018-06-26 14:23:38,2018-09-17 04:28:22,2018-07-19 18:25:11
-87,Zaneta,Alforde,961-173-4877,Female,2,70,67,2018-11-04 11:27:20,2018-06-09 18:02:24,2018-08-17 03:13:55,2018-09-02 00:28:34
-88,Grady,Kobiera,687-321-5987,Male,1,62,96,2018-06-23 20:06:04,2018-02-08 04:03:21,2018-12-05 23:36:55,2018-10-26 14:17:35
-89,Dominique,Carslake,230-143-4475,Male,4,38,64,2018-05-19 01:14:57,2018-09-24 20:05:20,2018-06-26 12:15:04,2018-10-30 18:21:59
-90,Julietta,Gude,585-759-9173,Female,7,59,75,2018-11-25 20:42:33,2018-08-31 21:59:40,2018-12-31 21:04:44,2018-11-23 04:31:37
-91,Lothaire,Macrow,510-775-2964,Male,1,91,59,2018-04-22 14:12:47,2018-07-05 11:15:49,2018-06-04 12:44:19,2018-10-12 10:28:46
-92,Agneta,Trematick,391-964-4713,Female,6,6,91,2018-01-31 13:00:06,2018-11-24 20:48:15,2018-07-27 00:12:28,2018-02-17 08:41:55
-93,Pet,Padgett,690-476-6060,Female,2,17,84,2018-03-21 11:54:39,2018-03-08 01:51:40,2018-12-21 16:41:23,2018-02-07 14:47:44
-94,Barclay,Binion,516-626-1251,Male,4,49,46,2018-10-19 13:16:50,2018-06-25 01:26:53,2018-02-23 06:44:17,2018-10-08 11:01:59
-95,Dillie,Quilleash,676-581-4684,Male,3,27,14,2019-01-02 01:50:23,2019-01-02 04:16:15,2018-07-21 11:54:44,2018-03-06 02:17:37
-96,Marissa,Twinterman,855-444-0657,Female,6,52,62,2018-07-27 09:41:53,2018-03-18 21:07:51,2018-09-05 04:45:41,2018-05-01 14:08:21
-97,Perl,Braun,639-977-9224,Female,7,93,88,2018-03-12 07:21:26,2018-11-12 18:45:36,2018-09-10 10:45:15,2018-09-30 10:28:01
-98,Gwenette,Jozef,804-642-8405,Female,1,67,64,2018-05-02 20:04:22,2018-05-25 14:26:42,2018-12-22 04:00:00,2018-04-27 17:36:21
-99,Gabbi,Nassey,998-336-4318,Female,3,35,79,2018-11-13 07:58:46,2018-02-23 01:11:32,2018-11-08 17:30:31,2018-07-24 03:14:25
-100,Susana,Ebben,826-329-4492,Female,5,9,98,2018-07-08 08:40:45,2018-07-12 00:41:34,2018-10-12 15:33:58,2018-06-16 15:33:32
-101,Mallorie,Marsters,637-209-9862,Female,5,18,34,2018-07-30 15:18:09,2018-09-23 17:22:45,2018-12-21 17:11:36,2018-07-06 19:42:09
-102,Marley,Bethell,102-272-8411,Female,6,70,7,2018-10-08 12:25:07,2018-04-13 04:50:24,2018-03-01 09:20:29,2018-04-05 02:02:24
-103,Genvieve,McMeekin,679-490-4369,Female,3,7,25,2018-02-25 15:34:48,2018-06-24 16:32:10,2018-07-03 20:04:57,2018-12-11 02:53:30
-104,Bartram,Leahey,830-491-8399,Male,3,100,57,2018-10-18 18:39:57,2018-04-08 02:16:41,2018-07-20 01:10:35,2018-06-20 09:41:56
-105,Livvyy,Thornewell,872-138-4425,Female,6,58,58,2019-01-17 09:36:59,2018-04-13 07:25:55,2018-11-28 12:44:14,2018-07-14 04:05:13
-106,Gabriella,Cornish,202-570-6308,Female,7,79,67,2018-08-25 01:20:02,2018-11-30 22:05:13,2018-10-21 11:47:58,2018-11-20 04:28:53
-107,Humberto,Lanfranchi,636-653-6377,Male,3,18,85,2018-08-21 03:38:03,2018-09-01 07:20:38,2018-04-27 04:23:16,2018-08-15 14:45:48
-108,Othella,Asaaf,386-100-6369,Female,4,59,93,2018-05-20 17:17:26,2018-10-23 01:32:55,2018-09-20 03:27:06,2018-12-17 21:11:18
-109,Mil,Booij,494-405-6566,Female,5,88,97,2018-07-12 19:07:28,2018-02-14 12:57:18,2018-12-11 19:22:18,2018-11-23 12:41:41
-110,Orel,Blaxland,309-372-2193,Female,7,30,93,2018-11-18 23:18:41,2018-06-21 19:30:46,2018-09-15 13:43:25,2018-02-04 09:51:54
-111,Sibeal,Fennelow,543-514-6831,Female,6,92,56,2018-10-09 14:48:56,2018-12-18 22:29:16,2018-04-09 03:30:48,2018-04-04 18:11:19
-112,Sidnee,Thorby,687-609-9785,Male,2,30,12,2018-03-28 10:27:39,2018-07-01 07:12:38,2018-02-16 12:58:34,2018-11-16 01:37:38
-113,Lora,Ridewood,164-690-9005,Female,1,37,55,2018-11-13 21:13:39,2018-09-06 23:00:39,2019-01-17 10:31:22,2018-12-05 22:14:39
-114,Velma,Brolan,359-817-6834,Female,6,26,38,2019-01-16 21:47:15,2018-11-25 16:54:49,2018-10-07 01:07:24,2018-10-11 01:14:02
-115,Vassili,Kirkbride,287-966-8144,Male,6,52,16,2018-12-20 23:23:45,2018-12-21 08:42:29,2018-02-25 02:07:40,2018-06-28 04:39:19
-116,Nettie,Bulmer,357-176-0651,Female,5,16,44,2018-09-19 15:07:17,2018-03-16 17:55:44,2018-09-06 22:48:36,2018-04-28 23:48:07
-117,Elyn,Matthew,176-489-8486,Female,4,25,95,2018-04-10 12:26:47,2019-01-10 03:12:46,2018-07-28 23:51:46,2018-07-07 16:15:50
-118,Nat,Enderlein,480-924-1165,Male,4,94,42,2018-09-10 06:56:16,2018-04-24 02:55:47,2018-03-17 15:48:28,2018-04-30 06:11:27
-119,Allister,Laundon,267-536-1588,Male,6,86,21,2018-08-21 09:06:13,2018-11-26 09:47:45,2018-04-05 15:01:24,2018-01-21 02:29:49
-120,Suzy,Dubery,631-694-1983,Female,5,54,24,2018-07-12 11:08:05,2018-07-21 06:22:14,2018-09-04 03:44:08,2018-11-14 04:59:58
-121,Reinaldo,Assel,927-809-0971,Male,3,4,44,2018-10-17 17:15:04,2018-12-24 21:34:45,2019-01-15 07:39:25,2018-02-06 19:16:38
-122,Claudell,Reckus,196-391-8789,Male,1,34,75,2018-08-12 02:29:09,2018-11-28 02:43:43,2018-08-21 09:11:37,2018-07-14 12:51:48
-123,Kary,Jee,656-508-4636,Female,2,67,47,2018-12-17 10:42:46,2018-05-16 14:59:10,2018-12-23 02:44:31,2018-12-19 01:28:44
-124,Giustino,Tinan,292-383-3412,Male,7,100,96,2018-02-05 08:16:51,2018-04-10 02:46:18,2018-12-07 23:33:47,2018-03-25 18:50:56
-125,Matthew,Palmer,562-655-9301,Male,5,87,100,2018-07-14 00:43:30,2018-01-22 19:53:17,2018-05-18 08:37:49,2018-04-16 10:03:59
-126,Helge,Paradyce,454-788-5166,Female,4,31,87,2018-12-29 12:15:52,2018-02-08 19:33:17,2018-12-29 06:51:56,2018-11-11 03:52:39
-127,Anthe,Cullinan,517-178-4457,Female,4,61,14,2018-07-10 05:46:54,2018-12-22 09:39:32,2018-12-06 16:59:53,2018-12-30 08:24:22
-128,Cindy,Bellie,284-376-7890,Female,6,26,95,2018-02-04 23:09:03,2018-09-06 16:30:50,2018-05-25 00:39:04,2018-12-20 19:03:33
-129,Deva,Branthwaite,272-973-9361,Female,5,4,74,2018-06-05 05:56:15,2018-06-26 03:23:22,2018-06-06 00:42:51,2018-12-01 12:32:06
-130,Karen,Frank,495-748-4715,Female,6,55,54,2018-10-26 02:19:25,2018-04-11 12:50:31,2019-01-12 15:29:42,2018-12-23 04:07:28
-131,Batsheva,Wressell,364-562-3308,Female,2,93,2,2018-12-08 11:50:47,2018-10-05 08:11:01,2018-12-29 23:04:37,2018-10-13 09:59:34
-132,Damian,Devon,617-622-5769,Male,1,81,84,2018-03-03 22:21:45,2018-02-09 02:46:29,2018-02-04 03:39:23,2018-06-24 17:15:50
-133,Mischa,Feldbrin,941-794-4981,Male,2,17,93,2018-07-05 04:20:06,2018-11-22 10:46:44,2018-10-25 23:05:17,2018-08-04 04:19:58
-134,Remus,Mochar,964-251-2864,Male,2,78,88,2018-05-08 23:51:18,2019-01-19 19:38:55,2018-06-13 09:14:44,2019-01-10 12:37:02
-135,Cleon,Ambroise,790-235-3681,Male,3,38,95,2018-08-11 11:07:40,2018-07-06 04:03:45,2018-12-18 09:47:22,2018-03-31 14:59:58
-136,Sidney,Duthy,739-113-2227,Male,4,95,79,2018-09-25 19:41:46,2018-09-05 07:30:26,2018-03-21 18:52:33,2018-04-25 13:58:28
-137,Xavier,Gon,942-422-7024,Male,5,35,47,2018-09-05 22:08:31,2018-05-10 20:19:08,2018-06-25 18:32:07,2018-04-11 04:44:32
-138,Ara,Gwyllt,450-595-4186,Male,4,4,28,2018-10-28 21:12:39,2018-02-13 08:26:45,2018-06-16 21:44:05,2018-08-01 11:08:43
-139,Libbey,Millwall,764-602-4598,Female,7,89,31,2019-01-14 17:04:08,2018-02-22 03:01:08,2018-02-27 08:56:59,2018-11-09 13:18:23
-140,Rosalynd,Wellard,798-378-3350,Female,4,58,44,2018-07-13 19:23:05,2018-11-20 20:46:20,2018-01-31 02:28:58,2018-10-25 13:31:05
-141,Reinaldo,O'Dea,896-868-5961,Male,2,72,69,2018-08-08 04:21:04,2018-06-23 08:34:34,2018-08-31 08:41:08,2018-11-19 09:43:28
-142,Veradis,Holyard,706-725-9446,Female,3,78,56,2018-09-05 17:38:33,2018-05-04 13:04:38,2018-11-05 12:46:29,2018-09-25 08:56:54
-143,Birdie,Collerd,319-380-7196,Female,2,22,1,2018-10-16 19:45:08,2018-08-05 06:38:47,2018-11-22 11:10:39,2018-10-20 09:48:22
-144,Sada,Schout,825-204-2108,Female,7,88,10,2018-04-13 13:44:13,2018-06-06 14:39:37,2018-11-09 10:29:00,2018-09-15 05:38:03
-145,Ambros,Mayoral,819-227-9360,Male,7,50,91,2018-03-27 15:27:05,2018-08-02 02:18:35,2018-06-27 17:10:44,2018-07-21 14:18:29
-146,Emile,Abraham,622-769-6584,Male,6,10,44,2018-09-15 10:24:46,2018-07-10 07:58:58,2018-06-18 22:49:01,2018-04-20 09:07:25
-147,Lira,Reboulet,983-623-7858,Female,4,54,39,2018-06-27 09:11:33,2018-10-01 07:52:19,2018-10-21 09:48:43,2018-09-09 22:17:51
-148,Bucky,Dubery,127-215-3377,Male,1,13,5,2018-02-15 22:03:55,2018-10-15 16:35:39,2018-02-20 10:22:26,2018-11-05 12:56:42
-149,Ced,Meert,482-685-6526,Male,1,85,40,2018-11-09 04:11:10,2019-01-16 14:34:41,2018-05-22 12:18:01,2018-03-13 13:09:46
-150,Hamlen,Boyson,723-641-9907,Male,7,3,25,2018-07-08 10:27:28,2018-09-06 08:34:00,2018-10-19 17:18:48,2018-02-22 19:31:50
-151,Scotty,Bromell,605-116-4137,Male,4,44,83,2018-02-03 11:10:35,2018-03-05 15:14:24,2018-06-20 01:47:46,2018-04-09 17:34:46
-152,Betti,Cuttelar,947-869-1635,Female,7,31,68,2018-07-14 23:35:44,2018-09-07 08:03:02,2018-05-02 15:32:32,2018-02-26 15:33:55
-153,Lothario,Whellams,959-671-9821,Male,3,91,85,2018-12-17 02:20:59,2018-08-31 16:54:24,2018-09-30 18:16:21,2018-06-06 21:30:55
-154,Maurits,Eisenberg,612-730-0598,Male,5,95,2,2018-07-11 05:38:06,2018-01-26 15:01:11,2018-11-11 22:30:48,2018-04-16 07:20:25
-155,Jeffry,Wilprecht,420-521-7374,Male,2,2,19,2018-07-31 12:28:42,2018-03-19 23:14:58,2018-11-29 11:25:28,2018-02-19 02:42:46
-156,Victor,Grinter,303-176-4062,Male,3,28,82,2018-04-15 17:22:33,2018-04-26 11:56:35,2018-10-06 23:26:58,2018-04-05 07:32:13
-157,Sonnie,Lobe,794-424-7798,Male,1,68,99,2018-12-23 13:36:28,2018-11-24 21:42:21,2018-03-23 12:37:58,2018-06-26 15:29:05
-158,Krishna,Fletcher,520-484-4343,Male,5,77,26,2018-05-15 11:17:35,2018-12-22 22:52:12,2018-05-10 22:50:32,2018-05-05 21:38:09
-159,Kerrie,Bartholomieu,654-465-0213,Female,4,7,56,2018-11-02 19:08:33,2018-01-27 09:11:25,2018-12-13 16:53:49,2018-05-06 13:16:06
-160,Rasla,Dibbs,192-692-8022,Female,5,58,38,2018-05-13 16:41:12,2018-12-17 01:00:54,2018-08-19 15:33:37,2019-01-11 00:33:40
-161,Mellisa,Prinn,884-570-1200,Female,3,2,44,2018-06-17 15:55:01,2018-02-09 22:16:46,2018-01-21 11:35:44,2018-03-22 01:43:34
-162,Shandeigh,Prise,284-912-2475,Female,1,24,63,2018-04-16 10:09:01,2018-09-06 02:59:22,2018-05-21 00:23:45,2018-05-18 15:59:58
-163,Aline,Pockey,145-621-3349,Female,5,46,99,2018-07-08 01:50:19,2018-07-23 03:08:04,2018-07-25 04:19:37,2018-08-02 04:28:24
-164,Quinton,Filon,646-285-5370,Male,7,71,90,2018-09-26 09:06:43,2018-09-15 14:33:28,2018-11-23 04:26:39,2019-01-20 06:07:08
-165,Brena,Bentke,336-549-2800,Female,6,52,73,2018-02-07 03:32:01,2018-09-21 09:41:58,2018-08-01 23:49:56,2018-04-24 04:41:42
-166,Kessia,Bentote,261-293-3478,Female,7,34,27,2019-01-03 17:12:12,2018-01-28 01:36:43,2018-09-06 07:29:49,2018-07-16 11:22:52
-167,Serene,Jellett,617-490-3045,Female,4,8,12,2018-08-01 05:18:42,2018-08-21 11:58:23,2018-02-17 11:36:57,2018-12-29 13:38:10
-168,Drugi,Chainey,642-531-9743,Male,1,27,14,2019-01-06 06:11:34,2018-11-02 16:07:48,2019-01-07 21:38:11,2018-10-26 19:58:30
-169,Benjamen,Angric,161-599-1534,Male,6,80,80,2018-03-25 19:49:25,2018-08-23 06:12:13,2018-05-09 05:22:56,2018-04-03 12:50:17
-170,Augustine,de Werk,986-128-9745,Male,7,40,35,2018-09-07 12:55:20,2018-09-15 16:46:53,2018-02-12 14:00:28,2018-02-20 11:54:00
-171,Rorke,Laidlaw,818-603-8046,Male,1,99,95,2018-12-25 21:27:03,2018-12-15 12:30:46,2018-11-06 04:14:51,2018-11-30 18:51:21
-172,Clywd,Cawdron,118-860-8931,Male,3,12,48,2018-10-28 00:46:43,2018-07-04 11:34:45,2018-01-24 04:40:57,2018-11-08 13:52:59
-173,Maudie,Habbijam,781-157-2628,Female,1,79,35,2018-11-30 12:07:21,2018-03-07 00:30:04,2018-11-14 11:01:28,2018-12-25 08:37:17
-174,Gina,Adamovitz,873-120-9534,Female,1,99,41,2018-10-15 08:16:08,2018-02-02 10:04:20,2018-03-11 05:28:50,2018-08-15 16:28:04
-175,Dorey,Berkeley,856-131-2818,Female,5,8,45,2018-06-14 02:47:10,2018-12-04 16:14:29,2018-02-20 02:19:22,2018-04-14 04:00:28
-176,Ashli,Stanistrete,507-365-9155,Female,3,50,85,2019-01-15 15:53:25,2018-09-08 00:21:37,2018-09-08 14:26:57,2018-08-12 18:04:42
-177,Aleta,Jessard,640-184-4123,Female,1,63,42,2018-12-13 11:45:35,2018-07-13 05:43:20,2018-07-19 01:29:59,2018-08-19 10:17:25
-178,Conney,Tansly,619-324-6788,Male,7,18,68,2018-02-06 08:22:33,2018-02-09 16:00:31,2018-02-17 20:35:12,2018-09-01 00:28:07
-179,Chaddie,Curling,967-821-4721,Male,3,50,28,2018-04-18 18:52:52,2018-10-04 03:42:09,2018-11-06 23:55:33,2018-12-23 04:01:13
-180,Skyler,Shard,173-185-5465,Male,7,48,76,2018-03-21 23:42:54,2018-05-22 07:52:04,2018-02-28 18:02:47,2018-08-20 09:21:32
-181,Alair,Bruggen,225-747-4685,Male,5,45,4,2018-03-13 03:01:40,2018-04-30 23:24:34,2018-04-25 03:14:01,2018-07-26 19:41:29
-182,Forester,Rackstraw,111-188-6870,Male,6,11,43,2018-09-26 04:05:12,2018-08-25 04:59:08,2018-02-09 16:50:13,2018-06-10 06:53:15
-183,Stinky,Klassmann,628-447-6394,Male,5,42,77,2018-12-29 18:37:51,2018-01-31 04:16:33,2018-05-27 09:10:31,2018-02-03 20:09:44
-184,Gabriella,Weeds,978-256-0633,Female,3,17,50,2018-10-10 14:56:11,2019-01-19 07:53:13,2018-08-28 10:02:39,2018-05-03 15:01:38
-185,Ingeborg,Swine,765-792-5768,Female,5,39,37,2018-10-13 03:17:12,2018-11-25 20:17:22,2018-08-25 21:54:45,2018-12-04 07:44:58
-186,Kain,Everingham,945-715-9320,Male,7,52,80,2018-10-22 01:58:35,2018-04-16 11:48:35,2018-02-22 17:07:19,2018-12-15 01:59:16
-187,Noble,Swadlinge,928-764-8545,Male,7,24,75,2018-07-02 20:12:53,2018-07-19 00:23:08,2018-08-06 21:50:01,2018-05-04 08:56:30
-188,Amber,Shropshire,983-928-3309,Female,1,100,37,2018-03-18 04:32:13,2018-08-09 03:16:35,2018-11-21 04:25:29,2018-10-03 06:20:54
-189,Hedy,Sapena,301-104-2652,Female,3,37,35,2018-07-04 12:54:06,2018-10-12 20:33:48,2018-04-10 07:32:31,2018-03-17 10:12:03
-190,Nerissa,Jedras,425-605-9508,Female,1,18,31,2019-01-12 14:05:35,2018-10-19 14:28:35,2018-06-14 21:07:02,2018-03-22 05:53:32
-191,Terrel,Kear,681-621-2157,Male,3,98,45,2018-02-20 10:42:05,2018-11-08 04:29:14,2018-03-09 02:33:02,2018-07-04 18:27:00
-192,Dennison,Theodoris,722-911-1627,Male,6,76,32,2018-11-18 05:15:23,2018-05-13 02:02:05,2019-01-15 21:45:30,2018-04-13 17:24:17
-193,Chrystel,Ferentz,497-580-2963,Female,2,96,6,2018-12-11 03:17:23,2018-05-23 04:35:09,2018-11-01 17:51:50,2018-08-02 02:44:29
-194,Devland,Butchers,313-172-9223,Male,6,42,38,2019-01-16 07:17:37,2018-11-09 17:37:48,2018-04-10 08:57:30,2018-05-02 13:47:28
-195,Simon,Astupenas,981-207-6441,Male,2,57,43,2018-11-07 15:02:00,2018-01-25 17:30:33,2018-07-13 05:08:19,2018-10-29 18:37:22
-196,Jordan,Attride,447-912-6034,Female,7,25,37,2018-06-06 14:29:43,2018-06-30 03:49:38,2018-12-09 18:11:23,2018-01-30 19:12:46
-197,Clerissa,Forsard,485-444-1412,Female,2,88,66,2018-09-19 00:56:28,2018-07-23 15:36:37,2018-11-17 11:43:09,2018-11-17 17:11:53
-198,Alastair,Toth,495-980-1189,Male,1,62,59,2018-03-04 17:35:31,2018-06-18 07:02:22,2018-07-30 02:31:33,2018-07-20 14:21:29
-199,Aldo,Bruton,488-482-2130,Male,2,33,99,2018-06-09 23:01:18,2018-08-22 04:11:53,2018-07-17 05:55:24,2018-04-08 10:13:23
-200,Lorin,Mourant,953-195-5245,Male,1,78,78,2018-02-04 07:07:36,2018-08-05 08:03:40,2018-11-22 13:04:30,2018-03-10 12:32:28
-201,Nonnah,Dreini,137-834-9350,Female,1,20,84,2018-11-23 21:47:29,2018-07-02 22:51:42,2018-05-28 23:33:57,2018-06-11 11:55:17
-202,Winnie,Muris,865-681-2171,Female,2,31,76,2018-10-29 21:53:21,2018-10-12 17:59:08,2018-03-21 12:37:41,2018-06-26 05:24:49
-203,Marybelle,Wedmore.,566-812-6518,Female,6,94,20,2018-04-08 17:51:07,2018-01-25 19:29:58,2018-05-29 08:33:21,2018-12-18 09:29:05
-204,Christophorus,La Batie,776-298-7693,Male,2,100,44,2018-11-26 01:22:44,2018-11-28 19:33:42,2018-01-26 20:34:09,2018-10-15 19:29:24
-205,Erhart,Laydon,425-854-5387,Male,7,15,35,2018-11-09 09:53:34,2018-11-11 01:43:14,2019-01-16 12:20:32,2018-08-01 09:34:47
-206,Myrta,Playfair,512-835-4158,Female,4,8,51,2018-05-04 15:28:01,2018-05-11 22:03:31,2018-05-21 02:28:31,2018-03-17 08:25:48
-207,Tybie,McGhee,155-902-0043,Female,5,25,55,2018-03-19 17:19:34,2018-11-11 01:53:03,2018-06-01 16:21:08,2018-08-21 09:39:26
-208,Augusto,Chelam,595-490-1187,Male,2,20,22,2018-02-26 17:51:20,2018-04-15 11:38:21,2018-11-29 19:44:24,2018-05-17 16:31:14
-209,Louisette,Firbanks,365-745-5229,Female,1,51,35,2018-04-17 19:42:14,2018-03-19 09:56:15,2018-04-22 09:48:27,2018-11-20 05:52:44
-210,Brucie,Speers,688-351-7130,Male,3,65,20,2018-09-22 06:54:04,2018-08-14 23:05:08,2018-03-13 17:38:11,2018-12-29 06:47:13
-211,Nathanial,Ayliff,974-831-8727,Male,1,23,95,2018-09-02 06:27:24,2018-06-18 22:51:54,2018-12-25 13:15:52,2018-03-09 05:29:17
-212,Shawnee,McClenan,177-754-4767,Female,4,54,82,2018-04-17 14:01:32,2018-08-15 14:02:31,2019-01-04 03:04:30,2018-10-20 15:25:45
-213,Giavani,Cay,976-869-6321,Male,1,92,27,2018-07-22 17:43:03,2018-02-20 03:19:34,2018-05-06 13:52:36,2018-12-05 18:14:41
-214,Claudia,Fazackerley,454-663-9623,Female,4,26,25,2019-01-07 16:15:34,2018-03-06 15:31:53,2018-02-06 23:08:37,2018-11-13 17:45:22
-215,Nevil,Phelp,255-488-7211,Male,5,22,40,2018-09-11 19:55:44,2018-05-13 12:07:30,2018-08-27 02:20:49,2019-01-17 21:11:30
-216,Elle,Antal,303-999-5794,Female,4,23,5,2018-02-15 19:05:09,2018-10-15 01:01:31,2018-12-31 16:08:16,2018-03-23 15:16:38
-217,Brnaby,Dewsbury,173-850-9221,Male,3,82,22,2018-05-19 14:05:10,2018-12-30 00:25:40,2018-09-27 06:32:26,2018-04-25 02:17:22
-218,Ronda,Dranfield,835-657-5998,Female,5,61,54,2018-10-18 03:51:07,2018-07-20 06:01:19,2018-11-17 15:09:04,2018-12-04 20:06:06
-219,Vania,Brannan,288-990-8986,Female,3,88,25,2018-07-03 11:19:29,2018-11-13 09:44:15,2018-02-28 13:57:43,2018-02-28 05:42:56
-220,Jessy,Baggelley,848-274-4914,Female,7,38,75,2018-12-04 13:44:16,2018-11-20 01:53:14,2018-02-25 03:07:40,2018-06-17 19:02:59
-221,Paxon,Kensall,262-215-6295,Male,1,3,63,2018-08-12 21:30:25,2018-11-24 01:13:12,2018-06-04 19:20:59,2018-08-16 14:48:22
-222,Tana,O'Donohue,514-853-5855,Female,4,99,1,2018-03-21 06:05:58,2018-05-01 00:39:55,2018-03-10 21:55:54,2018-10-31 10:25:10
-223,Thomasina,Gibbons,501-364-8398,Female,3,37,56,2018-11-22 03:07:49,2018-06-30 16:44:00,2018-08-05 18:41:13,2019-01-07 06:21:44
-224,Bellanca,Golston,673-979-3362,Female,1,57,53,2018-05-02 00:04:42,2018-01-21 22:51:57,2018-08-20 18:40:05,2018-02-09 08:22:11
-225,Maureene,Drejer,769-639-6612,Female,7,49,17,2018-02-20 12:00:24,2018-09-01 04:44:08,2018-08-15 22:04:06,2018-09-04 06:40:59
-226,Beatrisa,Blanche,914-368-0909,Female,4,47,77,2018-05-21 06:41:29,2018-10-04 19:21:53,2018-07-11 04:03:32,2018-04-05 01:26:21
-227,Melita,Filipczak,743-692-1202,Female,2,18,45,2018-04-04 09:49:49,2018-03-30 16:30:51,2019-01-09 03:34:15,2018-02-05 00:54:24
-228,Jean,Rzehor,983-469-9959,Male,2,76,95,2018-08-17 14:15:05,2018-09-25 10:05:56,2018-09-04 10:03:08,2018-02-16 04:46:04
-229,Netti,Odeson,140-282-6021,Female,7,46,99,2018-04-08 08:08:21,2018-05-07 06:33:55,2018-03-10 06:30:39,2018-08-27 01:08:06
-230,Lark,Komorowski,889-340-2588,Female,6,36,54,2018-05-27 12:55:35,2018-11-18 05:20:17,2018-11-19 04:18:03,2018-04-24 14:21:19
-231,Mae,Howells,972-510-2059,Female,2,54,97,2019-01-17 19:46:20,2018-07-10 05:29:24,2018-07-06 16:03:28,2018-07-18 17:41:40
-232,Brewer,Wolstencroft,243-516-4986,Male,4,68,84,2018-07-19 01:48:48,2018-12-31 10:44:05,2018-05-25 20:15:52,2018-04-27 22:38:48
-233,Chrystel,Buist,285-587-4608,Female,3,66,90,2018-02-06 05:43:09,2018-06-10 20:45:59,2018-11-27 09:40:02,2018-12-30 22:43:43
-234,Nicola,Feltoe,102-119-8958,Female,6,30,100,2018-05-26 00:46:42,2018-03-17 00:12:12,2019-01-16 18:40:38,2018-01-27 09:20:23
-235,Engelbert,Matveiko,868-192-6148,Male,3,3,6,2018-05-27 06:16:04,2018-12-24 00:22:43,2018-04-18 07:21:59,2018-12-03 08:49:41
-236,Lezley,M'Chirrie,749-737-2204,Male,4,16,17,2018-07-06 21:48:48,2018-02-09 05:08:49,2018-09-05 10:59:16,2018-03-22 01:13:02
-237,Josy,Larkins,826-241-0108,Female,1,8,18,2018-10-26 03:28:37,2018-02-24 22:01:17,2018-12-29 22:31:43,2018-05-23 06:28:27
-238,Armando,Girt,296-631-5842,Male,1,90,62,2018-03-14 20:02:06,2018-02-25 03:43:37,2018-05-29 18:43:28,2018-10-10 16:04:29
-239,Nevins,Boch,243-865-0986,Male,7,87,34,2018-04-27 05:11:05,2018-06-13 18:02:47,2018-04-17 23:28:45,2018-03-11 20:07:50
-240,Gordan,MacHoste,194-807-0099,Male,2,8,10,2018-11-30 11:09:40,2018-03-19 08:40:39,2018-07-07 07:58:57,2018-12-03 15:54:45
-241,Vlad,Urling,756-416-9308,Male,6,44,75,2018-07-01 08:10:44,2018-06-25 11:50:00,2018-02-18 09:41:26,2019-01-20 16:56:43
-242,Karney,Batteson,884-299-0885,Male,3,4,70,2018-04-21 18:15:47,2018-03-17 15:26:35,2018-09-21 06:49:36,2018-03-03 10:02:05
-243,Shep,Croisier,182-249-1703,Male,2,25,64,2018-11-27 15:36:38,2018-03-27 09:16:11,2018-07-27 14:14:49,2018-08-31 21:47:02
-244,Jackelyn,Franschini,203-947-2285,Female,5,93,18,2018-05-11 18:05:34,2018-07-09 02:56:23,2018-05-26 13:31:09,2018-11-12 13:51:54
-245,Kingston,Brazier,388-460-9266,Male,5,67,43,2018-11-17 16:06:03,2018-10-22 21:39:40,2018-11-03 04:17:45,2018-08-23 17:37:21
-246,Derwin,Postin,334-746-9914,Male,2,9,74,2018-04-28 10:39:48,2018-10-16 21:01:17,2019-01-17 12:54:27,2018-09-18 06:08:12
-247,Rustie,Boddington,499-187-3081,Male,7,75,92,2018-07-09 18:05:33,2018-09-15 11:50:52,2018-06-10 05:00:00,2018-09-23 05:52:23
-248,Araldo,MacConnulty,922-950-0398,Male,4,66,28,2018-11-11 12:46:18,2018-09-10 16:05:23,2018-09-21 00:16:24,2018-10-10 19:06:57
-249,Brear,Zuanelli,948-803-5705,Female,7,88,15,2018-03-03 08:12:37,2018-10-06 14:14:00,2018-09-06 21:02:50,2018-04-04 06:13:25
-250,Ruttger,Cartan,503-123-4758,Male,6,8,46,2018-07-02 10:33:15,2018-10-11 08:51:25,2018-02-16 18:01:35,2018-05-15 00:25:35
-251,Leta,Gravenell,798-643-9817,Female,6,19,50,2018-09-26 18:01:14,2018-12-31 01:08:01,2018-08-12 01:18:26,2018-08-12 11:04:31
-252,Nanny,Spellward,298-614-5133,Female,2,78,35,2018-07-31 14:38:53,2018-07-11 11:50:42,2018-06-24 03:47:24,2018-08-23 06:25:24
-253,Garwin,Overil,981-304-0345,Male,1,66,62,2018-02-20 05:57:01,2018-02-04 20:02:37,2018-04-21 16:30:20,2018-07-27 08:45:03
-254,Ancell,Bromidge,677-576-3535,Male,1,54,47,2018-02-27 19:07:09,2018-07-07 17:41:39,2018-03-14 05:31:23,2018-11-18 21:30:30
-255,Harmonia,McIleen,610-808-3838,Female,5,76,7,2018-05-09 14:49:35,2018-04-12 02:30:13,2019-01-17 21:59:23,2018-04-20 05:21:50
-256,Ginni,See,809-189-3574,Female,7,93,45,2018-04-11 20:22:07,2018-08-17 13:12:57,2018-04-20 06:30:33,2018-04-17 00:12:41
-257,Christi,Doorly,614-386-8893,Female,7,23,11,2018-08-09 05:01:14,2018-03-11 21:19:53,2018-05-09 23:37:07,2018-03-27 22:07:11
-258,Perice,Severy,433-906-1030,Male,1,18,5,2018-09-01 12:30:04,2018-09-13 05:34:16,2019-01-16 22:35:35,2019-01-09 20:31:49
-259,Dulsea,Yokel,279-965-4206,Female,2,93,50,2018-10-31 22:48:09,2018-07-13 09:25:20,2018-02-16 09:15:58,2018-05-12 21:55:26
-260,Marge,Cheng,549-350-8693,Female,2,10,68,2018-09-01 18:42:35,2018-04-07 10:58:53,2018-07-30 16:05:15,2018-10-20 05:45:19
-261,Baron,McIlmorow,283-456-4615,Male,2,41,89,2018-11-07 01:23:03,2018-12-20 18:25:28,2018-08-31 12:49:04,2018-05-27 07:39:41
-262,Rabbi,Beaglehole,904-724-1771,Male,2,94,67,2018-11-04 08:21:47,2018-12-28 20:54:40,2018-12-08 10:20:14,2018-11-29 19:47:40
-263,Olvan,Asher,310-502-8368,Male,7,76,67,2018-03-15 22:02:48,2018-07-30 12:08:26,2018-12-06 22:49:46,2018-06-17 22:03:05
-264,Yard,Beldon,209-784-5636,Male,7,90,9,2018-07-24 02:02:11,2018-05-11 03:54:15,2018-08-20 18:46:55,2018-03-10 04:11:33
-265,Enriqueta,Barraclough,332-372-0081,Female,6,75,69,2018-08-18 05:08:19,2018-01-27 11:36:23,2018-05-23 08:59:06,2018-02-11 04:14:36
-266,Alli,Smallcomb,255-741-1276,Female,2,8,1,2018-12-06 01:19:35,2018-07-16 16:10:12,2018-07-16 13:31:05,2018-10-26 07:03:09
-267,Shay,Amsden,816-821-9426,Male,6,25,35,2018-03-15 01:09:39,2018-11-26 05:57:30,2018-10-26 07:54:05,2018-04-14 01:42:13
-268,Jacynth,Boscott,970-121-0282,Female,1,88,89,2018-07-31 16:10:10,2018-05-10 10:28:44,2018-10-02 23:00:23,2018-07-02 13:55:14
-269,Dominique,Farny,782-337-7486,Male,1,17,29,2018-09-09 16:09:20,2018-04-10 15:04:16,2018-03-25 16:40:27,2019-01-04 20:02:55
-270,Keefe,Guerrero,440-254-1961,Male,4,96,29,2018-10-19 10:06:55,2018-07-28 18:31:17,2018-05-05 19:05:12,2018-06-01 05:55:25
-271,Buiron,Anfossi,512-333-8553,Male,2,92,51,2018-09-09 16:00:30,2018-02-20 14:27:22,2018-03-11 07:29:35,2018-07-12 02:19:45
-272,Eleonore,Puller,325-656-5909,Female,4,88,37,2018-12-04 20:39:40,2018-02-13 04:35:07,2018-08-16 18:26:05,2018-06-08 07:45:57
-273,Katuscha,Rasch,427-956-1046,Female,4,68,90,2018-06-19 01:20:15,2018-04-03 01:55:07,2018-04-17 19:09:09,2018-08-15 11:04:06
-274,Hoebart,Linger,972-926-3923,Male,4,74,74,2018-02-19 01:23:42,2018-01-26 15:34:23,2018-09-13 07:10:43,2018-06-07 23:26:33
-275,Patrick,Hryskiewicz,736-622-7490,Male,3,91,1,2019-01-13 09:57:23,2018-05-06 15:42:09,2018-07-01 17:16:14,2018-09-01 02:26:15
-276,Bert,Adrianello,485-126-1547,Female,7,33,12,2018-11-01 06:34:27,2018-09-14 23:29:20,2018-03-11 08:15:03,2018-09-21 21:53:26
-277,Burton,Spittall,570-586-9893,Male,7,46,33,2018-01-28 06:20:39,2018-06-15 19:15:06,2018-06-27 02:36:30,2018-03-24 02:52:03
-278,Tessy,Mitchley,167-983-7021,Female,5,45,26,2018-03-02 16:44:00,2018-12-25 18:48:13,2018-04-01 18:27:41,2018-04-03 20:39:05
-279,Nettle,Hause,489-296-7367,Female,2,99,15,2018-03-18 02:50:49,2018-03-02 01:31:53,2018-10-31 06:40:34,2018-04-28 11:21:14
-280,Pearce,Darwent,801-341-9474,Male,2,24,44,2018-05-28 23:54:50,2018-02-07 18:37:22,2018-09-11 02:48:52,2018-11-09 16:24:17
-281,Kelsey,Suttle,536-591-5133,Female,6,11,76,2018-02-14 19:05:31,2018-07-14 19:32:27,2018-03-26 04:59:22,2018-04-01 20:21:57
-282,Gillie,Sponton,270-390-2110,Female,2,94,57,2018-03-14 11:50:53,2018-03-27 13:10:53,2018-04-23 19:54:11,2018-08-22 21:53:12
-283,Marsha,Dietz,740-837-1891,Female,5,90,95,2018-03-18 04:53:06,2018-06-29 02:28:55,2018-08-10 05:14:09,2018-05-10 04:03:25
-284,Miles,Purchon,413-518-8673,Male,2,57,98,2018-04-02 15:49:11,2018-02-04 16:57:33,2018-09-09 22:23:44,2018-03-16 04:00:43
-285,Collette,Menego,440-637-0309,Female,3,21,56,2018-04-18 04:27:48,2018-05-05 08:32:40,2018-03-17 17:54:26,2018-04-12 00:24:45
-286,Heriberto,Dunkerton,321-657-4492,Male,5,99,91,2018-08-17 12:11:42,2018-10-26 14:33:07,2018-03-10 13:13:28,2018-05-03 02:14:30
-287,Hailee,Remington,671-722-2287,Female,6,69,53,2018-08-01 01:54:04,2018-08-20 04:05:47,2018-04-24 14:22:04,2018-11-02 14:15:55
-288,Ingunna,Errigo,378-401-4671,Female,3,83,66,2018-12-22 21:52:13,2018-12-24 18:41:25,2018-11-11 17:31:47,2018-06-27 14:34:11
-289,Barbie,Grishmanov,184-949-4040,Female,7,93,68,2018-08-14 21:25:04,2018-08-26 18:27:48,2018-04-28 10:50:33,2018-08-06 14:12:21
-290,Anderson,Freeland,842-434-1249,Male,6,22,71,2019-01-05 09:24:32,2018-06-13 07:19:06,2018-04-24 22:20:16,2018-05-31 21:18:59
-291,Olympie,Prantl,988-864-1736,Female,1,39,40,2018-04-06 05:52:26,2018-03-10 03:20:54,2018-02-06 16:53:55,2018-09-03 16:18:27
-292,Amos,Lanyon,552-486-3382,Male,1,17,35,2018-07-03 20:26:09,2018-11-16 06:37:27,2018-08-16 14:36:30,2018-04-07 19:59:08
-293,Gabby,Stockau,753-586-0632,Male,5,19,14,2018-12-10 13:52:07,2018-03-20 20:50:50,2018-04-11 20:54:39,2018-11-13 17:11:21
-294,Uta,Joan,660-241-5130,Female,7,67,41,2018-04-21 18:11:39,2018-07-12 17:47:45,2018-05-20 14:57:17,2018-09-19 10:15:02
-295,Elnar,Manjot,691-939-7359,Male,5,18,86,2018-11-24 20:27:24,2018-04-29 05:45:30,2018-09-06 04:15:04,2018-05-31 17:11:55
-296,Avram,Cowins,471-336-0418,Male,1,18,23,2018-10-07 19:35:52,2018-07-24 19:33:54,2018-05-09 12:43:10,2018-10-08 09:08:37
-297,Kyla,Fines,380-936-0659,Female,7,16,6,2018-12-04 05:02:51,2018-06-13 12:25:14,2018-05-15 10:17:43,2018-07-24 13:06:26
-298,Tymothy,Trevillion,740-586-6434,Male,4,99,5,2018-11-29 03:58:39,2018-07-10 01:02:27,2018-07-05 11:15:40,2018-07-17 11:17:00
-299,Rafaelita,Stiven,313-253-8470,Female,7,94,91,2018-05-11 22:09:50,2018-12-04 03:23:26,2018-06-04 22:13:06,2018-08-02 01:03:16
-300,Stanwood,Bracci,757-577-7619,Male,3,79,15,2018-06-02 08:42:52,2018-05-01 23:25:28,2018-10-27 06:54:40,2019-01-03 19:33:39
-301,Heddie,Pegden,750-484-0458,Female,3,85,74,2018-02-15 15:15:50,2018-08-13 10:31:12,2018-10-22 05:36:44,2018-07-27 21:33:04
-302,Letty,Ipplett,534-461-3389,Female,2,24,58,2018-12-30 15:03:42,2018-12-25 22:41:39,2018-08-06 08:13:19,2018-06-13 08:22:24
-303,Clarke,Buckberry,261-218-0110,Male,4,39,35,2018-11-25 02:26:48,2018-10-01 22:53:26,2019-01-01 07:32:49,2018-06-23 02:36:18
-304,Valina,Bouskill,292-930-7629,Female,2,63,32,2018-03-13 00:24:33,2018-12-08 10:39:51,2019-01-06 10:27:01,2018-07-13 00:19:13
-305,Jeth,Mosedall,326-529-0770,Male,7,44,30,2019-01-17 17:26:19,2018-03-23 23:40:12,2018-12-16 02:11:33,2018-09-08 07:21:24
-306,Mano,Komorowski,467-693-9454,Male,5,63,33,2018-12-05 21:37:13,2018-05-15 16:54:39,2018-11-21 08:06:01,2018-12-25 19:18:03
-307,Libby,Worters,651-420-4117,Female,6,62,73,2018-11-16 15:37:33,2018-04-10 08:47:08,2018-09-25 07:08:44,2018-12-20 18:11:17
-308,Ginnie,Dunnion,884-641-9401,Female,4,15,91,2018-02-25 04:25:07,2018-06-11 12:12:35,2018-12-20 05:03:34,2018-05-12 15:41:13
-309,Addi,Butterfill,454-471-6991,Female,6,45,37,2019-01-14 00:57:24,2018-10-22 12:37:53,2018-11-12 17:11:49,2018-04-13 08:39:20
-310,Eustace,Calway,260-583-1874,Male,3,98,96,2018-12-08 14:19:50,2018-03-09 04:33:52,2018-08-03 22:32:38,2018-12-17 16:12:12
-311,Greta,Tailour,864-452-3785,Female,5,63,78,2018-05-09 23:36:15,2018-09-16 21:21:51,2018-06-17 03:49:40,2018-06-25 12:40:03
-312,Nance,Ruston,102-299-0194,Female,4,66,33,2018-04-19 13:30:14,2018-08-22 20:55:14,2018-03-02 20:03:46,2018-03-23 09:40:10
-313,Harriette,Dicty,498-183-6908,Female,7,1,13,2018-12-31 04:35:08,2018-08-13 12:05:13,2018-04-30 22:42:00,2018-12-03 13:23:45
-314,Merrielle,Mapes,918-608-0262,Female,5,47,45,2018-12-16 07:21:41,2018-10-27 21:05:00,2018-03-23 14:05:12,2018-08-02 11:02:53
-315,Gris,Scurrell,392-726-5529,Male,4,71,95,2018-02-23 03:36:40,2018-04-09 20:52:39,2018-03-07 02:56:27,2018-10-22 11:26:37
-316,Novelia,Newdick,262-887-9640,Female,3,20,97,2018-05-18 12:00:02,2018-02-04 08:00:28,2018-01-28 09:59:37,2018-07-06 05:53:32
-317,Sean,Breton,304-133-1524,Male,4,91,49,2018-07-09 23:07:45,2018-10-27 06:20:49,2018-08-22 21:03:24,2018-06-13 05:33:30
-318,Winston,Springham,259-758-2788,Male,6,86,20,2018-03-20 19:07:57,2018-03-17 14:55:01,2018-08-05 20:39:20,2018-02-11 22:19:58
-319,Clemmie,Saywood,341-505-9072,Male,4,61,30,2018-06-08 02:47:28,2018-07-03 05:59:45,2018-10-28 00:28:02,2018-04-07 23:20:59
-320,Robinia,Vida,537-434-4616,Female,6,19,32,2018-05-03 10:52:24,2018-10-12 00:17:12,2018-11-27 23:47:32,2018-12-17 00:24:59
-321,Findley,Melanaphy,423-892-4436,Male,1,100,31,2018-02-04 19:41:01,2018-08-17 10:08:47,2018-11-04 14:37:46,2018-07-01 11:45:21
-322,Brooks,O'Keenan,840-842-8591,Female,4,82,84,2018-08-21 21:40:21,2018-08-31 04:07:15,2018-02-01 02:25:03,2018-12-09 10:50:21
-323,Caz,Leaf,797-315-7239,Male,6,75,22,2018-10-17 18:51:00,2018-11-22 19:03:52,2018-02-15 06:48:13,2018-04-19 22:25:23
-324,Ad,Harvey,336-617-2629,Male,4,46,5,2018-07-01 18:17:34,2018-05-24 05:31:23,2018-04-05 21:04:43,2018-07-21 06:50:26
-325,Berrie,Robardley,671-363-7428,Female,2,46,55,2018-05-14 01:56:01,2018-09-20 02:21:44,2019-01-14 15:24:34,2018-05-14 15:49:53
-326,Corliss,Halwell,154-276-2412,Female,3,63,16,2018-12-21 10:15:57,2018-11-26 00:14:11,2018-10-24 18:09:31,2018-05-27 11:41:11
-327,Carlina,Laird,357-488-1031,Female,5,59,27,2018-10-24 22:33:19,2018-10-23 00:27:39,2018-10-26 13:59:23,2018-07-19 02:41:32
-328,Rivalee,Bartolomucci,299-435-2483,Female,2,86,75,2018-06-02 23:51:58,2018-10-18 07:08:38,2019-01-11 07:36:58,2018-06-06 22:53:26
-329,Eleen,Girardy,254-561-4781,Female,6,99,58,2018-08-12 03:49:21,2019-01-04 17:17:54,2018-07-07 17:22:19,2018-02-08 00:26:15
-330,Junina,Scarisbrick,668-657-6435,Female,2,82,52,2018-10-08 01:58:04,2019-01-18 08:27:33,2018-10-13 10:36:18,2018-04-09 10:28:35
-331,Mufi,Insole,497-136-1156,Female,1,3,77,2018-07-01 05:18:23,2018-04-25 06:55:14,2018-12-23 07:35:00,2018-09-14 19:25:46
-332,Burk,Sawday,758-694-5610,Male,1,16,37,2018-05-18 17:42:50,2018-09-01 06:27:12,2018-05-06 06:30:31,2018-11-29 12:15:41
-333,Jess,Gully,330-430-8050,Female,5,2,75,2018-04-21 00:37:44,2018-06-12 02:34:07,2018-05-04 05:50:38,2018-11-25 21:02:42
-334,Dayna,Goublier,395-283-7995,Female,1,3,58,2018-03-10 13:08:34,2018-04-01 05:11:59,2018-07-17 13:31:35,2019-01-02 10:31:32
-335,Shandy,Grzegorczyk,271-236-1931,Female,6,32,34,2018-10-24 18:00:50,2018-03-17 19:11:58,2018-07-13 09:20:10,2018-07-07 03:03:48
-336,Alec,Nunnery,471-774-4412,Male,3,85,94,2018-02-15 09:59:45,2018-01-30 12:50:03,2018-03-01 13:47:47,2018-09-28 11:00:16
-337,Marti,Bedboro,799-912-3939,Female,3,88,26,2018-06-25 11:23:21,2018-10-30 08:43:52,2018-01-29 08:37:23,2018-06-06 00:28:49
-338,Malchy,Kobpac,819-568-9360,Male,4,9,14,2018-02-24 04:10:07,2018-10-05 18:35:23,2018-06-15 18:09:22,2018-11-15 08:05:17
-339,Egan,Dobbinson,366-886-7331,Male,5,43,13,2018-03-14 16:50:59,2018-08-08 23:17:17,2018-05-14 23:29:30,2018-08-19 04:54:16
-340,Maisie,Legg,700-620-6322,Female,7,85,21,2018-08-24 21:35:43,2018-08-30 17:01:27,2018-09-26 01:11:55,2018-12-29 12:04:35
-341,Fidel,Cossum,677-926-2862,Male,3,37,35,2018-07-26 05:42:32,2018-03-10 10:55:00,2018-12-22 17:20:15,2018-12-23 18:00:25
-342,Artemis,Beaton,301-959-5912,Male,7,15,26,2019-01-16 02:54:03,2018-10-28 11:08:44,2018-04-03 17:01:17,2018-04-13 04:28:34
-343,Culver,MacShirrie,126-770-5278,Male,4,28,43,2018-04-18 15:27:55,2018-08-01 10:35:51,2018-03-25 11:19:36,2018-06-01 19:23:11
-344,Jessa,Penreth,684-809-2622,Female,7,59,63,2018-08-04 18:24:37,2018-12-16 20:09:17,2018-04-11 13:14:28,2018-04-22 20:10:29
-345,Nydia,Hefner,535-519-6688,Female,3,70,29,2018-07-22 18:53:43,2018-09-02 00:30:03,2018-10-29 22:17:37,2018-05-07 11:46:24
-346,Ber,Naulty,311-674-2440,Male,6,85,70,2018-09-03 15:50:38,2018-07-04 22:30:51,2018-06-29 11:55:09,2018-05-16 11:27:56
-347,Margit,Greder,417-702-7174,Female,3,35,4,2018-04-28 01:58:34,2018-07-07 16:48:11,2018-10-08 13:45:45,2018-07-30 06:54:10
-348,Haroun,Kleynen,447-297-5270,Male,1,91,87,2018-05-14 13:40:54,2018-02-06 20:20:54,2018-04-24 19:09:01,2018-07-08 15:20:39
-349,Aylmer,Hubner,861-836-0360,Male,7,92,93,2018-02-10 06:55:44,2018-05-09 06:47:11,2018-07-09 15:46:54,2018-06-24 16:38:43
-350,Natty,Clemenzo,127-593-4758,Male,2,26,56,2018-02-01 13:02:29,2018-08-21 13:26:29,2018-12-24 09:27:03,2018-12-03 17:23:39
-351,Adriaens,Galier,704-551-4514,Female,1,64,55,2018-02-20 14:06:58,2018-09-17 22:53:46,2018-06-05 21:13:53,2018-02-25 03:51:27
-352,Raphaela,Higounet,493-470-8620,Female,7,72,86,2018-04-06 23:54:14,2018-11-09 00:01:09,2018-04-16 13:28:28,2018-10-15 09:06:54
-353,Nadia,Cureton,243-413-5780,Female,7,16,92,2018-10-20 06:35:07,2018-04-22 20:08:35,2018-04-07 15:26:39,2018-07-12 20:04:06
-354,Elliot,Bursell,301-524-4971,Male,6,59,44,2018-06-15 18:56:07,2018-03-14 21:30:37,2018-04-10 07:08:26,2018-10-03 23:57:13
-355,Miltie,Credland,318-601-5704,Male,3,5,47,2018-11-14 17:21:39,2018-04-18 22:02:24,2018-05-29 11:29:17,2018-08-28 19:44:37
-356,Andre,Dockray,631-438-3562,Male,2,1,36,2018-10-08 05:05:08,2018-10-07 10:51:00,2018-03-15 03:21:37,2018-08-31 02:22:35
-357,Marty,Vannah,864-526-0455,Male,5,32,9,2018-07-24 00:22:10,2018-08-26 02:40:08,2018-06-27 12:17:04,2018-05-23 03:24:49
-358,Timofei,Paddeley,941-127-2852,Male,4,75,28,2018-06-23 03:54:48,2018-06-16 19:23:26,2018-01-22 23:50:44,2018-09-17 11:00:42
-359,Susie,Bister,717-420-9064,Female,2,35,88,2018-08-09 20:57:50,2018-12-31 14:45:14,2018-04-16 17:44:38,2018-04-22 00:14:47
-360,Salvatore,Saenz,433-114-2254,Male,7,20,65,2018-02-14 15:55:26,2018-10-12 01:07:22,2018-11-29 04:17:51,2018-03-26 04:48:55
-361,Orin,Buttler,311-657-3727,Male,6,34,55,2018-04-05 00:11:44,2018-03-18 20:58:52,2018-09-14 10:40:07,2018-08-10 05:15:21
-362,Darbee,Garnam,956-529-4862,Male,1,75,26,2018-04-10 03:09:44,2018-06-29 01:39:43,2018-08-22 22:42:21,2018-09-15 23:23:04
-363,Lynne,Order,955-214-1203,Female,3,7,62,2018-11-15 23:40:31,2019-01-19 17:41:26,2018-02-14 19:18:25,2018-05-13 10:24:56
-364,Quinn,Hickinbottom,161-623-2666,Female,4,55,12,2018-10-24 13:06:05,2018-05-14 12:29:26,2018-10-03 06:49:08,2018-08-31 05:23:27
-365,Fionnula,Lonsbrough,271-974-9269,Female,2,24,86,2018-11-26 13:04:53,2018-04-30 00:03:02,2018-12-10 17:59:39,2018-11-02 17:56:01
-366,Blinni,Bound,991-243-7726,Female,3,34,55,2018-03-15 03:25:08,2018-07-17 18:13:35,2018-01-22 14:47:57,2018-07-03 10:31:07
-367,Joellyn,Duddin,562-523-2225,Female,3,91,7,2018-03-05 16:54:40,2018-07-11 18:24:57,2018-01-27 22:51:59,2018-06-14 14:18:32
-368,Shawn,McCandie,460-327-7535,Male,7,83,25,2018-03-04 04:08:42,2018-02-27 20:23:00,2018-05-03 04:17:09,2018-05-11 02:19:04
-369,Ferdinande,Beckinsall,555-188-2556,Female,1,23,71,2019-01-09 18:09:00,2018-11-05 21:40:44,2018-04-18 11:39:10,2018-05-26 04:58:28
-370,Dex,Hercock,124-933-7494,Male,4,66,59,2018-12-03 18:59:16,2018-08-24 14:54:44,2018-12-04 06:20:22,2018-02-12 20:16:12
-371,Auguste,Kindread,503-140-7531,Female,1,66,99,2018-09-06 02:28:55,2018-06-16 13:03:23,2018-12-18 05:54:52,2018-06-18 08:33:05
-372,Nicolle,McGarvie,977-633-1344,Female,3,80,30,2018-06-27 01:20:37,2019-01-03 07:13:42,2018-08-14 06:50:08,2018-03-11 19:28:21
-373,Meir,Poxson,979-687-3866,Male,4,95,64,2018-07-09 19:27:35,2018-05-02 06:46:01,2018-03-13 19:59:34,2018-02-14 17:37:01
-374,Aurore,Mathe,855-273-0680,Female,1,35,18,2018-12-25 19:52:38,2018-06-27 16:21:30,2018-09-05 07:24:39,2018-07-07 20:33:00
-375,Tara,Kovacs,289-503-0397,Female,1,96,65,2018-02-01 14:13:00,2018-09-13 12:52:31,2018-12-11 12:17:26,2018-01-27 11:08:31
-376,Oliver,Keitch,295-918-1908,Male,2,2,91,2018-04-25 08:41:36,2018-10-26 12:47:17,2018-09-17 07:21:55,2018-05-05 18:27:42
-377,Terrie,Van der Linde,437-439-0685,Female,5,40,86,2018-06-13 19:39:12,2018-09-27 20:24:10,2018-11-29 11:48:24,2018-08-03 10:38:12
-378,Dickie,Rhymes,812-341-6120,Male,1,20,77,2018-02-19 18:11:07,2018-07-04 05:36:40,2018-11-14 00:59:02,2018-11-24 17:13:06
-379,Lynette,Hadland,438-785-0370,Female,1,49,29,2018-04-01 23:24:07,2018-08-21 14:23:03,2018-05-13 18:28:29,2018-12-12 21:59:32
-380,Blanch,Smythin,325-174-1975,Female,1,13,78,2018-11-04 02:34:45,2018-10-22 10:28:57,2018-09-14 10:12:56,2018-10-23 19:59:00
-381,Camellia,Rickeard,681-592-9848,Female,5,53,4,2018-08-25 10:15:49,2018-04-05 06:53:58,2018-02-17 08:17:38,2018-09-11 01:16:02
-382,Muire,Minto,821-124-3681,Female,2,92,71,2018-10-16 14:05:30,2018-05-21 07:22:46,2018-10-26 09:45:10,2018-09-07 09:40:50
-383,Franz,Goullee,650-849-7354,Male,1,19,86,2018-07-14 09:46:05,2018-02-20 08:15:57,2018-06-05 10:37:07,2018-06-14 13:33:38
-384,Ardath,Salaman,542-596-2751,Female,6,3,76,2018-06-07 05:11:01,2018-09-17 09:28:04,2018-04-10 13:00:45,2018-11-27 21:22:09
-385,Graeme,Balshen,286-335-9657,Male,3,44,17,2018-07-18 15:23:20,2018-03-28 07:57:47,2018-11-12 18:58:13,2018-10-11 13:56:49
-386,Lola,Ainsbury,875-341-9340,Female,5,48,71,2018-03-12 03:05:33,2018-11-01 02:55:54,2018-02-15 04:37:51,2018-05-27 04:11:38
-387,Marchelle,Reynoollds,771-436-6596,Female,3,15,98,2018-07-21 06:24:35,2018-10-15 12:19:29,2018-10-21 17:19:01,2018-12-07 00:08:23
-388,Ruddie,Ginger,639-798-1010,Male,6,53,88,2018-10-27 08:25:56,2018-08-05 15:50:59,2018-05-26 08:38:25,2018-03-12 08:36:31
-389,Kathrine,Guiu,822-798-6112,Female,2,31,32,2018-07-21 23:07:18,2018-04-23 18:22:19,2018-12-14 04:50:14,2018-12-17 05:07:54
-390,Bobbie,Peet,321-618-4507,Male,6,40,10,2018-12-07 06:56:32,2018-05-26 07:54:01,2018-02-12 22:14:19,2018-03-02 09:00:13
-391,Waylon,Brignall,721-399-8864,Male,2,29,55,2018-03-05 23:01:40,2018-02-12 20:24:53,2018-10-03 20:50:03,2018-05-27 20:39:09
-392,Randene,M'Chirrie,274-833-8717,Female,7,5,29,2018-11-09 00:26:39,2018-05-08 06:51:45,2018-10-07 04:03:07,2018-07-11 15:11:59
-393,Venus,Burgise,888-311-2349,Female,5,49,10,2018-01-31 18:35:34,2018-06-11 17:52:44,2019-01-18 10:15:45,2018-01-29 11:19:22
-394,Patton,Atkirk,887-990-8528,Male,3,14,48,2018-09-02 04:16:21,2018-12-07 12:14:18,2018-09-11 02:15:29,2018-05-31 08:26:51
-395,Ethelda,Sclater,557-210-5737,Female,2,98,59,2018-10-08 02:45:35,2018-03-07 11:02:12,2018-03-09 22:30:08,2018-07-15 06:00:38
-396,Kerry,Mascall,178-504-3458,Male,3,83,65,2018-08-12 19:58:08,2018-07-06 21:04:42,2018-07-15 00:45:35,2018-11-11 14:30:45
-397,Kelsey,Bohills,477-245-9384,Female,4,23,40,2018-08-31 00:28:52,2018-04-23 15:33:52,2018-08-22 04:21:43,2018-12-27 08:56:06
-398,Brennen,Ahrendsen,666-924-8234,Male,3,47,82,2018-12-10 22:01:21,2018-11-13 15:03:16,2018-01-21 19:47:47,2018-07-28 11:39:10
-399,Carmelina,Mountcastle,871-952-4482,Female,6,14,79,2018-05-25 02:01:07,2018-05-15 19:52:19,2018-07-16 23:16:38,2018-05-22 23:32:19
-400,Dale,Eltone,663-128-1228,Male,1,80,26,2018-10-13 23:51:08,2018-02-19 19:49:01,2018-01-27 11:58:54,2018-07-02 14:05:44
-401,Brianna,Beauchop,922-667-1003,Female,5,12,12,2018-11-05 20:21:33,2018-06-24 22:17:20,2018-06-12 04:34:37,2018-07-09 06:48:22
-402,Guillema,MacKibbon,164-334-7955,Female,3,6,56,2018-06-24 21:01:56,2018-08-15 08:52:58,2018-07-21 21:42:02,2018-02-26 08:29:48
-403,Ettore,Brailsford,231-820-9624,Male,1,92,34,2019-01-17 05:37:42,2018-06-30 12:14:43,2018-04-11 20:55:10,2018-01-21 17:40:14
-404,Shawn,M'cowis,293-256-5912,Male,7,12,87,2018-02-07 20:27:20,2018-08-07 19:48:03,2019-01-01 18:32:27,2018-08-31 20:41:20
-405,Lolly,Rankcom,743-968-5736,Female,2,17,31,2018-11-03 21:37:26,2018-07-18 08:15:01,2018-12-28 23:22:36,2018-09-04 02:20:01
-406,Jeffry,Dingle,194-835-0524,Male,1,6,58,2018-10-21 16:42:07,2018-06-01 21:41:20,2018-06-17 04:34:34,2018-12-29 19:09:29
-407,Marcelle,Molder,766-356-8780,Female,3,85,45,2018-10-18 11:17:27,2018-10-17 20:18:59,2018-09-05 10:30:56,2018-08-10 06:31:09
-408,Binni,Ferney,214-353-4132,Female,6,6,98,2018-02-20 07:18:40,2018-06-24 11:40:51,2018-12-13 02:29:35,2018-10-04 10:52:07
-409,Pat,Whenham,885-713-4885,Male,7,64,5,2018-10-29 12:39:59,2018-11-08 18:24:31,2018-04-17 11:10:35,2018-10-13 22:45:02
-410,Towney,Mizzi,602-556-2702,Male,7,40,94,2018-03-02 22:07:54,2018-08-30 01:11:53,2018-12-21 22:30:09,2018-04-08 21:36:17
-411,Cecil,Bickerdike,577-476-2260,Male,2,2,65,2018-02-25 13:53:37,2018-02-13 04:10:01,2018-05-27 20:30:46,2018-04-11 01:36:56
-412,Franciskus,Thomlinson,539-919-8711,Male,4,69,14,2018-11-12 12:18:44,2018-02-07 14:40:19,2018-08-23 04:04:48,2018-02-11 00:53:06
-413,Valentino,Machent,281-308-6846,Male,6,76,23,2018-10-04 13:04:01,2018-09-13 11:56:13,2018-06-25 21:50:15,2018-06-03 04:08:35
-414,Evelin,Harpham,287-743-1401,Male,7,1,5,2018-06-15 01:25:03,2018-05-17 06:35:54,2018-10-22 08:02:14,2018-08-25 01:58:06
-415,Cassandre,Crum,685-341-7481,Female,3,25,79,2018-07-29 10:09:33,2018-01-30 23:16:22,2019-01-13 15:40:35,2018-07-14 15:57:17
-416,Loren,Leschelle,321-405-9197,Male,3,26,13,2018-04-29 06:52:46,2018-06-21 13:57:41,2018-07-30 18:39:32,2018-09-01 00:31:38
-417,Ilaire,Plose,336-197-4797,Male,1,53,50,2018-07-01 07:52:56,2018-10-30 02:18:43,2018-08-21 23:46:29,2018-08-07 13:03:11
-418,Hannis,Issacof,315-716-3782,Female,2,99,85,2019-01-19 10:40:46,2018-04-26 10:04:56,2018-11-15 02:42:19,2018-08-21 16:50:00
-419,Meggie,Beeke,556-174-4615,Female,1,13,85,2018-03-05 16:12:22,2018-09-19 22:31:27,2018-12-04 04:01:57,2018-04-14 20:14:40
-420,Clemens,Leades,183-447-0091,Male,7,98,12,2018-06-18 06:17:24,2018-04-10 09:59:20,2018-03-20 22:45:02,2018-05-10 17:11:21
-421,Dennis,Chifney,196-548-2180,Male,7,75,98,2018-12-09 00:19:15,2018-06-16 10:29:26,2018-09-12 03:26:46,2018-10-24 16:42:20
-422,Yuri,Laytham,783-967-4260,Male,7,72,60,2018-01-28 20:33:42,2018-11-10 15:59:58,2018-05-07 10:19:43,2018-03-08 01:03:28
-423,Rosaline,Gibbins,376-974-7633,Female,3,22,56,2018-06-26 01:09:00,2018-10-30 23:51:58,2018-07-31 09:43:54,2018-11-23 12:00:53
-424,Yule,Hadwick,795-430-0875,Male,2,99,87,2018-03-09 17:35:22,2018-07-18 20:38:41,2018-08-29 09:29:40,2018-12-12 13:21:09
-425,Albertine,Bernlin,415-360-6554,Female,2,40,34,2018-05-24 10:27:28,2018-10-31 10:36:51,2018-07-04 15:40:42,2018-05-23 02:22:57
-426,Donnell,Stammler,944-379-5489,Male,4,59,41,2018-06-28 16:01:29,2018-05-18 12:55:36,2018-03-07 06:37:32,2018-03-25 12:44:31
-427,Brant,Athy,987-827-3040,Male,5,52,7,2018-01-24 14:17:54,2018-10-06 04:10:01,2018-02-21 04:02:17,2018-05-27 20:44:43
-428,Rose,Barnaby,789-522-4912,Female,7,31,46,2018-11-30 14:09:29,2018-11-06 16:28:33,2018-12-13 23:06:42,2018-09-12 17:39:14
-429,Valina,Ply,729-373-7904,Female,3,65,27,2018-11-15 05:26:26,2018-11-28 15:21:13,2018-05-01 00:56:08,2018-02-05 07:37:50
-430,Berkley,Ferry,484-341-3250,Male,6,47,48,2018-07-26 11:27:11,2018-03-05 19:23:46,2018-08-05 08:09:19,2018-05-30 08:43:28
-431,Jarred,Cunnington,995-170-5820,Male,4,6,1,2018-07-12 11:36:19,2018-10-19 05:16:25,2018-06-28 12:38:16,2018-10-08 06:52:46
-432,Marys,Audry,449-892-8929,Female,2,25,86,2018-11-21 14:36:19,2018-11-26 13:31:27,2018-03-28 15:22:34,2018-01-28 08:07:21
-433,Coreen,Vinnicombe,267-868-6003,Female,6,64,45,2018-10-19 22:09:21,2018-12-08 23:41:39,2018-08-16 05:46:08,2018-11-18 14:46:52
-434,Spense,Sarfati,945-773-9494,Male,4,6,3,2018-03-18 13:44:54,2018-09-04 10:40:58,2019-01-15 18:23:55,2018-06-07 16:02:39
-435,Chandra,Burgin,130-446-4470,Female,4,95,96,2018-09-27 16:48:00,2018-05-20 20:44:56,2018-11-23 04:46:53,2018-01-31 16:31:14
-436,Tomas,Currin,756-279-5643,Male,4,41,4,2018-11-18 07:22:02,2018-11-14 14:03:44,2018-09-18 12:36:39,2018-06-05 09:28:43
-437,Conway,Wesson,754-769-3550,Male,1,52,46,2019-01-02 08:07:04,2018-02-14 00:52:12,2018-02-11 10:52:29,2018-02-16 15:31:25
-438,Stefa,Meese,610-105-6841,Female,7,39,28,2018-04-28 02:21:35,2018-10-21 01:33:19,2018-11-23 21:00:01,2019-01-10 08:36:35
-439,Theadora,Ubee,609-368-9924,Female,6,42,5,2018-04-22 17:02:17,2018-08-21 16:33:25,2018-08-25 04:12:23,2018-02-16 01:13:06
-440,Virginia,Flawn,711-955-7169,Female,3,20,19,2018-08-16 18:01:55,2018-04-18 10:08:19,2018-01-27 09:38:48,2018-05-29 02:29:24
-441,Marv,Mummery,820-192-1452,Male,5,75,95,2018-05-29 15:29:25,2018-03-19 14:00:54,2018-01-22 01:15:38,2018-08-17 08:36:00
-442,Fulton,Zelake,283-584-1765,Male,7,4,3,2019-01-13 23:55:52,2018-08-19 22:29:51,2018-04-16 02:28:54,2018-03-16 16:58:02
-443,Guillemette,Ferretti,776-408-8649,Female,3,24,79,2018-07-31 04:43:48,2018-08-04 15:42:51,2018-04-11 16:00:11,2019-01-07 22:56:20
-444,Egbert,Dureden,698-387-7895,Male,7,54,6,2018-02-25 13:06:41,2018-01-28 14:50:49,2019-01-01 05:38:28,2018-06-19 00:46:05
-445,Trumaine,Canniffe,563-399-5639,Male,1,79,93,2018-11-08 00:30:08,2018-04-07 04:15:13,2018-03-30 14:48:19,2018-04-10 23:29:00
-446,Burr,Goves,451-577-7757,Male,1,27,72,2018-03-16 22:57:50,2018-02-04 21:42:45,2018-06-27 04:37:53,2019-01-16 15:20:56
-447,Juditha,Luebbert,958-685-2937,Female,4,36,87,2018-08-30 23:36:55,2018-06-22 07:04:37,2018-08-23 08:03:08,2018-03-28 04:50:57
-448,Lilly,Smewings,246-392-5467,Female,7,61,65,2018-06-02 21:56:15,2018-09-21 22:08:51,2018-11-29 20:26:34,2018-09-09 13:36:55
-449,Beckie,Curd,239-783-9576,Female,2,18,69,2018-11-20 04:10:54,2018-07-09 08:10:52,2018-03-19 21:04:09,2018-06-19 12:09:53
-450,Cristie,Edmonds,203-875-9153,Female,6,75,68,2018-11-16 10:01:12,2018-08-05 10:10:51,2019-01-11 05:06:49,2018-02-27 23:12:37
-451,Richmound,Machans,259-901-3247,Male,6,7,81,2019-01-12 15:06:01,2018-08-04 16:50:11,2018-12-30 16:16:18,2018-04-24 01:22:55
-452,Murial,Heiss,686-151-1653,Female,6,62,44,2018-05-18 13:49:57,2018-07-31 04:58:55,2018-02-26 22:38:19,2018-06-03 12:20:39
-453,Leonhard,O'Hear,205-401-7116,Male,5,3,1,2018-12-01 20:43:01,2018-06-19 12:45:05,2018-06-17 21:31:22,2018-12-29 20:41:28
-454,Kin,Yakovlev,952-369-5846,Male,1,39,38,2018-03-05 07:22:54,2018-06-20 14:24:43,2018-12-24 17:56:41,2018-07-13 07:55:25
-455,Debee,Fransinelli,230-272-0203,Female,3,64,60,2018-03-28 12:01:56,2018-09-10 13:03:46,2018-05-10 07:50:05,2018-08-10 09:35:31
-456,Nappie,Seaton,809-725-5784,Male,1,67,52,2018-09-13 00:18:51,2018-07-10 10:46:33,2018-04-19 00:56:44,2018-06-17 12:12:31
-457,Adelheid,Wanjek,795-641-9749,Female,1,21,97,2018-03-09 22:00:35,2018-08-09 22:29:44,2018-05-22 14:09:09,2018-03-19 18:49:28
-458,Jeannette,Glanville,377-671-9024,Female,7,84,44,2018-10-05 10:17:32,2018-04-06 12:22:12,2018-05-10 03:19:20,2018-09-30 04:25:01
-459,Adrianna,Blezard,773-598-7742,Female,6,89,62,2018-04-25 21:46:25,2018-05-26 05:44:50,2018-07-14 12:21:26,2018-09-28 10:26:06
-460,Gloria,Keelin,120-823-7944,Female,4,9,68,2018-07-13 21:08:25,2018-03-14 15:55:09,2018-04-02 02:08:40,2019-01-09 11:56:17
-461,Becki,Gatling,845-125-4361,Female,4,1,71,2018-06-24 20:47:16,2018-08-28 04:29:09,2018-05-06 23:04:29,2018-06-01 05:37:42
-462,Gabriellia,Lobley,844-697-7445,Female,2,89,99,2018-09-15 13:57:53,2018-06-14 14:04:36,2018-12-09 22:49:10,2018-02-12 02:07:35
-463,Aaren,McVie,973-286-0455,Female,4,67,14,2018-10-24 23:51:19,2018-04-28 04:48:26,2018-08-01 00:36:09,2018-07-03 13:36:32
-464,Barbara,Luddy,270-772-2682,Female,6,31,58,2018-03-18 11:34:08,2018-11-10 07:14:55,2018-12-12 21:20:10,2018-08-31 00:15:05
-465,Ibrahim,Went,494-197-0863,Male,3,35,42,2018-10-27 09:01:43,2018-05-24 01:03:27,2018-11-12 00:22:15,2018-03-28 09:23:37
-466,Sherry,Meert,142-492-1426,Female,7,30,41,2018-09-05 17:22:28,2019-01-11 02:43:23,2018-08-24 21:28:14,2018-03-23 18:02:50
-467,Marietta,Caulwell,931-150-2069,Male,5,5,93,2018-10-20 06:55:32,2018-08-24 12:59:12,2018-02-05 08:19:35,2018-01-23 22:03:22
-468,Kaiser,Gossage,250-466-8664,Male,6,23,93,2018-08-30 13:38:00,2018-10-27 22:45:24,2018-06-20 08:32:02,2018-01-25 09:32:34
-469,Margie,Fenelon,570-190-5079,Female,7,83,33,2018-12-11 01:27:54,2018-02-18 13:46:59,2018-09-23 21:24:36,2018-09-12 19:05:02
-470,Alvin,Stallan,451-964-7576,Male,6,25,97,2018-10-10 09:53:05,2018-05-12 18:37:24,2018-04-16 18:11:41,2018-12-06 13:37:08
-471,Corbin,Lamport,950-956-7283,Male,2,80,15,2018-10-08 17:14:14,2018-12-03 17:06:55,2018-08-04 22:29:17,2018-12-03 12:04:13
-472,Imelda,Crat,346-373-3643,Female,3,60,21,2018-01-25 18:42:27,2018-02-02 11:29:22,2018-05-12 06:06:53,2018-05-24 14:31:31
-473,Kala,Aikenhead,665-137-0705,Female,3,5,83,2018-03-11 01:11:04,2019-01-18 14:28:39,2018-04-20 06:28:31,2018-06-21 01:18:27
-474,Bastien,Lavell,325-523-0512,Male,7,87,91,2019-01-14 08:13:00,2018-06-08 18:23:51,2018-12-02 08:49:53,2018-12-06 00:24:41
-475,Muriel,Cannon,174-249-0377,Female,7,11,49,2018-06-09 02:06:14,2018-05-19 03:47:34,2018-02-15 07:14:49,2018-03-19 22:10:50
-476,Duane,Presland,616-278-2852,Male,6,49,20,2018-02-10 17:43:55,2018-02-21 00:21:21,2018-10-01 16:17:22,2018-04-06 08:15:06
-477,Emmerich,Baildon,408-829-6965,Male,1,13,29,2018-11-08 20:46:52,2018-07-04 14:45:10,2018-05-22 03:30:35,2018-07-26 23:46:21
-478,El,Seabourne,745-257-4885,Male,6,13,60,2019-01-05 01:40:08,2019-01-07 01:30:58,2018-08-24 00:13:56,2018-08-20 02:39:03
-479,Rivalee,Robertsson,882-741-1264,Female,3,36,70,2018-10-07 07:27:41,2018-05-13 11:32:58,2018-12-07 16:22:21,2018-12-12 19:59:38
-480,Alexis,Lapping,852-596-1033,Female,1,48,44,2018-01-30 22:17:04,2018-08-30 00:34:37,2018-11-04 00:05:38,2018-12-11 02:01:32
-481,Francine,Slatcher,134-351-4974,Female,1,56,91,2018-12-25 00:49:27,2018-10-11 02:42:45,2018-10-09 12:40:08,2018-09-01 13:26:34
-482,Kristen,Petters,440-151-8788,Female,7,19,49,2018-02-17 01:34:14,2018-10-28 03:18:55,2018-09-13 14:14:48,2018-01-22 12:26:05
-483,Riobard,Helin,217-648-4050,Male,6,3,62,2019-01-08 20:13:28,2018-07-10 21:50:12,2018-02-14 10:54:30,2018-07-10 15:21:36
-484,Jere,Marrison,125-522-1752,Male,5,77,48,2018-03-27 12:35:57,2018-08-11 12:06:00,2018-07-10 09:48:33,2018-11-12 18:44:01
-485,Sheila,Edens,296-894-3200,Female,3,99,67,2018-09-02 05:54:01,2018-03-13 04:55:49,2019-01-07 04:42:32,2018-12-12 03:46:51
-486,Georgy,Sallings,860-751-2558,Male,5,42,98,2018-03-04 09:36:14,2018-12-18 03:43:57,2018-05-26 16:22:04,2018-09-02 12:56:41
-487,Marcellina,Ledgister,286-469-6281,Female,6,17,35,2018-07-04 19:56:40,2018-04-13 18:15:53,2018-06-22 16:02:20,2018-02-12 13:24:14
-488,Terra,Dodamead,913-630-9923,Female,1,82,1,2018-12-11 19:54:27,2018-12-11 20:39:38,2018-03-28 15:51:45,2018-08-25 04:08:21
-489,Alia,Morrel,431-923-1395,Female,7,7,38,2018-05-24 06:33:39,2018-03-16 20:23:58,2018-09-11 05:45:02,2018-02-10 15:26:52
-490,Henri,Cullivan,513-315-1874,Male,2,7,65,2018-02-21 12:36:28,2018-02-28 14:01:27,2018-11-04 00:27:37,2018-06-18 18:54:48
-491,Josie,Langrish,674-305-6918,Female,2,79,72,2018-07-26 02:09:25,2018-12-03 14:04:40,2018-10-18 18:24:47,2018-09-01 09:38:00
-492,Brittany,Redwall,197-871-1265,Female,2,98,39,2019-01-15 16:02:18,2018-07-28 14:31:51,2019-01-05 12:31:54,2018-02-17 02:21:53
-493,Phillie,Caulket,276-222-0475,Female,2,92,27,2018-10-19 10:58:15,2018-10-01 05:23:18,2018-10-31 20:41:31,2018-01-23 21:09:13
-494,Clair,Ring,789-696-3129,Female,1,60,89,2018-12-15 04:12:43,2018-03-14 18:18:51,2018-02-01 04:15:54,2018-05-24 23:34:59
-495,Allis,Sunner,601-505-2836,Female,3,37,9,2018-03-16 18:20:34,2018-10-03 23:52:03,2018-09-09 20:19:13,2018-09-12 13:27:03
-496,Madeleine,Haitlie,956-801-4088,Female,4,55,99,2018-05-23 09:24:40,2018-09-25 14:13:54,2018-12-07 06:01:21,2018-08-05 05:36:15
-497,Cirillo,Gronav,571-141-3522,Male,5,81,10,2018-10-11 20:52:40,2018-02-21 03:11:05,2018-09-16 08:35:42,2018-03-26 20:16:43
-498,Arturo,Cowitz,923-926-3273,Male,3,93,12,2018-11-22 05:37:54,2018-02-17 15:11:34,2018-03-31 00:15:46,2018-08-03 14:49:13
-499,Haley,Cartmill,139-891-8155,Female,2,79,56,2018-10-28 03:24:17,2018-09-26 03:42:15,2018-09-16 20:27:43,2018-10-21 06:51:02
-500,Caddric,Eim,386-491-4555,Male,4,88,81,2018-07-27 07:41:11,2018-11-06 13:25:15,2018-04-30 01:01:46,2018-10-27 13:14:40
-501,Sargent,Beart,958-909-4690,Male,1,44,9,2018-07-18 12:02:07,2018-07-23 05:06:07,2018-07-16 15:25:53,2018-08-26 13:12:34
-502,Dane,Gason,502-241-7120,Male,6,79,58,2018-11-17 04:47:55,2018-07-17 16:19:37,2018-12-02 13:39:55,2018-08-07 15:28:47
-503,Humfried,Canelas,275-518-1730,Male,3,82,87,2018-02-02 11:38:29,2018-10-18 04:09:03,2018-02-04 00:47:16,2018-07-23 00:37:30
-504,Sam,Freiberg,757-773-6048,Female,5,89,31,2018-12-01 22:51:12,2018-06-29 03:50:29,2018-07-25 14:13:37,2018-06-11 22:59:09
-505,Goddart,Crossgrove,820-183-6322,Male,3,41,25,2018-11-22 23:25:36,2018-01-29 08:12:28,2018-09-24 21:36:56,2018-03-04 00:40:37
-506,Joela,Bondesen,856-474-9498,Female,5,71,97,2018-05-08 08:23:42,2018-12-20 17:54:27,2018-08-13 11:11:11,2018-08-06 19:06:58
-507,Harrie,Nendick,943-670-9341,Female,3,66,6,2018-05-03 20:18:57,2018-11-06 16:35:09,2018-07-18 17:01:56,2018-03-29 06:23:00
-508,Farlay,Manderson,834-991-0018,Male,4,44,9,2018-08-16 12:51:08,2018-09-19 19:35:31,2018-09-28 07:49:16,2018-11-26 04:15:32
-509,Warren,Pegram,815-768-9711,Male,1,2,70,2018-11-03 16:08:12,2018-09-27 19:07:58,2018-02-15 09:57:42,2018-02-04 05:33:01
-510,Mylo,Haskayne,624-512-7407,Male,3,71,28,2018-11-05 07:00:48,2018-08-22 23:42:06,2018-10-13 09:30:14,2018-12-17 14:14:01
-511,Giacomo,Halsho,976-203-6826,Male,2,59,100,2018-04-11 12:30:03,2018-05-06 11:36:27,2019-01-03 21:44:26,2018-06-13 06:17:36
-512,Aloysia,Stuchburie,466-684-9228,Female,7,15,94,2018-05-05 20:22:16,2018-05-06 16:01:54,2018-11-13 03:57:27,2018-08-05 07:41:19
-513,Brit,Glentz,283-639-7723,Male,4,5,19,2018-04-07 12:25:00,2018-09-26 07:12:01,2018-06-05 14:15:20,2018-06-02 16:59:25
-514,Maddi,Kerfoot,543-438-1531,Female,3,16,67,2018-06-30 07:58:22,2018-12-16 23:14:43,2019-01-03 10:51:10,2018-09-10 17:15:42
-515,Ruy,Townson,801-852-6590,Male,1,31,55,2018-02-22 06:06:31,2019-01-20 20:38:59,2018-09-28 17:23:51,2018-09-30 05:48:30
-516,Rawley,Tatum,634-340-1379,Male,5,32,35,2018-02-14 04:43:24,2018-06-30 10:54:44,2018-11-22 11:23:10,2018-03-01 16:10:00
-517,Teddy,Pickford,404-551-3889,Male,2,82,54,2018-12-29 05:25:46,2018-04-13 05:46:18,2018-10-11 18:14:38,2018-11-07 08:35:59
-518,Randolph,Stealfox,750-802-5940,Male,3,91,11,2018-04-03 20:00:54,2018-11-29 19:15:41,2018-02-20 09:38:24,2018-03-25 15:38:52
-519,Filmer,Sumption,833-713-4339,Male,2,2,8,2018-01-26 04:26:32,2018-03-08 20:23:21,2018-07-27 10:41:05,2018-11-04 20:18:36
-520,Barbee,Salvador,475-228-6268,Female,1,13,71,2018-10-24 00:02:01,2018-01-21 21:29:45,2018-07-03 13:31:36,2019-01-07 14:59:36
-521,Valina,Spillett,588-767-5376,Female,5,27,8,2018-03-03 01:39:51,2018-01-28 01:22:11,2018-04-22 03:30:25,2018-03-08 04:45:22
-522,Taber,Faunt,428-608-8317,Male,5,46,93,2018-06-09 11:17:55,2018-09-01 01:34:32,2018-02-19 13:01:55,2018-08-15 06:00:48
-523,Lilas,Siemons,680-968-7597,Female,6,31,25,2018-11-26 22:08:43,2018-02-15 23:08:21,2018-12-26 05:40:57,2018-02-22 02:08:43
-524,Spence,Vosper,720-164-0499,Male,2,37,73,2018-01-31 08:38:29,2018-07-08 11:20:30,2018-10-31 09:36:46,2018-08-26 13:46:48
-525,Robinett,Le Breton De La Vieuville,690-247-8261,Female,6,47,79,2018-04-02 23:29:51,2018-04-26 14:43:49,2018-09-29 04:02:35,2018-02-22 14:19:00
-526,Pia,Favell,689-639-0553,Female,5,96,86,2018-09-27 09:10:13,2018-02-18 08:18:56,2018-09-16 10:20:42,2018-09-19 19:46:37
-527,Jody,Persich,875-503-4255,Female,5,1,63,2018-12-28 04:07:08,2018-02-19 06:37:40,2018-11-08 19:30:56,2018-04-16 19:36:13
-528,Mimi,Meach,664-906-5485,Female,3,46,59,2018-05-16 02:36:27,2018-02-06 06:52:54,2018-03-16 20:22:46,2018-12-24 10:29:20
-529,Nyssa,Hillborne,493-931-8360,Female,6,70,73,2018-12-08 00:07:34,2018-01-31 21:28:10,2018-04-14 12:33:22,2018-12-18 17:12:16
-530,Chrissie,Jagger,284-506-7117,Male,7,89,53,2018-03-21 21:11:02,2018-09-10 16:42:15,2019-01-17 21:11:31,2018-10-26 07:01:07
-531,Jacqueline,Hebblewhite,313-329-3935,Female,6,70,40,2018-07-29 17:39:44,2018-08-26 14:11:15,2018-12-18 12:39:49,2018-09-04 11:21:47
-532,Gunilla,Frarey,231-521-7913,Female,7,3,44,2018-03-05 06:12:04,2018-06-21 16:09:41,2018-01-27 13:30:18,2018-02-28 16:14:26
-533,Eleni,Fassum,149-906-0502,Female,1,40,27,2018-11-17 06:56:45,2018-06-22 05:04:09,2018-05-07 22:21:42,2018-12-21 21:39:19
-534,Kerr,Kelshaw,729-673-6161,Male,6,94,93,2018-11-23 00:31:03,2018-12-09 03:46:43,2018-02-22 16:54:17,2018-09-09 11:05:13
-535,Lisha,Curgenven,874-739-3351,Female,4,97,18,2018-02-20 18:11:36,2018-10-17 06:38:52,2018-12-26 02:40:15,2018-02-13 09:24:09
-536,Bevin,Jakeman,676-751-7045,Male,1,87,71,2018-02-20 13:08:18,2018-11-21 06:36:13,2018-09-22 09:22:49,2018-10-09 06:21:12
-537,Harland,Sidney,139-828-9659,Male,6,23,85,2018-09-18 02:15:28,2018-04-10 06:49:54,2018-12-05 20:08:51,2018-08-05 09:53:26
-538,Rori,Seys,180-907-6460,Female,5,27,23,2018-02-22 03:55:23,2018-08-01 08:40:28,2018-05-15 12:42:51,2018-10-22 00:24:22
-539,Baxy,Marjanski,580-759-7206,Male,4,95,30,2018-09-01 14:33:22,2018-12-14 23:01:00,2018-12-02 00:20:18,2018-08-25 13:54:47
-540,Martie,Dearsley,102-718-1278,Male,7,89,15,2018-03-15 10:57:25,2018-03-03 03:48:25,2018-07-12 03:48:36,2018-07-22 13:18:47
-541,Erin,Ruddick,969-222-3994,Male,1,97,56,2018-05-18 05:27:05,2018-02-23 12:22:10,2018-04-13 08:34:06,2018-11-25 15:51:09
-542,Tome,Lorne,842-424-5954,Male,4,58,76,2018-12-11 13:20:20,2018-07-29 14:26:59,2018-08-12 08:27:28,2018-04-03 13:46:52
-543,Nell,Borkin,228-448-5895,Female,1,3,1,2018-11-30 16:38:15,2018-07-27 09:33:59,2018-08-23 02:00:10,2018-09-28 00:28:48
-544,Teirtza,Springthorp,538-731-0003,Female,4,8,3,2018-03-28 08:52:40,2018-11-03 18:14:45,2019-01-02 06:05:23,2018-03-22 22:01:00
-545,Arlina,Feighney,977-853-6783,Female,1,18,40,2018-02-08 21:04:46,2018-03-02 23:43:32,2018-03-05 13:33:15,2018-05-12 17:12:38
-546,Melosa,Gifford,804-769-4197,Female,7,55,98,2018-02-04 08:50:49,2018-10-10 06:22:51,2018-06-01 01:45:08,2018-12-24 03:31:15
-547,Charin,Petrello,597-441-0171,Female,6,67,18,2018-09-10 08:11:34,2018-03-15 18:44:45,2018-10-22 23:24:05,2018-07-31 01:37:48
-548,Cristian,Stute,223-959-7392,Male,2,5,69,2018-11-01 17:37:10,2018-05-25 14:21:07,2018-10-09 08:14:38,2018-06-29 13:27:34
-549,Velvet,Morison,848-981-1316,Female,1,20,17,2018-02-14 02:16:51,2018-08-25 23:57:58,2018-04-20 10:47:18,2018-04-04 19:16:45
-550,Vivienne,Taillard,392-626-2926,Female,2,38,47,2019-01-20 11:03:04,2018-06-11 20:42:47,2018-05-13 17:10:39,2018-06-02 20:11:46
-551,Zach,Vials,406-733-0058,Male,1,28,88,2018-12-14 14:36:03,2018-02-25 21:52:58,2018-10-17 02:37:28,2018-02-05 17:26:59
-552,Christie,De Caville,268-836-0470,Male,6,28,91,2018-11-26 14:01:43,2018-06-27 22:05:24,2018-11-10 16:11:15,2018-07-20 12:01:34
-553,Dulce,Walesby,578-217-3628,Female,6,83,40,2018-04-12 08:24:30,2018-10-07 09:32:47,2018-04-10 12:21:16,2018-06-04 11:56:13
-554,Shermy,Martinek,627-924-0370,Male,1,54,86,2018-09-02 00:41:57,2018-09-16 02:42:47,2018-05-10 03:54:12,2018-11-14 17:54:47
-555,Curcio,Cudworth,355-447-1790,Male,2,43,26,2018-05-31 02:31:23,2018-11-18 23:47:19,2018-02-26 10:49:12,2018-08-22 04:36:14
-556,Newton,Zuan,810-756-5844,Male,5,85,20,2018-03-09 16:07:56,2018-10-12 16:45:42,2018-05-02 19:27:29,2018-04-21 12:37:09
-557,Gina,Keyho,566-476-0815,Female,4,8,83,2018-09-21 17:12:16,2018-03-20 10:13:51,2018-05-22 10:29:45,2018-09-23 17:23:36
-558,Emmanuel,Beacham,923-766-7627,Male,4,15,10,2018-10-14 04:16:42,2018-12-12 21:49:16,2018-04-06 14:49:00,2018-02-10 06:29:37
-559,Mel,Deniseau,577-125-8484,Female,3,12,6,2018-06-01 18:43:13,2018-11-09 14:24:38,2018-07-01 23:11:43,2018-11-24 10:28:42
-560,Deena,Knutton,544-886-5072,Female,2,98,57,2018-02-11 05:11:20,2018-08-19 14:06:35,2018-09-06 12:12:59,2018-10-07 13:53:21
-561,Flossy,Antonacci,108-542-7284,Female,2,20,36,2018-05-20 08:28:31,2018-04-01 14:07:20,2018-07-23 02:08:55,2018-08-29 00:40:11
-562,Gabriela,Gravy,565-992-0341,Female,3,54,70,2018-03-14 07:29:51,2018-11-24 05:40:57,2018-06-24 11:53:15,2018-12-13 15:23:43
-563,Emelen,Healey,821-210-6931,Male,7,16,79,2018-03-02 11:53:31,2018-07-04 07:43:45,2018-08-20 22:28:53,2018-08-20 07:13:10
-564,Anne-marie,Peatman,431-777-2495,Female,4,38,1,2018-02-28 18:38:14,2018-08-28 18:58:41,2018-09-01 22:16:42,2018-11-14 23:08:02
-565,Raymond,Truelock,671-694-3305,Male,5,20,76,2018-11-19 18:58:41,2018-10-26 03:22:35,2018-10-13 10:05:37,2018-10-20 00:19:39
-566,Gabriel,Matlock,704-620-9706,Female,1,29,24,2018-07-24 15:56:02,2018-12-23 10:03:48,2018-07-13 22:41:53,2018-06-12 13:42:42
-567,Leontine,Kollasch,793-692-0103,Female,4,68,78,2018-05-05 06:54:44,2018-07-26 19:41:30,2018-04-16 00:46:38,2018-11-20 19:18:17
-568,Amelina,Gullick,674-849-6707,Female,2,75,43,2019-01-01 12:18:11,2018-02-18 10:50:33,2018-09-11 05:30:18,2018-12-28 03:43:14
-569,Matthew,Gateley,450-774-0905,Male,5,93,62,2018-02-17 13:01:16,2018-06-17 01:27:48,2018-12-03 19:17:56,2018-08-31 08:39:46
-570,Alvan,Postance,415-301-3970,Male,5,36,24,2018-02-15 12:09:30,2018-07-26 19:38:06,2018-09-23 03:56:41,2018-09-17 12:13:43
-571,Ethelind,Otteridge,724-398-4832,Female,1,11,20,2019-01-09 07:06:55,2018-08-06 11:24:34,2018-02-04 09:08:33,2018-02-04 10:18:38
-572,Allayne,Pettegre,567-912-1393,Male,5,50,24,2019-01-17 18:14:12,2018-03-20 23:30:29,2018-07-08 17:53:55,2018-03-30 22:34:32
-573,Cindee,MacGill,218-267-2220,Female,3,85,36,2018-03-18 06:24:30,2018-08-12 14:08:59,2018-06-07 00:36:38,2018-07-28 18:35:16
-574,Jeramie,Fante,508-403-4318,Male,6,17,58,2018-09-05 21:09:00,2018-07-16 01:34:59,2018-05-30 20:15:51,2018-10-07 09:23:32
-575,Johannah,Treamayne,670-278-1704,Female,1,44,50,2018-09-01 12:25:07,2018-09-04 03:56:02,2018-03-25 08:47:56,2018-06-27 19:44:57
-576,Dari,Brocklesby,128-823-5176,Female,2,21,47,2018-11-06 21:06:13,2018-11-23 13:57:31,2019-01-01 00:06:22,2018-06-23 04:39:41
-577,Cob,Younglove,589-756-5544,Male,1,34,97,2018-06-06 05:48:43,2019-01-01 02:48:52,2018-09-20 02:52:35,2018-06-29 07:02:35
-578,Genevieve,Benois,718-259-0218,Female,1,83,84,2018-12-29 00:38:58,2019-01-17 14:12:58,2018-08-21 12:35:34,2018-07-15 16:50:51
-579,Demetra,Clineck,537-758-1797,Female,7,58,86,2018-04-18 09:28:08,2018-05-18 03:26:25,2018-02-24 07:51:24,2018-12-11 05:21:46
-580,Edithe,MacNeilage,254-201-0040,Female,5,2,46,2018-12-06 12:01:57,2018-02-16 02:10:30,2018-02-23 14:46:58,2018-12-31 17:43:21
-581,Osbourne,Scranny,682-363-4094,Male,3,29,17,2018-05-10 17:26:21,2018-11-02 03:59:01,2018-04-07 16:21:34,2018-07-31 23:17:47
-582,Con,Shyre,535-646-6697,Male,3,96,16,2018-07-14 00:36:39,2018-12-15 03:20:27,2018-07-23 10:48:18,2018-03-18 15:21:58
-583,Sylvester,Cannicott,456-905-3181,Male,1,2,37,2018-12-19 15:40:57,2018-06-18 08:08:39,2018-05-16 20:50:39,2018-10-02 13:56:35
-584,Any,Hayter,822-858-6900,Male,7,52,26,2018-11-06 13:05:00,2018-12-07 12:10:14,2018-12-30 08:34:38,2018-11-29 15:54:05
-585,Hebert,Casaro,238-642-2667,Male,5,86,80,2018-08-01 13:07:27,2018-11-03 12:14:52,2018-07-26 07:25:57,2019-01-19 03:35:48
-586,Lurlene,Feare,839-313-4455,Female,3,59,51,2018-08-26 04:27:58,2018-10-22 00:35:33,2018-06-26 19:50:51,2018-10-10 07:34:07
-587,Benjy,Mulvenna,623-742-2472,Male,3,63,37,2018-12-05 12:16:12,2018-06-01 06:27:24,2018-05-17 11:06:08,2018-11-03 18:34:32
-588,Odey,Burborough,705-824-5381,Male,7,35,7,2018-11-08 11:17:04,2018-10-02 17:39:13,2018-06-12 17:24:31,2018-11-16 00:36:52
-589,Marin,Piddletown,544-353-3265,Female,6,65,12,2018-06-05 02:00:30,2018-04-15 22:13:49,2018-10-25 08:48:07,2018-02-10 21:14:37
-590,Cyrille,holmes,735-579-6221,Male,7,40,75,2018-04-16 22:04:54,2018-04-25 22:55:07,2018-07-02 18:15:05,2018-08-10 21:27:54
-591,Maurice,Rains,484-638-1467,Male,5,73,91,2018-09-29 20:25:42,2018-05-09 00:14:24,2018-10-31 20:25:12,2018-10-07 19:50:59
-592,Caddric,Styant,859-759-9605,Male,6,10,38,2018-06-20 20:21:02,2018-12-09 17:47:12,2019-01-09 04:06:46,2018-07-19 05:01:01
-593,Vittorio,Straffon,643-403-5013,Male,3,46,7,2018-05-21 05:19:11,2018-12-06 16:33:18,2018-07-03 02:13:34,2018-11-03 12:50:20
-594,Filippo,Adamiak,414-171-7965,Male,5,13,70,2018-03-31 10:48:08,2018-07-07 00:41:10,2018-04-29 13:12:05,2018-08-12 07:32:33
-595,Abran,Blanning,940-243-9117,Male,6,88,72,2018-06-16 02:58:35,2018-08-17 14:00:04,2018-12-23 23:58:31,2018-04-15 17:28:52
-596,Ewart,Van Der Weedenburg,712-866-8225,Male,1,35,99,2018-12-03 12:43:02,2018-08-22 00:51:23,2018-06-03 09:07:10,2018-02-26 12:18:48
-597,Charmion,Toffoletto,139-517-9898,Female,2,60,33,2018-11-24 14:11:52,2018-10-26 03:35:48,2018-12-18 20:13:45,2018-09-23 22:05:26
-598,Marcellus,Casserly,253-327-1311,Male,3,41,81,2018-10-18 22:38:49,2018-03-13 20:31:39,2018-07-25 06:23:51,2018-06-05 20:55:02
-599,Bryant,Woonton,888-556-3681,Male,7,81,99,2018-02-28 09:31:55,2018-05-01 06:19:53,2019-01-09 14:08:26,2018-10-29 15:33:42
-600,Annnora,Beswick,518-587-0261,Female,6,56,22,2018-08-12 21:49:43,2019-01-08 19:34:20,2018-02-14 10:14:40,2018-08-01 03:40:27
-601,Donovan,Stenner,407-562-5695,Male,1,80,76,2018-08-31 07:45:11,2018-12-01 06:25:41,2018-02-28 03:57:11,2018-10-17 16:19:27
-602,Northrup,Limprecht,463-752-6075,Male,5,65,42,2018-11-03 02:27:29,2018-05-05 19:23:48,2018-02-18 06:54:36,2018-08-13 18:47:59
-603,Linoel,Havenhand,896-715-3769,Male,7,7,25,2018-05-10 18:45:39,2018-01-27 18:35:14,2018-06-14 18:17:37,2018-03-26 21:54:40
-604,Gerrilee,Morant,914-846-5569,Female,4,45,66,2018-02-17 20:40:10,2018-01-31 19:47:44,2018-10-09 22:28:47,2018-10-27 15:15:40
-605,Giralda,Daine,952-600-0863,Female,4,28,60,2018-06-15 19:09:55,2018-04-30 04:40:26,2018-09-14 06:48:06,2018-02-10 19:05:33
-606,Joela,Rand,808-932-6875,Female,1,27,3,2018-08-16 15:12:40,2018-09-27 15:12:49,2018-05-27 05:51:56,2018-04-13 14:52:35
-607,Obidiah,Beecham,324-361-0116,Male,7,72,18,2018-10-28 13:32:50,2018-06-20 18:58:47,2018-11-02 11:51:44,2018-03-11 23:29:03
-608,Karlik,Witherington,317-493-1871,Male,4,86,11,2019-01-20 03:58:33,2018-07-21 12:26:45,2018-08-23 20:09:48,2018-11-13 10:55:23
-609,Andras,Buttler,645-159-2747,Male,4,66,45,2018-05-19 10:32:36,2018-12-15 13:10:33,2018-01-31 23:04:41,2019-01-04 22:38:54
-610,Kevan,MacCorkell,902-102-9241,Male,5,35,56,2018-02-09 15:51:27,2018-07-17 04:54:50,2018-01-23 13:03:51,2018-07-12 19:59:05
-611,Dena,Welbelove,824-828-1515,Female,7,30,100,2018-03-13 11:53:11,2018-07-30 19:53:24,2019-01-12 19:12:36,2018-10-23 22:10:25
-612,Susette,Spurman,489-770-6265,Female,5,20,53,2018-05-08 12:17:52,2018-06-18 00:15:28,2018-04-14 14:12:49,2018-09-21 10:34:01
-613,Lisabeth,Clarycott,631-189-3198,Female,7,98,3,2018-08-30 00:26:48,2018-03-16 13:08:16,2018-10-12 07:44:50,2018-10-14 13:12:31
-614,Sheryl,Cicchinelli,508-453-8265,Female,6,77,4,2019-01-04 08:57:07,2018-07-27 00:56:42,2018-07-07 09:59:19,2018-08-03 11:54:06
-615,Milly,Harston,959-484-0888,Female,3,13,88,2018-05-02 06:02:08,2018-05-11 09:34:20,2019-01-11 22:42:26,2018-02-06 01:43:05
-616,Avery,Cant,144-536-3636,Male,5,59,83,2018-04-28 04:40:20,2018-02-20 10:21:26,2018-01-23 14:16:23,2018-06-13 17:25:06
-617,Odelle,Hauxley,311-843-6729,Female,2,15,33,2018-06-09 20:51:56,2018-03-22 15:09:29,2019-01-07 22:47:04,2018-10-30 05:38:26
-618,Symon,Dewi,944-566-4724,Male,6,79,26,2018-10-23 03:09:57,2018-04-20 15:52:10,2018-07-23 05:16:26,2018-09-05 13:15:24
-619,Nora,Ingerson,644-377-6341,Female,7,58,74,2018-12-31 01:35:50,2018-03-08 16:01:54,2018-04-02 13:25:27,2018-08-14 07:54:24
-620,Mariam,Ceschelli,722-520-3029,Female,5,65,57,2018-08-17 21:29:16,2018-11-14 18:13:13,2018-11-22 04:05:56,2018-03-25 21:13:21
-621,Guglielma,Fearey,399-683-0328,Female,2,48,46,2018-12-01 22:21:56,2018-05-05 09:53:28,2018-04-01 03:22:30,2018-10-31 05:13:13
-622,Putnem,Glanton,730-351-4499,Male,3,47,40,2018-05-06 05:22:19,2018-08-03 21:56:51,2018-07-27 03:42:36,2018-09-26 03:10:38
-623,Blancha,Penney,585-209-3640,Female,4,92,1,2018-04-24 13:18:47,2018-06-21 21:42:31,2018-03-11 01:50:30,2018-08-17 20:04:54
-624,Zacherie,Manoelli,211-774-0854,Male,7,41,38,2018-09-29 10:25:00,2018-11-03 11:49:35,2018-06-04 21:41:44,2018-10-24 16:29:50
-625,Seymour,Murphey,840-189-1695,Male,7,27,18,2018-11-22 02:51:55,2018-11-13 09:09:33,2018-04-26 15:24:33,2018-06-10 09:01:11
-626,Kiele,Potte,807-707-7776,Female,5,50,79,2018-12-10 09:07:14,2018-02-09 02:17:04,2018-12-02 09:39:41,2018-06-25 03:58:33
-627,Pascale,Keat,699-216-9743,Male,6,20,78,2018-11-23 01:30:15,2018-04-11 09:12:08,2018-03-14 20:14:59,2018-02-13 23:24:16
-628,Krisha,Sails,162-533-7679,Male,5,91,52,2019-01-17 01:56:16,2018-02-15 14:18:56,2018-03-05 23:19:21,2018-02-24 11:00:50
-629,Justen,McFeat,266-387-8084,Male,4,45,19,2018-10-17 20:09:40,2018-12-29 10:26:10,2018-07-21 12:46:49,2018-02-12 09:50:24
-630,Daloris,Adamson,173-669-4723,Female,4,46,42,2018-02-02 14:26:53,2018-08-15 00:31:20,2018-10-28 08:17:42,2018-04-21 16:28:19
-631,Saleem,Cawood,885-138-1413,Male,6,96,11,2018-09-04 09:51:41,2018-06-12 17:37:20,2018-12-09 22:22:56,2018-11-28 01:06:26
-632,Yettie,Glynn,672-547-0019,Female,3,84,66,2018-11-13 23:39:10,2018-07-14 16:14:12,2018-09-21 13:43:43,2018-09-29 15:57:10
-633,Waite,Smalls,320-394-9058,Male,4,2,55,2018-12-17 16:32:29,2018-11-15 01:45:06,2018-03-17 23:34:19,2018-03-06 15:12:35
-634,Tomkin,Adam,674-394-3276,Male,3,5,48,2018-02-07 03:48:38,2018-07-13 14:23:03,2018-06-02 14:48:09,2018-07-13 06:23:14
-635,Dilly,Strotone,507-532-7349,Male,2,63,15,2018-10-18 04:02:56,2018-08-29 04:49:57,2018-12-04 03:03:45,2018-10-13 05:54:24
-636,Townsend,Eytel,577-285-3662,Male,2,28,15,2018-06-01 12:29:01,2018-09-17 04:48:03,2018-08-12 21:57:59,2018-02-19 20:48:01
-637,Aurel,Taleworth,826-459-6823,Female,4,21,100,2018-08-04 08:00:57,2018-09-17 17:45:32,2018-10-01 19:40:32,2018-09-12 06:23:27
-638,Barnabe,Eldredge,303-934-8908,Male,1,96,66,2018-11-25 12:08:25,2018-09-28 04:15:54,2018-06-11 18:54:23,2018-01-29 01:27:18
-639,Brnaby,Kwietek,319-338-0342,Male,1,99,21,2018-07-11 05:00:49,2018-09-07 18:54:24,2019-01-02 06:20:42,2018-07-12 08:08:28
-640,Thea,Beamont,123-463-1082,Female,4,95,46,2018-03-04 16:30:12,2018-09-20 03:40:11,2018-07-15 14:34:10,2018-08-29 13:50:42
-641,Moyra,MacAne,973-363-3000,Female,3,50,66,2018-06-15 02:19:51,2018-07-18 23:41:04,2018-11-11 08:19:50,2018-07-28 23:40:30
-642,Luce,Hanmer,800-595-2765,Male,4,71,68,2018-12-29 07:24:16,2018-03-30 23:25:56,2018-12-20 17:22:17,2018-07-01 23:50:17
-643,Clement,Gobert,746-653-6801,Male,2,29,39,2018-03-16 20:33:17,2018-12-07 02:34:52,2018-12-12 16:45:55,2018-12-20 02:30:28
-644,Dalenna,Kingsnorth,366-634-1391,Female,5,42,29,2018-04-19 19:41:25,2018-11-24 02:02:50,2018-03-31 05:36:32,2018-07-16 23:22:03
-645,Gigi,MacGilrewy,807-983-4035,Female,2,5,59,2018-11-13 14:20:34,2018-09-08 20:54:02,2018-09-05 14:11:49,2018-06-05 21:12:39
-646,Josi,De Vere,191-357-7492,Female,3,26,63,2018-11-16 05:10:17,2018-05-10 22:27:04,2018-05-04 18:15:10,2018-06-28 04:26:52
-647,Ripley,Pedrozzi,564-644-3142,Male,4,81,64,2018-05-06 05:04:31,2018-09-08 07:15:00,2018-11-12 01:23:19,2018-08-14 01:45:01
-648,Angelina,Henriques,513-742-8132,Female,1,4,93,2018-08-13 00:14:12,2018-03-20 21:24:41,2018-08-07 23:30:14,2018-11-23 00:03:11
-649,Anthia,Dandy,983-456-4556,Female,4,7,96,2018-05-12 17:03:46,2018-12-07 05:19:51,2018-11-20 12:22:28,2018-10-03 06:06:09
-650,Audry,Mourge,237-352-0739,Female,3,87,13,2018-01-25 21:10:18,2018-04-08 08:11:55,2018-01-26 17:58:05,2018-11-22 16:05:01
-651,Konstanze,Skocroft,383-376-4511,Female,5,53,15,2018-07-06 21:47:19,2018-07-21 19:51:01,2019-01-05 19:31:14,2018-12-02 06:51:10
-652,Koo,Arington,553-175-9464,Female,4,51,72,2018-03-12 21:13:34,2018-01-29 22:50:56,2018-07-28 03:59:46,2018-09-08 20:40:26
-653,Mimi,Novakovic,282-495-5519,Female,2,13,7,2019-01-18 17:51:56,2018-03-21 21:03:27,2018-11-26 23:09:59,2018-02-15 16:11:04
-654,Aldus,Aguirrezabal,289-348-2177,Male,2,27,59,2018-11-06 12:55:08,2018-03-03 18:13:31,2018-04-12 18:22:35,2018-08-22 21:43:49
-655,Sullivan,Kelmere,595-312-2528,Male,1,42,45,2018-12-01 16:35:57,2018-10-25 18:48:22,2018-09-07 00:15:27,2018-04-27 11:34:49
-656,Taite,Ruffli,824-778-7852,Male,4,22,46,2018-07-28 22:54:17,2018-10-09 20:45:57,2018-08-03 04:38:39,2018-11-26 15:09:49
-657,Rosene,Missen,922-494-6731,Female,7,91,26,2018-08-17 20:06:33,2018-11-02 02:58:57,2018-03-05 00:03:11,2018-05-31 23:55:38
-658,Ty,Fucher,639-989-4144,Male,3,68,100,2018-07-22 01:36:24,2018-03-08 21:12:38,2018-05-04 21:30:03,2018-09-06 22:32:37
-659,Heddi,Assel,246-263-2031,Female,6,39,92,2018-02-15 08:32:02,2018-02-26 03:11:18,2018-01-27 11:33:06,2018-08-29 02:37:29
-660,Boonie,Djorvic,425-328-7219,Male,7,49,6,2018-02-11 23:21:23,2018-03-07 04:45:35,2019-01-16 02:27:45,2018-10-14 03:27:02
-661,Aurie,Esgate,216-596-1681,Female,4,44,39,2019-01-07 13:40:58,2018-09-29 18:40:04,2018-04-15 01:33:28,2018-11-16 00:59:28
-662,Dana,Edginton,762-855-7796,Male,7,11,7,2018-10-22 11:13:58,2018-02-25 07:53:13,2018-10-26 14:30:34,2018-02-21 08:22:28
-663,Christiana,Haslock(e),322-905-9384,Female,7,66,50,2018-04-08 07:38:08,2018-05-11 05:11:14,2018-03-13 07:31:38,2018-03-24 02:07:40
-664,Lelia,Kimmerling,462-373-6886,Female,2,22,47,2018-07-16 09:28:30,2018-01-31 12:00:37,2018-05-14 00:22:05,2018-04-02 06:04:19
-665,Kaleena,Ferrier,462-635-6827,Female,2,74,81,2018-06-05 11:09:52,2018-02-12 09:02:48,2018-09-11 19:00:25,2018-02-20 13:36:01
-666,Davon,Kroch,646-511-8893,Male,2,79,55,2018-08-11 12:22:33,2018-03-09 12:30:22,2018-10-04 12:31:33,2018-08-30 17:50:16
-667,Agathe,Kingwell,879-547-0641,Female,7,16,34,2018-06-12 18:22:16,2018-08-05 00:55:58,2018-12-27 12:31:25,2018-02-19 01:39:48
-668,Tami,Brum,925-535-6949,Female,6,13,16,2018-08-10 21:16:02,2018-07-10 10:42:55,2018-10-03 09:45:52,2018-09-02 08:04:13
-669,Meghan,Behan,666-973-3081,Female,2,37,72,2018-07-01 22:33:57,2018-12-23 00:46:46,2018-11-15 22:39:02,2018-02-15 12:20:24
-670,Elliott,Gumbley,891-324-8441,Male,3,100,20,2018-05-26 06:24:39,2018-08-24 00:11:20,2018-10-25 08:36:44,2018-03-14 08:39:54
-671,Roosevelt,Headan,845-354-1730,Male,5,3,90,2018-07-01 14:50:49,2018-09-15 12:12:40,2018-12-24 01:35:47,2018-09-20 01:17:08
-672,Judie,Cozins,719-163-5308,Female,6,64,26,2018-02-13 12:22:03,2018-11-17 08:43:37,2018-04-01 06:23:24,2018-09-21 04:32:48
-673,Zebulen,Bulger,201-223-1332,Male,5,98,53,2018-07-16 23:18:20,2018-05-29 12:17:27,2018-04-28 14:52:50,2018-06-12 16:22:23
-674,Lea,Fairpo,372-782-5016,Female,6,19,3,2018-09-03 14:58:22,2018-07-08 18:29:06,2018-05-16 18:23:32,2018-12-29 08:04:07
-675,Mayer,Demead,393-580-8088,Male,4,41,38,2018-09-26 07:35:23,2018-04-11 15:45:52,2018-03-27 15:23:59,2018-06-06 14:21:04
-676,Ana,Cannaway,492-510-5265,Female,7,61,68,2018-02-10 08:21:49,2019-01-17 12:57:29,2018-03-10 19:54:59,2018-05-05 03:55:19
-677,Cathrine,Featonby,618-545-3336,Female,2,13,20,2018-04-06 05:43:01,2018-08-08 21:16:04,2018-09-14 02:33:02,2018-02-21 21:27:00
-678,Wilt,Pitcher,482-827-8133,Male,3,100,4,2018-07-28 21:19:49,2018-05-26 09:01:20,2018-08-20 04:38:04,2018-12-20 10:33:58
-679,Mirabelle,Beric,266-468-3958,Female,5,22,72,2018-05-02 12:06:49,2018-04-08 13:34:02,2018-05-08 04:55:38,2018-04-08 07:43:46
-680,Kitti,Duell,600-603-0212,Female,6,63,11,2018-06-03 10:32:14,2018-02-28 16:15:11,2018-07-14 16:05:36,2018-12-09 20:55:23
-681,Pebrook,Tunuy,901-401-5075,Male,7,43,54,2018-05-03 21:20:23,2018-08-22 11:35:56,2019-01-16 05:29:13,2018-06-03 07:29:48
-682,Maryanna,Eayrs,722-426-6216,Female,1,44,84,2018-09-20 04:29:10,2018-03-25 00:25:02,2018-08-24 07:28:58,2019-01-03 12:51:01
-683,Obadias,Izzatt,379-850-7154,Male,6,6,27,2018-04-26 21:45:09,2018-02-05 01:27:51,2018-12-26 01:14:44,2018-08-29 23:20:18
-684,Cherye,Androletti,524-563-8173,Female,3,68,10,2018-04-21 19:56:00,2018-09-14 22:29:43,2018-12-29 20:14:49,2018-11-16 03:21:36
-685,Joachim,Dimitrescu,795-788-5797,Male,5,76,2,2018-12-11 21:33:57,2018-12-23 19:18:05,2018-11-25 10:00:13,2018-03-10 21:29:53
-686,Wynne,Heyward,438-352-0453,Female,2,24,17,2018-08-16 11:16:51,2018-07-01 06:49:35,2018-10-07 18:27:47,2018-12-09 16:04:47
-687,Renell,Ipwell,436-940-9778,Female,2,20,24,2018-11-20 12:06:05,2018-07-02 02:09:18,2018-09-08 02:49:34,2018-02-07 07:39:15
-688,Ricky,Alderson,299-400-6861,Male,6,41,97,2018-11-21 08:08:46,2018-08-15 09:54:31,2018-12-12 09:05:40,2018-11-13 16:44:40
-689,Rubi,Gail,300-475-1452,Female,4,37,26,2019-01-06 23:27:48,2018-11-12 10:44:21,2018-12-06 15:12:08,2018-03-19 12:15:14
-690,Sherill,Tallent,643-731-0806,Female,5,51,98,2018-10-30 15:13:26,2018-06-09 19:32:29,2018-02-06 16:40:35,2018-09-26 13:31:25
-691,Elia,Hairon,537-738-0445,Male,4,59,64,2018-06-18 11:14:46,2018-08-12 09:09:40,2018-03-22 09:13:44,2018-10-25 02:49:49
-692,Johny,Degoe,380-482-6021,Male,2,89,51,2018-12-18 10:28:50,2018-10-04 06:40:46,2018-05-17 13:11:00,2019-01-18 01:02:48
-693,Kirstin,Hopfer,520-920-5762,Female,7,7,29,2018-03-03 02:44:21,2018-07-08 01:31:26,2018-08-12 04:22:03,2018-10-07 12:47:55
-694,Addie,Hrycek,411-674-4016,Female,1,62,3,2018-08-02 04:43:18,2018-09-19 05:54:13,2018-09-30 17:33:04,2018-10-24 16:39:15
-695,Devinne,Dilrew,876-178-5135,Female,6,97,69,2018-01-22 00:39:19,2018-05-29 02:38:21,2018-10-17 03:17:47,2018-07-28 12:51:30
-696,Violet,Petren,352-136-0037,Female,3,20,80,2018-09-23 03:09:33,2018-05-22 01:24:08,2019-01-06 00:46:41,2018-04-13 04:27:36
-697,Inga,Cahalan,847-549-2394,Female,3,98,26,2018-03-24 12:37:49,2018-10-09 07:41:56,2019-01-07 22:18:43,2018-01-28 07:44:56
-698,Burg,Liepina,736-158-9051,Male,4,89,88,2018-02-04 05:58:15,2018-02-24 08:51:40,2018-12-01 18:20:15,2018-02-20 12:27:18
-699,Winnifred,Yakutin,165-178-2008,Female,1,74,93,2018-12-14 06:11:18,2018-12-23 02:09:43,2018-04-07 06:12:17,2018-08-16 08:12:28
-700,Boone,Walsom,432-415-6784,Male,3,91,63,2018-06-12 09:05:08,2018-09-21 04:24:21,2018-09-18 05:27:12,2018-09-10 10:46:37
-701,Alberto,Steadman,316-988-9996,Male,7,11,32,2019-01-13 23:25:22,2018-04-27 23:46:24,2018-12-07 20:28:14,2018-02-03 17:46:13
-702,Babbie,Ossulton,151-959-6676,Female,5,50,26,2018-08-25 20:54:00,2018-05-14 14:24:06,2019-01-14 14:14:51,2018-08-15 11:39:35
-703,Franni,Sparwell,411-855-1156,Female,6,95,65,2018-07-21 15:32:34,2018-07-03 10:42:20,2018-08-26 02:44:03,2018-04-13 05:52:48
-704,Carlos,MacNair,911-726-0315,Male,1,17,35,2018-12-05 14:25:42,2018-12-12 19:39:25,2018-04-29 02:34:46,2018-10-22 16:34:03
-705,Violetta,Crosbie,334-280-9073,Female,3,9,44,2018-01-30 22:36:27,2018-08-07 22:12:15,2018-11-27 09:47:02,2018-07-24 14:22:36
-706,Malanie,Perico,380-744-7181,Female,6,68,23,2019-01-03 09:08:46,2018-08-26 01:51:30,2018-12-02 23:44:58,2018-02-16 00:13:42
-707,Ivor,Bowcher,940-535-1313,Male,2,79,20,2018-09-05 08:48:25,2018-05-21 21:40:04,2018-05-02 19:33:24,2018-06-22 07:46:47
-708,Hyacinth,Trump,241-552-4852,Female,4,46,7,2018-02-27 17:57:22,2018-11-20 03:54:39,2018-10-27 00:44:50,2018-09-19 14:04:29
-709,Vittorio,Blincko,136-124-8825,Male,5,70,20,2018-12-17 01:45:05,2018-06-06 18:10:29,2018-12-01 18:11:21,2018-04-14 00:43:46
-710,Tadeas,Glendenning,676-274-2212,Male,3,70,6,2018-08-22 07:42:12,2018-07-12 17:41:15,2018-07-24 17:42:26,2018-12-27 05:34:05
-711,Sky,Flew,681-787-4952,Male,3,81,48,2018-11-13 09:16:39,2018-09-14 09:34:51,2018-09-23 01:44:43,2018-11-21 00:50:58
-712,Luise,Olivetti,707-121-6217,Female,4,68,4,2018-03-29 01:36:21,2018-05-06 13:38:59,2018-05-06 09:55:05,2018-02-04 21:27:46
-713,Drusy,Capinetti,432-896-1449,Female,5,94,80,2018-07-31 20:38:50,2018-11-28 22:41:39,2018-04-27 00:57:40,2018-08-30 16:16:05
-714,Miranda,Rissen,700-512-2672,Female,7,29,40,2018-03-15 22:30:44,2018-02-01 02:59:19,2018-11-25 08:02:09,2018-11-05 07:51:59
-715,Ario,Bashford,390-128-4254,Male,2,54,93,2018-06-14 08:22:43,2018-08-12 18:13:42,2018-11-28 21:40:32,2018-12-29 03:09:26
-716,Rochella,Shackesby,103-592-3468,Female,7,41,14,2018-02-12 09:44:28,2018-08-22 10:35:46,2019-01-09 02:25:18,2018-10-10 17:50:44
-717,Wes,Christofides,192-457-7659,Male,1,69,89,2018-05-29 07:31:48,2019-01-06 03:40:33,2018-02-04 22:20:47,2019-01-05 16:01:23
-718,Griselda,Lukes,232-886-4996,Female,2,96,24,2019-01-10 12:54:29,2019-01-16 20:13:57,2018-04-26 08:07:43,2018-10-11 15:19:29
-719,Dredi,Cooksey,432-670-0783,Female,2,24,32,2018-02-28 00:00:07,2018-06-20 03:18:14,2018-07-21 09:03:11,2018-07-06 03:35:06
-720,Coleman,Trudgian,273-535-1414,Male,4,97,47,2018-04-25 23:41:09,2018-03-11 00:03:13,2018-06-03 13:44:14,2018-08-08 09:07:58
-721,Averill,Shilstone,947-108-0865,Male,7,1,64,2018-08-18 22:21:07,2018-04-12 03:24:47,2018-03-10 08:58:27,2018-08-03 14:04:03
-722,Carmen,Syder,954-822-0919,Female,2,47,75,2018-10-09 09:52:34,2018-11-21 01:16:40,2018-02-12 03:27:35,2018-08-12 12:44:55
-723,Morgan,Leband,547-933-1630,Female,5,39,97,2018-03-25 07:19:57,2018-03-28 20:22:12,2018-09-05 13:29:46,2018-10-12 09:04:17
-724,Milzie,Fearnley,589-230-2628,Female,1,44,13,2018-05-17 23:12:31,2018-07-29 21:43:28,2018-07-06 23:27:59,2018-09-14 01:45:16
-725,Deb,Marjot,495-276-5931,Female,2,27,81,2018-09-28 15:27:35,2018-02-05 23:31:12,2018-07-29 04:18:34,2018-03-08 09:45:23
-726,Shayne,Artinstall,628-997-5039,Male,2,23,20,2018-11-19 12:44:15,2018-05-30 23:05:12,2018-11-23 10:18:23,2018-06-08 14:18:23
-727,Annabal,Standbridge,226-314-1568,Female,5,16,5,2018-08-11 00:40:31,2018-12-20 14:06:36,2018-04-15 04:33:53,2018-11-09 16:42:45
-728,Sheffield,Crielly,360-797-0933,Male,7,57,90,2018-03-08 03:32:17,2018-11-07 06:12:46,2019-01-12 22:10:52,2019-01-02 03:16:09
-729,Antonius,Delagua,336-343-1380,Male,2,64,60,2018-10-14 22:56:26,2018-11-13 00:50:03,2018-07-02 11:23:06,2018-08-09 09:29:32
-730,Brana,Cannell,157-563-3935,Female,2,74,73,2019-01-19 15:41:16,2018-04-20 09:34:06,2018-08-02 12:28:32,2018-02-21 06:29:31
-731,Bartram,Praundlin,631-372-3479,Male,2,35,7,2018-10-03 02:00:10,2018-07-31 21:29:22,2019-01-10 21:43:54,2018-08-28 13:29:28
-732,Anthiathia,Goodwins,664-616-3575,Female,7,76,19,2018-04-26 19:57:15,2018-10-28 17:07:05,2018-02-21 16:38:28,2018-04-18 17:58:31
-733,Deborah,Ekell,577-861-9112,Female,5,74,43,2018-10-19 06:10:04,2018-03-12 03:47:48,2018-05-17 13:04:37,2018-02-09 05:48:05
-734,Lauri,Arunowicz,862-427-5257,Female,4,58,81,2018-07-29 04:16:13,2018-04-23 19:30:17,2018-03-19 21:30:18,2018-11-18 20:46:33
-735,Fowler,Markus,496-831-3082,Male,7,18,52,2018-01-26 22:36:43,2019-01-14 00:15:25,2018-03-18 05:45:56,2018-02-27 16:55:19
-736,Klara,Ramme,103-225-9086,Female,7,28,67,2018-05-21 17:32:08,2018-10-05 08:11:16,2018-06-25 15:05:21,2018-03-13 14:44:42
-737,Melisenda,Chitham,158-699-0591,Female,2,49,88,2018-04-16 02:24:00,2018-10-27 14:52:15,2018-12-31 05:39:31,2018-12-09 09:15:25
-738,Hale,Layborn,106-961-8551,Male,3,66,90,2018-04-22 19:39:27,2018-08-12 19:37:50,2018-08-17 10:37:06,2018-09-14 15:17:48
-739,Ferdie,Stubbeley,686-279-8217,Male,7,62,96,2018-09-25 03:29:05,2018-06-27 23:31:12,2018-07-21 18:49:24,2018-06-19 05:08:00
-740,Licha,O' Kelleher,463-395-9023,Female,6,75,16,2018-07-05 08:29:21,2018-09-13 21:32:41,2018-08-26 17:50:16,2018-05-22 11:30:48
-741,Gualterio,Kernar,123-900-7702,Male,2,32,18,2018-12-30 04:11:35,2018-04-15 10:32:33,2018-06-21 01:01:16,2018-11-15 03:34:48
-742,Harley,Lyokhin,664-282-7716,Male,1,42,45,2018-07-26 12:07:21,2018-02-09 02:23:37,2018-05-10 13:39:44,2018-11-25 04:15:16
-743,Kylila,Hukin,726-438-0889,Female,2,2,74,2019-01-02 09:08:44,2018-04-07 07:18:19,2018-04-19 16:12:40,2018-03-03 19:00:47
-744,Natividad,Dahlbom,744-480-5437,Female,5,82,39,2018-12-30 01:10:05,2018-08-01 07:16:39,2018-05-29 02:44:28,2018-11-08 15:02:54
-745,Virgina,Couzens,947-544-5389,Female,6,14,38,2018-11-19 23:03:21,2018-11-21 13:32:21,2018-03-29 02:30:25,2018-12-31 17:15:49
-746,Peyter,Diano,292-294-3875,Male,2,23,76,2018-12-22 18:12:13,2018-09-23 11:29:07,2018-07-12 06:09:06,2018-07-28 01:53:40
-747,Jayme,O'Curneen,225-447-3103,Male,3,10,84,2018-06-20 10:29:48,2018-08-12 10:14:12,2018-04-28 06:15:40,2018-07-19 08:52:24
-748,Frederik,Wilkisson,963-312-1892,Male,3,4,18,2018-09-11 01:05:52,2018-05-17 07:03:54,2018-09-07 08:19:47,2018-12-27 05:51:37
-749,Yankee,Harrower,918-854-5590,Male,7,14,88,2019-01-12 12:01:21,2018-08-24 08:15:58,2018-06-19 00:06:07,2018-06-19 07:16:40
-750,Tamas,MacRinn,856-289-3684,Male,6,83,91,2018-05-18 15:58:10,2018-06-09 14:45:50,2018-06-16 17:21:20,2018-12-18 02:23:35
-751,Valentin,Rawlings,663-758-9646,Male,4,16,34,2018-06-27 02:03:36,2018-12-10 04:32:16,2018-07-07 05:22:18,2018-02-03 06:54:36
-752,Colver,Yarr,382-717-8711,Male,5,23,10,2018-05-13 09:06:22,2018-04-06 13:41:41,2018-05-07 22:03:59,2018-11-12 22:30:08
-753,Alisa,Gaynesford,417-789-3077,Female,2,87,12,2018-07-16 18:03:56,2018-02-06 07:27:34,2018-07-12 08:51:47,2018-02-22 19:25:29
-754,Xavier,Cornwell,777-401-4183,Male,7,12,13,2018-12-03 01:19:40,2018-05-30 23:16:32,2018-10-01 21:33:13,2018-04-13 23:28:08
-755,Hy,Raftery,563-495-4017,Male,7,54,93,2018-03-25 23:10:24,2018-11-20 07:01:06,2018-03-27 13:09:16,2018-08-03 19:07:10
-756,Holt,Rosenfeld,962-871-1674,Male,2,43,35,2018-08-07 04:15:12,2018-05-21 13:42:27,2018-11-13 19:28:52,2018-03-27 10:14:19
-757,Zola,Andreopolos,543-481-8409,Female,4,31,62,2018-02-08 17:26:50,2018-12-12 15:18:24,2018-02-22 22:58:13,2019-01-02 05:55:39
-758,Letty,Scotting,546-656-5808,Female,5,15,94,2018-10-21 18:22:43,2018-05-26 22:09:39,2018-08-04 15:48:15,2018-08-28 06:14:32
-759,Celina,Lauchlan,202-835-6732,Female,7,47,9,2018-02-26 21:58:16,2019-01-10 01:22:46,2018-11-03 06:10:35,2018-04-21 11:00:49
-760,Drusy,Izakov,129-552-5886,Female,1,45,2,2018-10-11 13:12:08,2018-06-26 11:16:28,2018-08-24 23:58:00,2018-09-09 05:46:23
-761,Shana,Ference,915-610-3990,Female,3,1,7,2018-10-31 18:47:12,2018-09-13 14:33:49,2018-10-02 08:57:46,2018-02-18 19:30:28
-762,Colin,Broy,868-631-8529,Male,2,22,32,2018-05-23 19:06:02,2018-09-08 07:49:35,2018-12-13 21:44:07,2018-06-25 07:15:53
-763,Farlay,Cromett,640-335-8081,Male,7,34,25,2018-06-29 00:34:53,2018-12-08 12:49:01,2018-12-03 13:22:54,2018-06-24 16:33:04
-764,Leandra,Hamberstone,786-332-2588,Female,2,37,90,2018-12-22 15:35:04,2018-02-04 16:23:41,2018-01-28 07:51:51,2018-06-01 00:25:34
-765,Gan,McCahey,450-198-7472,Male,2,79,7,2018-06-26 07:22:06,2018-09-27 09:42:20,2018-03-13 20:28:52,2018-02-10 02:05:39
-766,Brose,Gane,302-470-1094,Male,7,27,25,2018-11-05 10:16:11,2018-12-27 12:00:35,2018-12-13 06:00:05,2018-08-30 01:03:46
-767,Mabel,Nurdin,417-183-7374,Female,1,73,62,2018-12-09 01:11:47,2018-06-09 12:03:13,2018-11-30 04:52:49,2018-04-24 11:08:53
-768,Melinda,Belliard,277-126-4396,Female,6,39,83,2018-07-13 11:17:51,2018-08-04 10:46:43,2018-11-19 23:27:57,2018-02-19 10:08:24
-769,Percy,Yakolev,433-703-6347,Male,4,78,23,2018-11-02 23:32:50,2018-12-19 16:21:10,2018-02-17 21:43:15,2018-12-09 00:57:58
-770,Portia,Legonidec,761-180-7070,Female,6,27,1,2018-11-01 18:06:54,2018-05-16 10:56:05,2018-08-06 20:45:16,2018-08-14 23:11:03
-771,Harry,Lapley,950-309-5667,Male,5,8,58,2018-03-01 15:09:46,2018-05-13 05:24:49,2018-03-22 16:01:05,2018-08-18 02:05:37
-772,Cort,Gathwaite,964-902-3053,Male,5,33,89,2018-08-18 20:12:14,2018-01-27 16:39:37,2018-12-26 22:46:04,2018-02-16 22:31:58
-773,Moselle,Davidsson,813-238-6494,Female,4,13,60,2019-01-10 17:47:31,2018-11-04 08:31:17,2018-04-30 12:09:52,2018-04-14 19:47:23
-774,Avril,Britton,340-286-1678,Female,3,4,16,2018-03-18 00:19:38,2018-12-29 06:21:49,2019-01-03 02:03:40,2018-11-29 12:05:59
-775,Linn,Confait,176-807-2635,Female,6,18,70,2018-09-18 15:11:21,2018-05-02 18:32:49,2018-03-25 13:46:21,2018-10-06 04:29:26
-776,Joletta,Fyldes,547-358-7132,Female,3,92,6,2019-01-07 18:53:22,2018-09-07 21:09:08,2018-10-22 08:38:19,2018-07-08 07:01:37
-777,Alvira,Donnersberg,712-756-5926,Female,1,20,77,2018-06-09 12:41:30,2018-12-29 01:15:22,2018-02-11 19:44:50,2018-02-21 21:33:12
-778,Kirbee,Sherer,434-167-9643,Female,4,84,40,2018-10-18 16:54:05,2018-04-02 11:16:17,2018-02-26 17:45:53,2018-10-21 05:24:45
-779,Griffy,Guitton,326-132-8573,Male,4,60,19,2018-06-22 17:59:44,2018-09-11 16:22:32,2018-11-10 19:43:30,2018-02-06 10:06:41
-780,Emmott,Allitt,969-852-2661,Male,2,48,37,2018-04-24 21:17:55,2018-11-01 06:07:42,2018-07-30 22:56:18,2018-11-13 22:12:50
-781,Tailor,Dalman,521-811-9925,Male,5,60,72,2018-12-14 09:37:55,2018-12-26 17:43:11,2018-12-10 03:38:52,2018-04-07 11:21:07
-782,Brennen,Vasilechko,176-109-0551,Male,6,90,55,2018-08-31 12:29:00,2018-05-23 03:06:41,2018-10-10 17:44:50,2018-10-21 00:47:28
-783,Dotty,MacCallam,161-881-1371,Female,1,56,3,2018-03-01 02:18:19,2019-01-06 18:00:59,2018-04-28 10:26:11,2018-02-05 06:43:37
-784,Jedidiah,Guerin,645-152-7671,Male,5,56,83,2018-04-02 15:07:20,2018-08-24 12:42:12,2018-12-21 09:17:44,2018-10-06 00:19:03
-785,Hannie,Danes,716-637-6821,Female,5,40,18,2019-01-08 07:08:18,2019-01-05 20:40:19,2019-01-16 12:16:28,2018-03-23 12:28:41
-786,Galen,Scogin,453-714-4492,Male,6,38,18,2018-10-27 06:54:20,2018-06-19 09:26:07,2019-01-10 20:55:20,2018-10-18 17:04:44
-787,Aleta,Melonby,207-524-3356,Female,3,98,3,2018-04-04 12:13:59,2018-06-06 11:50:34,2018-05-05 17:38:56,2018-03-11 12:56:42
-788,Adlai,Mauchlen,601-977-7616,Male,7,88,64,2018-03-24 04:33:36,2018-03-07 16:46:25,2018-03-16 19:25:21,2018-07-20 14:23:29
-789,Amelie,Le Barre,576-950-1963,Female,5,28,53,2018-08-10 15:52:25,2018-05-07 07:19:32,2018-12-21 21:24:49,2018-07-21 01:11:54
-790,Rodrique,Lestor,723-918-4771,Male,4,26,72,2018-08-29 10:37:42,2018-11-16 09:49:01,2018-08-31 05:08:59,2018-12-18 10:17:32
-791,Gifford,Challoner,411-352-9364,Male,5,63,18,2018-10-16 12:33:24,2018-11-24 23:28:34,2018-05-29 22:33:36,2018-11-06 07:45:09
-792,Dorene,Pogosian,391-363-6103,Female,4,74,78,2018-04-26 20:52:42,2018-08-02 21:57:23,2018-04-29 21:18:02,2018-02-13 20:06:09
-793,Lynnell,De Filippi,418-713-4087,Female,7,78,47,2018-12-03 11:11:47,2019-01-05 12:23:37,2018-05-05 19:10:42,2018-04-22 16:11:38
-794,Tod,Willicott,354-257-4541,Male,5,34,18,2018-07-19 20:11:25,2018-08-12 01:02:55,2018-08-05 23:52:34,2018-02-15 00:20:31
-795,Myrlene,Wornham,331-165-6672,Female,7,51,75,2018-11-24 15:52:43,2018-07-01 06:12:47,2018-08-08 19:45:02,2018-11-19 14:51:56
-796,Sibyl,Burnsyde,981-826-0517,Female,2,24,2,2018-11-01 05:15:20,2018-10-15 22:30:33,2018-05-27 21:38:42,2018-05-08 15:34:52
-797,Barron,Hanrott,700-573-3103,Male,6,81,99,2018-08-12 21:14:38,2018-03-09 17:36:14,2018-06-03 16:23:47,2018-10-13 22:34:58
-798,Marylee,Hunnawill,501-536-2494,Female,7,30,33,2018-10-19 01:54:52,2018-04-12 03:02:28,2018-09-12 02:31:37,2018-06-14 10:58:08
-799,Chip,Hunnaball,249-377-3740,Male,3,98,59,2018-04-25 10:02:32,2019-01-18 16:00:40,2018-02-14 02:06:39,2018-06-17 23:32:24
-800,Major,Milsap,444-230-6774,Male,7,20,23,2018-03-11 15:00:30,2018-01-24 21:48:22,2018-12-27 13:31:21,2018-08-10 23:29:32
-801,Ashli,Romain,886-453-6140,Female,6,12,19,2018-02-24 23:38:53,2018-05-01 04:29:06,2018-06-03 17:41:53,2018-04-21 00:29:25
-802,Keeley,Frankom,894-599-8867,Female,3,30,4,2019-01-05 18:11:47,2018-03-25 21:31:27,2018-05-26 03:52:58,2018-10-23 16:17:28
-803,Curcio,Widmoor,934-987-4248,Male,5,23,84,2018-03-31 07:42:11,2018-10-25 03:00:08,2018-10-18 23:14:42,2018-05-07 19:58:59
-804,Crawford,Ballston,771-623-5016,Male,5,24,48,2018-12-27 07:13:13,2019-01-09 10:21:22,2018-05-20 08:49:26,2018-09-16 08:46:49
-805,Templeton,Snibson,650-275-2922,Male,3,55,83,2018-12-04 05:22:30,2018-09-29 21:23:56,2018-03-08 17:58:24,2018-08-15 02:38:52
-806,Gail,Peachment,906-123-1774,Male,4,59,13,2018-09-21 07:44:03,2018-08-02 22:34:51,2018-03-08 07:11:39,2018-08-25 15:39:59
-807,Adriano,Escale,888-817-1180,Male,3,92,78,2018-07-24 16:32:20,2018-02-22 12:50:50,2018-06-26 12:34:44,2018-06-06 08:04:35
-808,Trista,Tudgay,340-171-5505,Female,5,44,62,2018-09-04 12:09:41,2018-01-31 01:51:09,2018-09-13 08:38:11,2018-06-06 05:49:50
-809,Angie,Carradice,507-239-1908,Female,1,12,69,2018-08-13 18:36:13,2018-11-23 03:57:42,2018-06-17 07:44:13,2018-12-09 16:37:23
-810,Maureen,Perllman,116-299-8383,Female,1,91,1,2018-04-10 06:12:02,2018-08-02 04:31:50,2018-06-18 00:58:20,2018-10-12 02:31:31
-811,Renaud,Roylance,267-492-3743,Male,3,36,85,2018-02-05 00:59:50,2018-11-26 02:06:43,2018-11-16 00:25:18,2018-02-02 08:38:37
-812,Dasi,Rhucroft,505-753-6245,Female,4,96,84,2018-11-10 18:36:26,2018-07-17 05:54:10,2018-07-12 13:34:58,2018-11-10 01:57:49
-813,Arlee,Gerdts,220-186-9711,Female,6,76,30,2018-12-26 11:24:38,2018-11-21 22:09:49,2018-04-16 14:09:09,2018-05-27 06:24:32
-814,Darrel,Maseres,540-324-4576,Male,6,20,52,2018-01-25 13:38:00,2019-01-11 23:26:49,2018-07-10 03:23:20,2018-06-27 20:41:34
-815,Merle,Haggie,867-109-5010,Male,5,96,73,2018-05-11 09:41:28,2018-02-26 23:39:01,2019-01-18 06:20:26,2018-11-24 20:35:34
-816,Jordanna,Slocket,561-552-2274,Female,3,64,28,2018-06-05 00:07:31,2018-10-24 18:30:56,2018-04-11 09:26:10,2018-07-17 15:09:53
-817,Rosa,Naulty,104-742-5724,Female,1,49,36,2018-07-25 05:49:28,2018-12-11 01:12:50,2018-04-02 22:12:09,2019-01-03 01:54:51
-818,Abbott,Nenci,465-121-4813,Male,3,79,57,2018-11-14 21:12:41,2018-03-22 04:16:18,2018-08-26 18:31:33,2018-05-16 10:32:25
-819,Edsel,Brockley,602-592-8501,Male,1,30,96,2018-01-21 00:58:41,2018-12-18 18:52:15,2018-03-18 23:16:05,2018-11-22 00:32:00
-820,Rickie,Tace,237-311-6775,Female,4,14,43,2018-07-28 16:49:11,2018-12-14 07:30:09,2018-02-18 22:51:29,2018-06-05 16:20:34
-821,Shaun,Stagg,356-435-7464,Female,7,54,35,2018-09-08 08:01:19,2018-03-09 16:11:16,2019-01-16 17:00:44,2018-11-30 14:34:15
-822,Alfred,Durden,512-204-4387,Male,2,79,46,2018-12-31 13:42:16,2018-05-24 21:54:26,2018-07-10 23:33:53,2018-08-10 04:33:54
-823,Christoffer,Bolderstone,868-441-3575,Male,6,4,30,2018-11-03 00:46:09,2018-04-30 04:41:20,2018-07-26 23:17:02,2018-04-27 19:17:50
-824,Ellette,Robard,806-623-1247,Female,2,59,70,2018-01-31 22:31:04,2018-01-28 04:26:39,2018-06-10 10:50:30,2018-11-01 17:27:42
-825,Loren,Mitrovic,840-960-0930,Male,7,27,31,2018-11-09 15:53:11,2018-03-25 02:53:57,2018-12-15 15:59:47,2018-06-20 19:00:48
-826,Leigh,Beebee,483-570-2017,Male,5,56,99,2018-03-27 09:56:26,2018-09-05 03:40:42,2018-12-13 03:28:25,2018-11-23 13:45:15
-827,Debby,Beldham,686-330-6591,Female,5,65,17,2018-08-01 18:59:48,2018-03-24 14:20:35,2019-01-10 21:11:42,2018-12-19 08:48:47
-828,Barron,Lowell,115-742-2050,Male,5,85,65,2018-03-03 04:32:02,2018-04-24 11:36:33,2018-09-15 15:08:52,2018-06-25 22:46:54
-829,Randi,Skydall,223-820-2793,Male,2,86,47,2018-07-20 00:15:51,2018-06-26 06:23:43,2018-12-28 12:30:07,2018-08-25 21:27:50
-830,Isabel,Roggerone,927-569-8210,Female,4,85,41,2018-09-23 13:54:09,2018-03-09 05:31:36,2018-06-11 18:42:42,2018-04-13 18:03:27
-831,Alvan,Geake,401-432-6471,Male,5,71,66,2018-03-29 20:49:43,2018-04-25 14:10:14,2018-07-24 01:39:45,2018-12-24 09:39:46
-832,Sophey,Jeandel,338-361-7635,Female,6,60,18,2018-08-05 04:29:15,2018-09-21 05:26:30,2018-06-08 07:30:41,2018-12-27 19:45:07
-833,Howie,Maxsted,197-553-3905,Male,5,94,69,2018-08-25 23:10:38,2019-01-20 02:12:00,2019-01-15 20:11:44,2018-02-13 17:46:32
-834,Padraig,Torr,529-469-0039,Male,1,62,17,2018-03-14 04:21:56,2018-06-08 14:24:59,2018-10-08 09:31:56,2018-08-30 02:59:15
-835,Shirl,Jeratt,713-937-2841,Female,2,18,89,2018-09-25 03:56:45,2018-07-17 02:54:21,2019-01-08 21:13:42,2018-02-20 17:16:17
-836,Durward,Kermott,625-654-9223,Male,6,67,25,2018-02-14 15:04:10,2018-03-23 06:42:12,2018-09-16 13:56:48,2018-03-10 11:02:08
-837,Rici,Francie,992-830-9527,Female,5,75,22,2018-07-21 07:35:59,2018-01-24 21:24:42,2018-12-20 10:46:19,2018-04-27 01:28:33
-838,Vivian,Suddell,715-471-1918,Female,6,81,45,2018-07-29 02:06:46,2018-10-15 14:05:00,2018-06-12 17:01:30,2018-05-12 22:22:36
-839,Eugenia,Tudhope,527-791-1359,Female,2,95,25,2019-01-05 12:54:58,2018-12-12 20:07:15,2018-12-22 12:11:13,2018-12-25 06:49:43
-840,Hamnet,Gateman,792-883-5589,Male,1,84,45,2018-06-26 04:15:53,2018-02-21 14:45:49,2018-08-29 18:01:14,2019-01-15 09:03:43
-841,Laraine,Emma,256-645-5057,Female,3,88,52,2018-05-29 03:26:20,2018-03-10 02:05:24,2018-05-12 21:29:38,2018-12-10 02:39:56
-842,Harley,D'Ruel,359-498-9270,Male,4,16,57,2018-02-15 07:39:22,2018-08-24 22:23:10,2018-07-19 02:07:06,2018-02-13 01:45:52
-843,Theda,Eades,222-998-5683,Female,7,2,71,2018-03-06 07:09:18,2018-09-14 11:46:16,2018-11-23 01:07:59,2018-12-02 03:37:22
-844,Giselbert,Klimentyev,165-127-0115,Male,6,88,93,2018-11-13 03:56:59,2018-04-09 02:33:25,2018-07-04 00:34:57,2018-06-04 05:56:23
-845,Lynnet,Tidd,370-452-0577,Female,6,47,3,2018-08-25 20:45:24,2018-08-27 16:50:01,2018-08-22 21:33:48,2018-06-27 22:32:52
-846,Donetta,Giannassi,913-466-0403,Female,2,39,55,2018-02-26 16:34:20,2018-07-10 03:35:10,2018-09-29 14:23:34,2018-10-10 13:02:02
-847,Conrade,Ruddell,514-722-6544,Male,3,13,100,2018-05-31 18:14:29,2018-10-19 05:51:32,2018-11-22 09:45:08,2018-02-22 05:00:31
-848,Tynan,Ritchie,552-416-9192,Male,4,50,26,2018-02-13 01:36:31,2018-07-24 03:28:00,2018-02-22 16:37:36,2018-02-28 07:16:54
-849,Erv,Huggins,528-946-8385,Male,3,73,82,2018-03-23 14:37:41,2018-08-12 12:26:08,2018-02-09 16:56:21,2018-08-09 06:33:34
-850,Darby,Ambrogini,266-338-3612,Male,6,92,21,2018-04-27 07:57:01,2018-10-19 20:49:49,2018-02-10 16:39:59,2018-09-03 00:53:41
-851,Devondra,Boc,775-193-7341,Female,5,36,51,2018-10-18 04:36:21,2018-10-28 14:06:15,2018-03-25 15:47:47,2018-08-20 22:31:48
-852,Meggy,Paxforde,435-917-5199,Female,2,40,85,2018-08-27 05:39:35,2018-01-23 20:18:58,2018-02-01 16:43:58,2018-08-07 12:15:36
-853,Pattie,Macallam,901-370-6579,Female,1,4,32,2018-07-11 10:30:06,2018-10-01 12:33:55,2018-02-08 02:35:59,2018-11-30 18:29:14
-854,Ray,Thayre,822-459-4971,Male,7,13,79,2018-07-13 03:44:05,2018-07-07 04:47:58,2018-08-28 19:24:34,2018-08-13 10:49:03
-855,Sindee,Gong,919-731-7167,Female,6,23,3,2018-12-08 15:31:48,2018-04-27 17:10:40,2018-07-14 14:10:46,2018-09-23 22:25:52
-856,Giovanni,Elsie,739-171-4273,Male,3,100,20,2018-11-22 21:14:50,2018-09-29 00:12:07,2018-05-17 09:43:48,2018-04-05 15:26:17
-857,Hadley,Davidde,813-908-2619,Male,7,64,65,2018-01-31 19:26:32,2018-11-10 09:08:19,2018-06-20 20:16:01,2018-04-18 19:33:00
-858,Jud,Schleicher,248-874-9513,Male,1,24,8,2018-04-06 16:54:03,2018-12-08 11:46:05,2018-10-02 20:22:44,2018-04-27 09:04:18
-859,Berget,Rasmus,599-370-5265,Female,2,46,8,2018-11-11 16:46:30,2018-11-24 16:42:40,2019-01-05 12:53:51,2018-12-17 18:41:54
-860,Floris,Lelliott,495-754-4852,Female,1,35,72,2019-01-06 22:27:28,2018-06-14 07:45:20,2018-10-25 10:22:56,2018-07-09 00:48:19
-861,Marianna,Huge,136-171-1910,Female,1,92,73,2018-05-04 21:19:28,2018-04-08 18:45:35,2018-02-08 18:06:55,2018-07-12 04:08:00
-862,Federica,Seakings,778-573-1492,Female,5,42,100,2018-02-18 23:40:45,2018-08-29 22:07:35,2018-06-27 21:37:10,2018-06-27 18:13:29
-863,Christophorus,Tett,516-196-0657,Male,3,68,89,2018-09-28 20:37:26,2018-07-03 17:19:03,2018-04-20 09:01:09,2018-06-11 11:31:56
-864,Kathlin,Hubbart,400-384-9734,Female,5,75,62,2018-09-23 11:16:50,2018-11-22 07:00:07,2018-11-12 09:29:23,2018-03-19 17:19:13
-865,Toinette,Christophersen,222-218-6841,Female,4,81,80,2018-06-24 07:27:29,2018-03-08 17:25:42,2018-11-19 23:39:15,2018-04-02 19:04:26
-866,Mei,Dragon,891-366-6658,Female,2,14,22,2018-03-17 07:33:08,2018-06-23 20:11:56,2018-10-30 02:21:14,2018-10-21 08:13:29
-867,Wileen,Guittet,118-965-5819,Female,7,22,73,2018-04-12 00:04:07,2018-08-23 08:56:44,2018-07-09 08:29:36,2018-02-22 06:45:41
-868,Bill,Skechley,531-261-9052,Female,1,93,45,2018-03-25 13:30:53,2018-10-17 03:47:35,2018-10-18 02:26:47,2018-12-08 19:05:40
-869,Chrissy,Rapley,863-731-8995,Female,7,17,8,2018-09-12 01:35:07,2018-07-22 15:02:10,2018-06-18 19:06:47,2018-12-22 20:00:52
-870,Gwynne,Hymers,287-878-7144,Female,4,6,29,2018-08-19 13:41:56,2018-12-10 18:23:35,2018-03-26 01:25:26,2018-07-05 16:38:09
-871,Noland,Hornung,922-794-4200,Male,3,66,82,2018-12-10 23:49:55,2018-09-15 01:16:20,2018-10-24 20:37:00,2018-10-27 07:56:58
-872,Sybila,Caren,883-754-0514,Female,1,40,33,2018-11-01 03:29:07,2018-03-07 23:07:07,2018-09-06 01:46:46,2018-03-17 17:02:04
-873,Arnaldo,Ortelt,648-204-0219,Male,5,24,20,2018-04-19 00:26:30,2018-06-09 12:00:33,2018-09-30 00:21:33,2018-07-12 18:59:39
-874,Sascha,Thurner,504-586-1976,Male,4,9,34,2018-09-03 11:21:47,2018-12-06 09:23:43,2018-11-21 03:43:51,2018-05-28 10:34:36
-875,Janeva,Stallion,168-726-9745,Female,7,66,60,2018-07-05 04:40:09,2018-02-19 12:25:04,2018-05-31 06:47:34,2018-01-24 21:43:59
-876,Imelda,Daintrey,786-279-9676,Female,5,26,12,2018-07-29 01:32:45,2018-05-21 03:09:18,2018-10-03 07:09:16,2018-12-28 16:07:06
-877,Kaia,Pauleit,937-468-0457,Female,5,36,63,2018-08-20 03:23:06,2018-09-20 08:02:05,2018-05-13 23:35:38,2018-03-19 03:48:06
-878,Fons,Coton,673-868-5084,Male,6,51,39,2018-02-28 14:51:40,2018-08-02 02:13:10,2018-02-01 01:38:38,2018-09-22 00:49:46
-879,Amalee,Groneway,336-875-1596,Female,5,19,58,2018-04-12 20:12:37,2018-11-27 06:39:14,2018-09-18 01:05:22,2018-04-22 08:39:02
-880,Paquito,O'Dwyer,324-526-6202,Male,7,99,41,2018-10-12 02:35:53,2018-02-10 00:52:03,2018-02-07 03:12:35,2018-10-18 21:47:13
-881,Shea,Forst,307-995-7972,Female,1,41,41,2018-10-02 02:03:39,2018-10-24 23:10:36,2019-01-09 17:11:05,2018-05-16 22:15:02
-882,Sylvan,McKimm,999-270-7047,Male,1,21,29,2018-04-13 22:42:31,2018-12-10 08:47:39,2018-05-25 01:01:54,2018-04-17 08:03:17
-883,Peyter,Rotte,919-508-2810,Male,3,51,43,2018-05-25 10:20:40,2018-05-07 09:22:17,2018-10-29 16:34:13,2018-05-08 18:00:18
-884,Ernestine,Jankowski,998-337-8068,Female,3,7,44,2018-01-22 22:18:12,2018-11-04 08:03:53,2019-01-14 13:07:45,2018-01-25 10:53:54
-885,Bridget,Pettet,701-799-8814,Female,5,84,73,2018-04-24 13:06:17,2018-09-27 02:37:43,2018-11-23 08:52:31,2018-08-30 18:46:39
-886,Arlana,Wisniowski,321-679-5349,Female,7,1,72,2018-09-20 23:41:45,2018-07-29 00:16:49,2018-02-08 09:55:35,2018-04-07 15:34:38
-887,Darbie,Verzey,936-723-9548,Female,6,95,2,2018-09-15 23:29:11,2018-01-28 09:02:28,2018-12-05 08:05:15,2018-02-08 01:02:29
-888,Quintana,Nutley,896-674-7083,Female,3,38,16,2018-07-22 20:52:42,2018-09-19 19:39:52,2018-04-29 01:58:40,2018-07-30 22:07:46
-889,Padgett,Urwen,141-490-6319,Male,6,32,96,2018-05-09 19:03:45,2018-04-07 12:16:47,2018-07-17 13:53:15,2018-09-19 15:54:14
-890,Beret,Gilchrest,524-146-3660,Female,3,30,45,2018-12-09 18:08:35,2018-02-09 07:44:41,2019-01-09 06:31:54,2018-06-03 05:03:43
-891,Jacquette,McBlain,180-111-2719,Female,5,45,49,2018-08-25 23:50:05,2018-08-14 04:10:24,2018-06-28 01:21:56,2018-05-09 06:07:09
-892,Heriberto,Hardway,934-411-2153,Male,7,100,67,2018-08-12 07:20:08,2018-08-30 23:04:44,2018-08-09 09:18:15,2018-08-21 13:19:44
-893,Charo,Cheeney,934-194-4292,Female,2,79,49,2018-02-19 02:30:34,2018-07-26 21:32:10,2018-03-14 13:44:53,2019-01-02 19:07:53
-894,Barty,Delacour,352-181-7260,Male,7,68,1,2018-02-08 11:54:33,2018-12-22 15:26:11,2018-05-05 05:29:07,2018-01-28 16:34:01
-895,Arel,Qusklay,776-527-2939,Male,6,72,24,2018-02-10 23:05:47,2018-10-31 16:14:34,2018-04-17 06:36:20,2018-09-21 15:13:20
-896,Cleon,Westcarr,735-686-0389,Male,1,56,68,2018-12-14 09:21:13,2018-08-04 15:56:37,2018-09-09 11:38:56,2018-03-19 03:56:38
-897,Nicole,Gorrie,216-361-4858,Female,2,65,77,2018-08-05 12:50:07,2018-03-10 14:07:01,2018-02-15 13:44:04,2019-01-06 01:05:50
-898,Issiah,Patters,901-518-7348,Male,7,40,78,2018-04-27 08:43:08,2018-11-08 01:40:02,2018-02-05 13:31:35,2018-08-05 21:26:55
-899,Kimbell,Torvey,614-432-0846,Male,3,46,34,2018-10-10 17:10:59,2018-04-09 13:30:10,2018-12-26 11:39:55,2018-03-11 23:16:21
-900,Arabel,Reaney,250-158-0650,Female,1,39,42,2018-06-24 20:58:36,2018-06-13 01:05:57,2018-09-21 12:17:09,2018-05-12 01:20:04
-901,Lowrance,Birckmann,707-493-5823,Male,3,40,81,2018-10-30 04:51:25,2018-04-14 15:18:25,2018-11-26 23:12:20,2018-11-19 04:09:43
-902,Sande,Tinson,644-792-0309,Female,1,4,93,2018-06-05 12:30:15,2018-06-05 01:29:00,2018-04-13 08:38:54,2018-12-10 03:36:34
-903,Antonetta,Twelve,506-143-6120,Female,7,95,49,2018-10-15 14:45:58,2018-11-02 11:22:24,2018-02-20 23:03:06,2018-02-13 22:09:09
-904,Marietta,Christensen,356-415-2235,Male,5,80,58,2018-10-22 11:01:49,2018-02-01 17:42:35,2018-11-22 02:54:09,2018-09-30 12:05:18
-905,Jodee,Pietraszek,300-188-3774,Female,4,10,37,2018-10-23 06:33:49,2018-10-21 00:19:46,2018-11-29 18:55:51,2018-11-06 15:41:47
-906,Marthena,Kitson,601-214-9309,Female,4,28,26,2018-07-23 16:23:23,2018-06-22 12:00:30,2018-12-01 12:31:45,2018-12-23 15:09:43
-907,Daron,Mackison,682-377-7219,Female,4,89,96,2018-05-28 03:03:57,2018-03-02 01:25:11,2018-12-01 09:00:59,2018-02-21 02:24:22
-908,Hube,Feldhuhn,942-914-1709,Male,5,24,35,2018-10-02 04:25:27,2018-06-28 02:29:41,2018-02-06 01:40:17,2018-04-23 22:52:35
-909,Loretta,Pritty,613-563-4790,Female,7,75,82,2018-07-08 17:13:07,2018-11-22 22:05:00,2018-10-30 22:21:57,2018-08-13 01:05:15
-910,Cathyleen,McCallion,883-828-7510,Female,2,45,11,2018-09-09 14:13:19,2018-01-31 15:57:37,2018-08-13 04:55:36,2018-09-09 10:40:55
-911,Haleigh,Gleader,682-121-8965,Female,3,8,38,2018-12-20 06:38:29,2019-01-17 01:03:40,2019-01-19 16:23:53,2018-08-17 03:24:51
-912,Charles,Rosengarten,618-810-0078,Male,2,93,36,2018-10-15 16:20:15,2018-12-28 14:31:32,2018-09-12 08:52:44,2018-07-15 21:25:15
-913,Bjorn,Erskine Sandys,529-581-3333,Male,3,75,81,2018-04-14 17:13:49,2018-10-21 15:46:05,2018-05-28 06:44:30,2018-08-10 04:37:46
-914,Darill,Bostock,625-801-8242,Male,2,81,23,2018-10-11 19:47:21,2018-06-12 04:06:41,2019-01-15 07:56:54,2018-04-28 18:50:01
-915,Noellyn,Benwell,722-714-8266,Female,2,11,12,2018-02-15 07:48:35,2018-04-07 17:41:22,2018-05-18 02:39:40,2018-07-22 07:12:38
-916,Dov,Letrange,300-443-5277,Male,5,91,80,2018-12-27 15:44:06,2018-06-01 01:17:13,2018-06-04 08:44:49,2018-05-07 08:43:09
-917,Olympie,Louden,465-432-6397,Female,1,19,13,2018-01-25 05:55:27,2018-06-19 10:44:49,2018-10-06 16:38:12,2018-11-22 20:46:37
-918,Finley,Casbon,425-531-2684,Male,4,85,21,2018-09-04 05:24:44,2018-10-19 21:14:24,2018-09-24 15:11:59,2018-07-02 05:41:48
-919,Dewitt,Krolak,700-648-7734,Male,7,43,3,2018-04-11 09:56:52,2018-07-11 01:42:09,2018-02-28 00:53:56,2018-06-20 15:39:09
-920,Noby,De la Yglesia,405-211-0709,Male,4,1,49,2018-12-05 02:27:53,2018-07-03 10:21:49,2018-09-23 09:16:19,2018-05-12 09:47:56
-921,Aloysius,Baggs,272-379-1280,Male,2,40,53,2018-05-14 22:08:41,2018-12-03 09:03:58,2018-07-15 04:40:33,2018-09-15 06:12:36
-922,Karilynn,Costa,732-871-1192,Female,2,68,76,2018-12-29 05:05:50,2018-12-23 08:00:43,2018-04-13 17:50:47,2018-03-24 11:13:51
-923,Ky,Waggett,413-701-9663,Male,5,7,59,2018-02-05 06:17:54,2018-07-07 03:47:33,2018-09-13 00:11:05,2018-12-21 03:34:37
-924,Leeanne,Neasam,130-573-1360,Female,4,74,77,2018-11-30 04:33:22,2018-03-26 07:40:00,2018-05-03 16:40:09,2018-05-10 20:32:41
-925,Muffin,Eades,789-128-6773,Male,4,84,25,2018-04-19 09:16:22,2018-11-08 03:55:54,2018-02-03 05:36:52,2018-10-21 22:14:13
-926,Iormina,Baudry,644-418-1313,Female,4,72,19,2019-01-16 00:35:47,2018-11-23 14:35:13,2018-04-06 17:59:52,2018-02-24 14:59:31
-927,Fabe,Ziemke,161-651-6080,Male,6,36,12,2019-01-14 08:28:41,2018-10-28 13:42:55,2018-07-07 04:53:18,2018-08-04 15:37:00
-928,Danni,Leese,768-599-1814,Female,7,30,71,2018-11-08 21:26:44,2018-05-01 17:49:56,2018-08-03 00:06:41,2018-10-18 08:20:47
-929,Rycca,Petegre,433-887-8926,Female,4,5,34,2018-07-02 07:56:41,2018-02-10 11:51:30,2018-04-08 10:08:07,2018-06-11 19:43:31
-930,Berkie,Reveland,791-499-2738,Male,1,57,80,2018-07-18 08:14:02,2018-04-15 08:54:18,2018-09-25 23:13:59,2018-08-12 23:49:27
-931,Tommy,Swett,915-667-6497,Male,1,98,11,2018-06-26 21:33:59,2018-07-10 07:15:47,2018-07-15 03:48:37,2018-04-01 11:58:09
-932,Derrek,Perkinson,169-674-1053,Male,4,85,25,2018-08-23 05:51:09,2018-09-17 23:57:57,2018-03-06 01:30:46,2018-05-06 22:10:10
-933,Lonni,Polycote,996-492-4081,Female,2,7,6,2018-06-09 22:12:53,2018-04-09 09:26:53,2018-07-19 03:36:09,2018-12-01 16:50:15
-934,Phyllis,Gallamore,788-489-0287,Female,4,32,71,2018-07-08 11:07:20,2018-05-25 09:24:00,2018-04-23 04:53:35,2019-01-19 16:11:17
-935,Brittne,Caccavari,799-716-9817,Female,2,53,25,2018-12-11 00:47:47,2018-01-22 09:05:30,2018-09-30 14:03:22,2018-08-09 05:13:34
-936,Quinn,Torricina,342-560-9404,Female,3,60,72,2019-01-06 06:26:23,2018-11-21 02:21:54,2018-02-15 20:43:28,2018-08-15 17:59:55
-937,Jacklyn,Bohling,824-878-8873,Female,1,41,34,2018-06-23 18:52:40,2018-05-14 22:58:13,2018-06-08 07:11:42,2018-08-04 15:57:46
-938,Hunfredo,Millam,816-916-1240,Male,1,86,58,2019-01-08 04:02:45,2019-01-12 17:57:33,2019-01-20 14:58:14,2018-12-03 22:29:35
-939,Hymie,Vosse,980-181-0718,Male,4,67,59,2018-12-25 10:58:52,2018-03-09 06:50:03,2018-03-03 04:06:28,2018-07-17 18:26:14
-940,Olia,Wyse,875-970-0069,Female,4,54,96,2018-11-01 03:11:42,2018-12-29 18:01:06,2018-05-01 00:58:58,2018-08-29 17:41:08
-941,Arline,Bierton,582-902-5400,Female,4,36,83,2018-11-19 01:06:41,2018-07-20 08:22:25,2018-02-15 09:56:13,2018-07-07 12:00:06
-942,Nevsa,Jacquot,393-800-6240,Female,3,76,100,2018-12-03 03:10:20,2018-10-05 15:30:34,2018-07-03 08:10:26,2019-01-15 02:21:14
-943,Analise,Bechley,178-276-2761,Female,4,10,4,2018-08-03 05:49:50,2018-12-25 22:04:16,2019-01-18 22:16:01,2018-03-07 14:10:26
-944,Siffre,Cumbridge,484-369-3900,Male,1,30,98,2018-11-08 05:35:17,2018-07-06 03:41:16,2018-03-26 11:58:13,2018-10-05 22:55:01
-945,Berget,Oldred,470-380-7016,Female,2,34,3,2018-12-24 01:06:10,2018-03-01 08:09:07,2018-08-27 19:47:44,2018-07-15 17:16:39
-946,Aaron,Arnison,195-265-7576,Male,4,68,94,2018-06-02 08:58:18,2018-08-03 00:56:00,2018-05-06 14:32:24,2018-02-16 07:54:16
-947,Fay,Sarfas,659-275-5381,Female,6,34,96,2019-01-02 00:06:51,2018-12-05 05:47:36,2018-04-22 03:52:40,2018-09-20 19:16:51
-948,Pennie,Rappaport,851-746-8484,Female,5,74,57,2018-06-13 14:50:19,2018-05-10 00:00:37,2018-05-03 15:48:50,2018-12-06 19:15:10
-949,Ennis,Schindler,685-873-8208,Male,2,9,15,2018-12-06 10:23:57,2018-06-05 08:10:38,2018-08-19 04:20:22,2018-08-31 04:33:19
-950,Sean,Breckwell,511-213-8051,Male,4,65,20,2018-06-20 04:24:36,2018-12-24 06:27:41,2018-07-17 12:13:34,2018-06-02 06:09:01
-951,Carlene,Vials,130-632-1736,Female,2,71,12,2018-03-26 06:16:26,2018-03-25 23:10:14,2018-03-02 15:43:13,2018-06-01 08:30:44
-952,Nerita,Paolinelli,943-836-6191,Female,2,88,24,2018-11-19 20:39:05,2018-09-18 20:22:25,2018-07-16 13:15:28,2018-12-21 02:47:25
-953,Benoite,Gheorghescu,378-772-1101,Female,4,78,44,2018-11-09 02:22:49,2019-01-09 18:23:07,2018-04-05 11:06:02,2018-05-14 05:39:56
-954,Erik,Auchterlonie,766-720-0406,Male,1,17,56,2018-03-01 23:55:26,2018-10-18 17:00:40,2018-07-12 00:30:26,2018-11-30 15:38:30
-955,Jesse,Laurenty,586-274-7629,Female,2,10,63,2018-08-16 12:07:51,2018-05-30 23:14:43,2018-06-19 02:24:42,2018-03-17 09:24:46
-956,Lynnet,Cuel,652-339-4808,Female,1,26,91,2018-11-05 19:14:13,2018-11-22 14:06:17,2018-12-25 13:52:54,2018-10-28 04:14:09
-957,Marge,Adnet,901-394-6344,Female,7,46,51,2018-02-21 02:14:23,2018-12-29 22:39:34,2018-03-27 11:48:32,2019-01-05 10:26:02
-958,Clara,Rangeley,154-876-9501,Female,6,73,50,2018-07-06 13:52:31,2018-05-28 12:09:22,2018-09-08 04:19:54,2018-09-02 19:04:27
-959,Ciel,Ching,205-133-7293,Female,7,26,19,2018-06-13 01:00:22,2018-06-26 05:13:24,2018-03-12 09:15:19,2018-06-29 23:55:03
-960,Phebe,Pentecust,671-551-3717,Female,6,79,26,2018-07-17 05:39:55,2018-05-28 11:52:54,2018-11-17 17:20:32,2018-07-27 07:39:40
-961,Koralle,Doleman,733-411-3555,Female,4,50,86,2018-10-28 12:28:47,2018-04-19 14:24:23,2018-04-19 20:04:57,2018-11-03 19:27:30
-962,Gypsy,Ashard,627-318-6598,Female,4,84,97,2018-10-29 08:41:04,2018-04-12 21:17:30,2019-01-12 05:14:18,2018-08-11 13:20:16
-963,Zebedee,Guilayn,442-936-9333,Male,3,35,49,2018-06-20 19:42:39,2019-01-19 12:44:29,2018-10-03 09:52:26,2018-11-23 22:02:46
-964,Waldemar,Isaak,931-973-9119,Male,4,82,45,2018-04-23 15:37:02,2018-05-09 15:24:30,2018-02-14 08:08:10,2018-07-03 18:15:44
-965,Zackariah,Donaway,407-803-4328,Male,1,89,1,2018-07-08 18:27:27,2018-03-04 22:13:59,2018-12-18 03:57:10,2018-07-07 10:37:23
-966,Amie,Graddon,806-683-2495,Female,7,24,48,2018-08-12 05:59:08,2018-02-02 02:33:10,2018-12-27 15:24:26,2018-09-09 14:21:34
-967,Homere,Hinkes,540-511-8058,Male,2,18,21,2018-02-15 19:40:23,2018-03-12 20:21:06,2018-07-24 20:35:51,2018-11-05 01:00:48
-968,Hildy,Romaint,578-715-3279,Female,1,64,42,2018-08-05 12:04:55,2018-10-30 20:39:13,2018-11-01 10:01:51,2018-10-13 16:01:35
-969,Norbie,Reinhardt,522-463-8856,Male,6,73,39,2018-12-26 16:31:22,2018-05-04 07:05:54,2018-11-01 19:36:06,2018-08-22 23:10:40
-970,Farlee,Robb,921-988-0129,Male,2,92,32,2018-09-25 21:50:48,2018-01-29 06:17:06,2018-12-05 20:31:07,2018-10-13 17:19:53
-971,Husain,Coomes,167-999-0871,Male,7,43,13,2018-06-22 13:30:21,2018-10-26 10:44:10,2019-01-18 20:06:38,2018-12-02 03:52:12
-972,Milo,Ikringill,848-791-7005,Male,6,93,81,2018-04-23 10:40:56,2018-11-16 14:25:18,2018-05-08 11:02:49,2018-09-27 02:14:23
-973,Bendite,Ivan,797-480-5441,Female,4,19,47,2018-03-24 15:48:07,2019-01-18 21:56:06,2018-12-06 11:16:03,2018-03-11 17:19:09
-974,Dall,Hewson,361-271-3138,Male,7,51,12,2018-12-01 16:04:22,2018-07-19 15:52:08,2018-11-26 22:45:09,2018-02-12 13:20:52
-975,Alfonse,Paddell,215-108-6430,Male,3,67,39,2018-07-20 13:46:01,2019-01-09 15:28:08,2018-05-15 18:34:09,2018-09-24 12:02:27
-976,Rachelle,Parysowna,448-408-3939,Female,6,75,37,2019-01-08 01:40:17,2019-01-14 14:50:14,2018-09-08 20:13:56,2018-08-26 17:34:13
-977,Vlad,Massimo,958-382-7108,Male,6,83,50,2019-01-10 14:14:56,2018-04-03 09:05:13,2018-07-12 05:33:43,2018-08-03 07:52:29
-978,Orran,Labarre,603-601-4448,Male,1,62,88,2018-08-02 15:34:45,2018-11-25 10:40:27,2018-02-25 06:26:19,2018-08-02 08:35:57
-979,Goldi,McGaw,418-474-0723,Female,4,92,65,2018-03-02 17:07:20,2018-03-04 17:41:59,2018-08-24 13:07:56,2018-11-04 17:29:39
-980,Ddene,O'Halloran,269-505-5567,Female,5,40,82,2018-07-19 04:42:22,2018-10-21 20:43:40,2018-05-31 18:32:46,2018-05-27 23:48:19
-981,Franciska,Randalston,518-473-9956,Female,2,31,23,2018-08-10 19:43:25,2018-07-27 07:50:34,2018-09-05 22:19:23,2018-07-03 10:32:00
-982,Izak,Aaronson,559-374-9622,Male,3,42,87,2018-08-15 00:25:50,2018-04-15 10:04:12,2018-06-28 14:35:41,2018-01-21 11:06:29
-983,Trey,Battany,836-241-1102,Male,2,15,41,2019-01-10 17:37:08,2018-09-21 17:33:14,2018-10-31 07:41:53,2018-05-02 04:23:23
-984,Monty,Print,851-362-4374,Male,1,72,54,2018-07-29 11:59:40,2018-02-03 04:49:31,2019-01-10 13:05:52,2018-12-15 23:29:32
-985,Ashton,Cunnah,780-629-9147,Male,4,45,20,2018-02-10 13:25:53,2018-12-25 15:32:25,2018-01-23 10:29:44,2018-05-31 16:54:21
-986,Becky,Stavers,700-388-4222,Female,6,10,79,2018-03-04 20:32:59,2018-06-13 23:55:18,2018-06-02 22:41:53,2018-11-04 01:01:48
-987,Udall,Undrell,187-686-5781,Male,3,62,28,2018-07-29 04:00:33,2018-08-10 09:03:52,2018-12-20 19:09:22,2019-01-08 10:29:39
-988,Wait,Kippie,649-986-8035,Male,6,59,99,2018-08-26 01:11:54,2018-11-11 11:43:07,2018-11-26 18:10:23,2018-05-03 06:50:49
-989,Laurence,Dubs,725-348-9456,Male,5,80,78,2018-12-29 16:41:25,2018-03-04 01:25:41,2018-07-24 13:07:42,2018-03-02 01:22:54
-990,Cyrille,Ansley,464-181-4696,Male,7,71,25,2018-10-12 23:36:32,2018-05-18 07:03:48,2018-12-20 18:06:28,2018-09-16 15:37:18
-991,Emory,Manwell,415-640-8797,Male,5,12,56,2018-07-21 03:32:53,2018-11-15 00:05:50,2018-06-11 19:00:20,2018-03-30 06:39:44
-992,Chevalier,Pelosi,210-721-7355,Male,1,3,53,2018-09-10 01:51:52,2018-08-05 09:51:36,2018-07-09 12:50:42,2018-12-20 17:33:50
-993,Vale,Havenhand,885-493-5138,Male,5,37,72,2018-11-01 03:19:32,2018-11-02 06:12:05,2019-01-18 19:09:41,2018-07-29 06:54:36
-994,Shel,Kislingbury,683-829-9112,Female,2,57,75,2018-07-06 02:01:25,2018-11-17 02:11:03,2018-07-20 22:39:14,2018-05-08 16:48:52
-995,Kris,Rodear,296-572-9182,Male,5,24,22,2018-05-21 05:50:41,2018-12-04 14:27:38,2018-09-01 22:26:00,2018-09-05 14:46:56
-996,Jillayne,Mackerel,971-775-5324,Female,4,77,29,2018-08-29 09:36:26,2018-09-18 17:04:42,2018-12-08 03:41:26,2018-11-28 11:52:45
-997,Wilona,Yonnie,985-459-5111,Female,4,36,62,2018-08-07 01:17:54,2018-08-25 03:30:42,2018-02-17 20:26:24,2018-03-09 21:08:39
-998,Herbert,Liver,278-295-4343,Male,1,10,92,2018-04-27 00:14:36,2018-08-27 15:24:57,2018-02-11 18:53:26,2018-05-05 07:57:16
-999,Charmian,Robison,320-515-4401,Female,4,66,52,2018-11-29 19:32:40,2018-09-14 07:10:38,2018-07-04 09:36:16,2018-02-20 20:59:30
-1000,Cad,Alster,344-476-5489,Male,6,50,52,2018-09-12 09:49:44,2018-04-09 17:52:16,2018-12-18 02:15:47,2018-04-22 03:31:11
diff --git a/vitess-mixin/e2e/external_db/mysql/grant.sh b/vitess-mixin/e2e/external_db/mysql/grant.sh
deleted file mode 100755
index 9371377d074..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/grant.sh
+++ /dev/null
@@ -1,7 +0,0 @@
-
-echo '**********GRANTING PRIVILEGES START*******************'
-echo ${mysql[@]}
-# PURGE BINARY LOGS BEFORE DATE(NOW());
-mysql --protocol=socket -uroot -hlocalhost --socket=/var/run/mysqld/mysqld.sock -p$MYSQL_ROOT_PASSWORD -e \
-"GRANT ALL PRIVILEGES ON *.* TO '$MYSQL_USER'@'%' IDENTIFIED BY '$MYSQL_PASSWORD'"
-echo '*************GRANTING PRIVILEGES END****************'
diff --git a/vitess-mixin/e2e/external_db/mysql/mysql56.cnf b/vitess-mixin/e2e/external_db/mysql/mysql56.cnf
deleted file mode 100644
index fdd34b1bd2e..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/mysql56.cnf
+++ /dev/null
@@ -1,41 +0,0 @@
-[mysqld]
-# Options for enabling GTID
-# https://dev.mysql.com/doc/refman/5.6/en/replication-gtids-howto.html
-gtid_mode = ON
-log_bin
-log_slave_updates
-enforce_gtid_consistency
-
-# Crash-safe replication settings.
-master_info_repository = TABLE
-relay_log_info_repository = TABLE
-relay_log_purge = 1
-relay_log_recovery = 1
-
-# Native AIO tends to run into aio-max-nr limit during test startup.
-innodb_use_native_aio = 0
-
-# Semi-sync replication is required for automated unplanned failover
-# (when the master goes away). Here we just load the plugin so it's
-# available if desired, but it's disabled at startup.
-#
-# VTTablet will enable semi-sync at the proper time when replication is set up,
-# or when a primary is promoted or demoted based on the durability policy configured.
-plugin-load = rpl_semi_sync_master=semisync_master.so;rpl_semi_sync_slave=semisync_slave.so
-
-# When semi-sync is enabled, don't allow fallback to async
-# if you get no ack, or have no slaves. This is necessary to
-# prevent alternate futures when doing a failover in response to
-# a master that becomes unresponsive.
-rpl_semi_sync_master_timeout = 1000000000000000000
-rpl_semi_sync_master_wait_no_slave = 1
-server-id = 1
-# Remove ONLY_FULL_GROUP_BY until queries are fixed
-sql_mode = "STRICT_TRANS_TABLES,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION"
-
-general_log = 1
-general_log_file = /var/log/mysql/query.log
-
-slow_query_log = 1
-long_query_time = 1 # seconds
-slow_query_log_file = /var/log/mysql/slow.log
\ No newline at end of file
diff --git a/vitess-mixin/e2e/external_db/mysql/mysql57.cnf b/vitess-mixin/e2e/external_db/mysql/mysql57.cnf
deleted file mode 100644
index ebf301187eb..00000000000
--- a/vitess-mixin/e2e/external_db/mysql/mysql57.cnf
+++ /dev/null
@@ -1,43 +0,0 @@
-# This file is auto-included when MySQL 5.7 is detected.
-
-# MySQL 5.7 does not enable the binary log by default, and
-# info repositories default to file
-[mysqld]
-gtid_mode = ON
-log_slave_updates
-enforce_gtid_consistency
-expire_logs_days = 3
-master_info_repository = TABLE
-relay_log_info_repository = TABLE
-relay_log_purge = 1
-relay_log_recovery = 1
-
-# In MySQL 5.7 the default charset is latin1
-
-character_set_server = utf8
-collation_server = utf8_general_ci
-
-# Semi-sync replication is required for automated unplanned failover
-# (when the master goes away). Here we just load the plugin so it's
-# available if desired, but it's disabled at startup.
-#
-# VTTablet will enable semi-sync at the proper time when replication is set up,
-# or when a primary is promoted or demoted based on the durability policy configured.
-plugin-load = rpl_semi_sync_master=semisync_master.so;rpl_semi_sync_slave=semisync_slave.so
-
-# When semi-sync is enabled, don't allow fallback to async
-# if you get no ack, or have no slaves. This is necessary to
-# prevent alternate futures when doing a failover in response to
-# a master that becomes unresponsive.
-rpl_semi_sync_master_timeout = 1000000000000000000
-rpl_semi_sync_master_wait_no_slave = 1
-
-## Specific edits for this external_db ##
-log_bin
-server-id = 1
-general_log = 1
-general_log_file = /var/log/mysql/query.log
-
-slow_query_log = 1
-long_query_time = 1 # seconds
-slow_query_log_file = /var/log/mysql/slow.log
diff --git a/vitess-mixin/e2e/externaldb_vreplication.sh b/vitess-mixin/e2e/externaldb_vreplication.sh
deleted file mode 100755
index 1138909aa13..00000000000
--- a/vitess-mixin/e2e/externaldb_vreplication.sh
+++ /dev/null
@@ -1,75 +0,0 @@
-#!/bin/bash
-
-# Copyright 2020 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.
-
-set -ex
-
-VTCTLD_SERVER=${VTCTLD_SERVER:-'vtctld:15999'}
-
-# Wait until source and destination primaries are available
-until (/vt/bin/vtctlclient --server $VTCTLD_SERVER ListAllTablets | grep "ext_" | grep "primary" ); do
- echo 'waiting for external primary..';
- sleep 1;
-done
-
-until (/vt/bin/vtctlclient --server $VTCTLD_SERVER ListAllTablets | grep -v "ext_" | grep "primary" ); do
- echo 'waiting for managed primary..';
- sleep 1;
-done
-
-
-# Get source and destination tablet and shard information
-TABLET_INFO=$(/vt/bin/vtctlclient --server $VTCTLD_SERVER ListAllTablets)
-source_alias=$(echo "$TABLET_INFO "| grep "ext_" | grep "primary" | awk '{ print $1 }')
-dest_alias=$(echo "$TABLET_INFO "| grep -v "ext_" | grep "primary" | awk '{ print $1 }')
-source_keyspace=$(echo "$TABLET_INFO "| grep "ext_" | grep "primary" | awk '{ print $2 }')
-dest_keyspace=$(echo "$TABLET_INFO "| grep -v "ext_" | grep "primary" | awk '{ print $2 }')
-source_shard=$(echo "$TABLET_INFO "| grep "ext_" | grep "primary" | awk '{ print $3 }')
-dest_shard=$(echo "$TABLET_INFO "| grep -v "ext_" | grep "primary" | awk '{ print $3 }')
-source_tablet=$(echo "$TABLET_INFO "| grep "ext_" | grep "primary" | awk '{ print $2 "/" $3}')
-dest_tablet=$(echo "$TABLET_INFO "| grep -v "ext_" | grep "primary" | awk '{ print $2 "/" $3}')
-
-
-# Disable foreign_key checks on destination
-/vt/bin/vtctlclient --server $VTCTLD_SERVER ExecuteFetchAsDba $dest_alias 'SET GLOBAL FOREIGN_KEY_CHECKS=0;'
-
-# Get source_sql mode
-source_sql_mode=$(/vt/bin/vtctlclient --server $VTCTLD_SERVER ExecuteFetchAsDba $source_alias 'SELECT @@GLOBAL.sql_mode' | awk 'NR==4 {print $2}')
-
-# Apply source sql_mode to destination
-# The intention is to avoid replication errors
-/vt/bin/vtctlclient --server $VTCTLD_SERVER ExecuteFetchAsDba $dest_alias "SET GLOBAL sql_mode='$source_sql_mode';"
-
-# Verify sql_mode matches
-[ $source_sql_mode == $(/vt/bin/vtctlclient --server $VTCTLD_SERVER ExecuteFetchAsDba $dest_alias 'SELECT @@GLOBAL.sql_mode' | awk 'NR==4 {print $2}') ] && \
-echo "Source and Destination sql_mode Match." || echo "sql_mode MisMatch"
-
-until /vt/bin/vtctlclient --server $VTCTLD_SERVER GetSchema $dest_alias; do
- echo "Waiting for destination schema to be ready..";
- sleep 3;
-done
-
-# Copy schema from source to destination shard
-/vt/bin/vtctlclient --server $VTCTLD_SERVER CopySchemaShard $source_tablet $dest_tablet || true
-
-# Verify schema
-/vt/bin/vtctlclient --server $VTCTLD_SERVER GetSchema $dest_alias
-
-# Start vreplication
-/vt/bin/vtctlclient --server $VTCTLD_SERVER VReplicationExec $dest_alias 'insert into _vt.vreplication (db_name, source, pos, max_tps, max_replication_lag, tablet_types, time_updated, transaction_timestamp, state) values('"'"''"$dest_keyspace"''"'"', '"'"'keyspace:\"'"$source_keyspace"'\" shard:\"'"$source_shard"'\" filter: > on_ddl:EXEC_IGNORE '"'"', '"'"''"'"', 9999, 9999, '"'"'primary'"'"', 0, 0, '"'"'Running'"'"')'
-
-# Check vreplication status
-/vt/bin/vtctlclient --server $VTCTLD_SERVER VReplicationExec $dest_alias 'select * from _vt.vreplication'
-
diff --git a/vitess-mixin/e2e/fix_replication.sh b/vitess-mixin/e2e/fix_replication.sh
deleted file mode 100755
index e6a06c71124..00000000000
--- a/vitess-mixin/e2e/fix_replication.sh
+++ /dev/null
@@ -1,83 +0,0 @@
-#!/bin/bash
-
-# Copyright 2019 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.
-
-# This is a helper script to sync replicas for mysql.
-# It handles the special case where the primary has purged bin logs that the replica requires.
-# To use it place a mysql dump of the database on the same directory as this script.
-# The name of the dump must be $KEYSPACE.sql. The script can also download the mysqldump for you.
-# Replication is fixed by restoring the mysqldump and resetting the replica.
-# https://dev.mysql.com/doc/refman/5.7/en/replication-mode-change-online-disable-gtids.html
-# https://www.percona.com/blog/2013/02/08/how-to-createrestore-a-slave-using-gtid-replication-in-mysql-5-6/
-
-cd "$(dirname "${BASH_SOURCE[0]}")"
-
-function get_replication_status() {
- # Get replica status
- STATUS_LINE=$(mysql -u$DB_USER -p$DB_PASS -h 127.0.0.1 -e "SHOW SLAVE STATUS\G")
- LAST_ERRNO=$(grep "Last_IO_Errno:" <<< "$STATUS_LINE" | awk '{ print $2 }')
- SLAVE_SQL_RUNNING=$(grep "Slave_SQL_Running:" <<< "$STATUS_LINE" | awk '{ print $2 }')
- SLAVE_IO_RUNNING=$(grep "Slave_IO_Running:" <<< "$STATUS_LINE" | awk '{ print $2 }')
- MASTER_HOST=$(grep "Master_Host:" <<< "$STATUS_LINE" | awk '{ print $2 }')
- MASTER_PORT=$(grep "Master_Port:" <<< "$STATUS_LINE" | awk '{ print $2 }')
-
- echo "Slave_SQL_Running: $SLAVE_SQL_RUNNING"
- echo "Slave_IO_Running: $SLAVE_IO_RUNNING"
- echo "Last_IO_Errno: $LAST_ERRNO"
-}
-
-function reset_replication() {
- # Necessary before sql file can be imported
- echo "Importing MysqlDump: $KEYSPACE.sql"
- mysql -u$DB_USER -p$DB_PASS -h 127.0.0.1 -e "RESET MASTER;STOP SLAVE;CHANGE MASTER TO MASTER_AUTO_POSITION = 0;source $KEYSPACE.sql;START SLAVE;"
- # Restore Master Auto Position
- echo "Restoring Master Auto Setting"
- mysql -u$DB_USER -p$DB_PASS -h 127.0.0.1 -e "STOP SLAVE;CHANGE MASTER TO MASTER_AUTO_POSITION = 1;START SLAVE;"
-}
-
-# Retrieve replication status
-get_replication_status
-
-# Exit script if called with argument 'status'
-[ ${1:-''} != 'status' ] || exit 0;
-
-# Check if SLAVE_IO is running
-if [[ $SLAVE_IO_RUNNING = "No" && $LAST_ERRNO = 1236 ]]; then
-
- echo "Primary has purged bin logs that replica requires. Sync will require restore from mysqldump"
- if [[ -f $KEYSPACE.sql ]] ; then
- echo "mysqldump file $KEYSPACE.sql exists, attempting to restore.."
- echo "Resetting replication.."
- reset_replication
- else
- echo "Starting mysqldump. This may take a while.."
- # Modify flags to user's requirements
- if mysqldump -h $MASTER_HOST -P $MASTER_PORT -u$DB_USER -p$DB_PASS --databases $KEYSPACE \
- --triggers --routines --events --hex-blob --master-data=1 --quick --order-by-primary \
- --no-autocommit --skip-comments --skip-add-drop-table --skip-add-locks \
- --skip-disable-keys --single-transaction --set-gtid-purged=on --verbose > $KEYSPACE.sql; then
- echo "mysqldump complete for database $KEYSPACE"
- echo "Resetting replication.."
- reset_replication
- else
- echo "mysqldump failed for database $KEYSPACE"
- fi
- fi
-
-else
-
- echo "No Actions to perform"
-
-fi
diff --git a/vitess-mixin/e2e/grafana/provisioning/dashboards/dashboard.yml b/vitess-mixin/e2e/grafana/provisioning/dashboards/dashboard.yml
deleted file mode 100644
index edfcd4f2271..00000000000
--- a/vitess-mixin/e2e/grafana/provisioning/dashboards/dashboard.yml
+++ /dev/null
@@ -1,11 +0,0 @@
-apiVersion: 1
-
-providers:
-- name: 'Vitess Mixin'
- orgId: 1
- folder: ''
- type: file
- disableDeletion: false
- editable: true
- options:
- path: /etc/grafana/provisioning/dashboards
\ No newline at end of file
diff --git a/vitess-mixin/e2e/grafana/provisioning/datasources/datasource.yml b/vitess-mixin/e2e/grafana/provisioning/datasources/datasource.yml
deleted file mode 100644
index a2140b8834e..00000000000
--- a/vitess-mixin/e2e/grafana/provisioning/datasources/datasource.yml
+++ /dev/null
@@ -1,22 +0,0 @@
-apiVersion: 1
-
-deleteDatasources:
- - name: Prometheus
- orgId: 1
-
-datasources:
-
-- name: Prometheus
- type: prometheus
- access: proxy
- orgId: 1
- url: http://prometheus:9090
- user:
- database:
- basicAuth: false
- basicAuthUser:
- basicAuthPassword:
- withCredentials:
- isDefault: true
- version: 1
- editable: true
\ No newline at end of file
diff --git a/vitess-mixin/e2e/lfixrepl.sh b/vitess-mixin/e2e/lfixrepl.sh
deleted file mode 100755
index bd3e5646019..00000000000
--- a/vitess-mixin/e2e/lfixrepl.sh
+++ /dev/null
@@ -1,25 +0,0 @@
-#!/bin/bash
-
-# Copyright 2019 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.
-
-# Enable tty for Windows users using git-bash or cygwin
-if [[ "$OSTYPE" == "msys" ]]; then
- # Lightweight shell and GNU utilities compiled for Windows (part of MinGW)
- tty=winpty
- script=//script//fix_replication.sh
-fi
-
-# This is a convenience script to fix replication on replicas.
-exec $tty docker-compose exec ${CS:-vttablet2} ${script:-/script/fix_replication.sh} "$@"
\ No newline at end of file
diff --git a/vitess-mixin/e2e/lmysql.sh b/vitess-mixin/e2e/lmysql.sh
deleted file mode 100755
index d735e3783a8..00000000000
--- a/vitess-mixin/e2e/lmysql.sh
+++ /dev/null
@@ -1,24 +0,0 @@
-#!/bin/bash
-
-# Copyright 2019 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.
-
-# Enable tty for Windows users using git-bash or cygwin
-if [[ "$OSTYPE" == "msys" ]]; then
- # Lightweight shell and GNU utilities compiled for Windows (part of MinGW)
- tty=winpty
-fi
-
-# This is a convenience script to run mysql client against the local example.
-exec $tty docker-compose exec ${CS:-vttablet101} mysql "$@"
diff --git a/vitess-mixin/e2e/load_test.sql b/vitess-mixin/e2e/load_test.sql
deleted file mode 100644
index 71e7d805386..00000000000
--- a/vitess-mixin/e2e/load_test.sql
+++ /dev/null
@@ -1,46 +0,0 @@
--- INSERT TEST DATA
--- mysql --port=15306 --host=127.0.0.1 < load_test.sql
--- SIMULATED QUERIES
--- mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=test_keyspace:80-@primary --query="SELECT * FROM messages;"
--- mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=test_keyspace:80-@replica --query="SELECT * FROM messages;"
--- mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=lookup_keyspace:-@primary --query="SELECT * FROM messages_message_lookup;"
--- mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=lookup_keyspace:-@replica --query="SELECT * FROM messages_message_lookup;"
--- SIMULATED ERRORS
--- ╰─$ mysqlslap --port=15306 --host=127.0.0.1 --iterations=10000 --create-schema=test_keyspace:80-@primary --query="SELECT name FROM messages;"
--- ╰─$ mysqlslap --port=15306 --host=127.0.0.1 --iterations=10000 --create-schema=lookup_keyspace:-@replica --query="SELECT name FROM messages_message_lookup;"
-
-USE test_keyspace:80-@primary;
-INSERT INTO messages (page,time_created_ns,message) VALUES
-(1,1,'test'),
-(2,2,'test'),
-(3,3,'test'),
-(4,4,'test'),
-(5,5,'test'),
-(6,6,'test'),
-(7,7,'test'),
-(8,8,'test'),
-(9,9,'test');
-
-USE test_keyspace:-80@primary;
-INSERT INTO messages (page,time_created_ns,message) VALUES
-(10,1,'test'),
-(11,2,'test'),
-(12,3,'test'),
-(13,4,'test'),
-(14,5,'test'),
-(15,6,'test'),
-(16,7,'test'),
-(17,8,'test'),
-(18,9,'test');
-
-USE lookup_keyspace:-@primary;
-INSERT INTO messages_message_lookup (id,page,message) VALUES
-(1,1,'test'),
-(2,2,'test'),
-(3,3,'test'),
-(4,4,'test'),
-(5,5,'test'),
-(6,6,'test'),
-(7,7,'test'),
-(8,8,'test'),
-(9,9,'test');
diff --git a/vitess-mixin/e2e/lookup_keyspace_vschema.json b/vitess-mixin/e2e/lookup_keyspace_vschema.json
deleted file mode 100644
index f67289821fe..00000000000
--- a/vitess-mixin/e2e/lookup_keyspace_vschema.json
+++ /dev/null
@@ -1,26 +0,0 @@
-{
- "sharded": true,
- "tables": {
- "messages_message_lookup": {
- "column_vindexes": [
- {
- "column": "id",
- "name": "hash"
- }
- ]
- },
- "tokens_token_lookup": {
- "column_vindexes": [
- {
- "column": "id",
- "name": "hash"
- }
- ]
- }
- },
- "vindexes": {
- "hash": {
- "type": "hash"
- }
- }
-}
\ No newline at end of file
diff --git a/vitess-mixin/e2e/lvtctl.sh b/vitess-mixin/e2e/lvtctl.sh
deleted file mode 100755
index 64110743073..00000000000
--- a/vitess-mixin/e2e/lvtctl.sh
+++ /dev/null
@@ -1,24 +0,0 @@
-#!/bin/bash
-
-# Copyright 2019 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.
-
-# Enable tty for Windows users using git-bash or cygwin
-if [[ "$OSTYPE" == "msys" ]]; then
- # Lightweight shell and GNU utilities compiled for Windows (part of MinGW)
- tty=winpty
-fi
-
-# This is a convenience script to run vtctlclient against the local example.
-exec $tty docker-compose exec ${CS:-vtctld} vtctlclient -server vtctld:15999 "$@"
diff --git a/vitess-mixin/e2e/package-lock.json b/vitess-mixin/e2e/package-lock.json
deleted file mode 100644
index d686ef99233..00000000000
--- a/vitess-mixin/e2e/package-lock.json
+++ /dev/null
@@ -1,1483 +0,0 @@
-{
- "requires": true,
- "lockfileVersion": 1,
- "dependencies": {
- "@colors/colors": {
- "version": "1.5.0",
- "resolved": "https://registry.npmjs.org/@colors/colors/-/colors-1.5.0.tgz",
- "integrity": "sha512-ooWCrlZP11i8GImSjTHYHLkvFDP48nS4+204nGb1RiX/WXYHmJA2III9/e2DWVabCESdW7hBAEzHRqUn9OUVvQ==",
- "dev": true,
- "optional": true
- },
- "@cypress/request": {
- "version": "3.0.1",
- "resolved": "https://registry.npmjs.org/@cypress/request/-/request-3.0.1.tgz",
- "integrity": "sha512-TWivJlJi8ZDx2wGOw1dbLuHJKUYX7bWySw377nlnGOW3hP9/MUKIsEdXT/YngWxVdgNCHRBmFlBipE+5/2ZZlQ==",
- "dev": true,
- "requires": {
- "aws-sign2": "~0.7.0",
- "aws4": "^1.8.0",
- "caseless": "~0.12.0",
- "combined-stream": "~1.0.6",
- "extend": "~3.0.2",
- "forever-agent": "~0.6.1",
- "form-data": "~2.3.2",
- "http-signature": "~1.3.6",
- "is-typedarray": "~1.0.0",
- "isstream": "~0.1.2",
- "json-stringify-safe": "~5.0.1",
- "mime-types": "~2.1.19",
- "performance-now": "^2.1.0",
- "qs": "6.10.4",
- "safe-buffer": "^5.1.2",
- "tough-cookie": "^4.1.3",
- "tunnel-agent": "^0.6.0",
- "uuid": "^8.3.2"
- }
- },
- "@cypress/xvfb": {
- "version": "1.2.4",
- "resolved": "https://registry.npmjs.org/@cypress/xvfb/-/xvfb-1.2.4.tgz",
- "integrity": "sha512-skbBzPggOVYCbnGgV+0dmBdW/s77ZkAOXIC1knS8NagwDjBrNC1LuXtQJeiN6l+m7lzmHtaoUw/ctJKdqkG57Q==",
- "dev": true,
- "requires": {
- "debug": "^3.1.0",
- "lodash.once": "^4.1.1"
- },
- "dependencies": {
- "debug": {
- "version": "3.2.7",
- "resolved": "https://registry.npmjs.org/debug/-/debug-3.2.7.tgz",
- "integrity": "sha512-CFjzYYAi4ThfiQvizrFQevTTXHtnCqWfe7x1AhgEscTz6ZbLbfoLRLPugTQyBth6f8ZERVUSyWHFD/7Wu4t1XQ==",
- "dev": true,
- "requires": {
- "ms": "^2.1.1"
- }
- }
- }
- },
- "@types/node": {
- "version": "18.17.17",
- "resolved": "https://registry.npmjs.org/@types/node/-/node-18.17.17.tgz",
- "integrity": "sha512-cOxcXsQ2sxiwkykdJqvyFS+MLQPLvIdwh5l6gNg8qF6s+C7XSkEWOZjK+XhUZd+mYvHV/180g2cnCcIl4l06Pw==",
- "dev": true
- },
- "@types/sinonjs__fake-timers": {
- "version": "8.1.1",
- "resolved": "https://registry.npmjs.org/@types/sinonjs__fake-timers/-/sinonjs__fake-timers-8.1.1.tgz",
- "integrity": "sha512-0kSuKjAS0TrGLJ0M/+8MaFkGsQhZpB6pxOmvS3K8FYI72K//YmdfoW9X2qPsAKh1mkwxGD5zib9s1FIFed6E8g==",
- "dev": true
- },
- "@types/sizzle": {
- "version": "2.3.3",
- "resolved": "https://registry.npmjs.org/@types/sizzle/-/sizzle-2.3.3.tgz",
- "integrity": "sha512-JYM8x9EGF163bEyhdJBpR2QX1R5naCJHC8ucJylJ3w9/CVBaskdQ8WqBf8MmQrd1kRvp/a4TS8HJ+bxzR7ZJYQ==",
- "dev": true
- },
- "@types/yauzl": {
- "version": "2.10.0",
- "resolved": "https://registry.npmjs.org/@types/yauzl/-/yauzl-2.10.0.tgz",
- "integrity": "sha512-Cn6WYCm0tXv8p6k+A8PvbDG763EDpBoTzHdA+Q/MF6H3sapGjCm9NzoaJncJS9tUKSuCoDs9XHxYYsQDgxR6kw==",
- "dev": true,
- "optional": true,
- "requires": {
- "@types/node": "*"
- }
- },
- "aggregate-error": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/aggregate-error/-/aggregate-error-3.1.0.tgz",
- "integrity": "sha512-4I7Td01quW/RpocfNayFdFVk1qSuoh0E7JrbRJ16nH01HhKFQ88INq9Sd+nd72zqRySlr9BmDA8xlEJ6vJMrYA==",
- "dev": true,
- "requires": {
- "clean-stack": "^2.0.0",
- "indent-string": "^4.0.0"
- }
- },
- "ansi-colors": {
- "version": "4.1.3",
- "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.3.tgz",
- "integrity": "sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw==",
- "dev": true
- },
- "ansi-escapes": {
- "version": "4.3.2",
- "resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-4.3.2.tgz",
- "integrity": "sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==",
- "dev": true,
- "requires": {
- "type-fest": "^0.21.3"
- }
- },
- "ansi-regex": {
- "version": "5.0.1",
- "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-5.0.1.tgz",
- "integrity": "sha512-quJQXlTSUGL2LH9SUXo8VwsY4soanhgo6LNSm84E1LBcE8s3O0wpdiRzyR9z/ZZJMlMWv37qOOb9pdJlMUEKFQ==",
- "dev": true
- },
- "ansi-styles": {
- "version": "4.3.0",
- "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-4.3.0.tgz",
- "integrity": "sha512-zbB9rCJAT1rbjiVDb2hqKFHNYLxgtk8NURxZ3IZwD3F6NtxbXZQCnnSi1Lkx+IDohdPlFp222wVALIheZJQSEg==",
- "dev": true,
- "requires": {
- "color-convert": "^2.0.1"
- }
- },
- "arch": {
- "version": "2.2.0",
- "resolved": "https://registry.npmjs.org/arch/-/arch-2.2.0.tgz",
- "integrity": "sha512-Of/R0wqp83cgHozfIYLbBMnej79U/SVGOOyuB3VVFv1NRM/PSFMK12x9KVtiYzJqmnU5WR2qp0Z5rHb7sWGnFQ==",
- "dev": true
- },
- "asn1": {
- "version": "0.2.6",
- "resolved": "https://registry.npmjs.org/asn1/-/asn1-0.2.6.tgz",
- "integrity": "sha512-ix/FxPn0MDjeyJ7i/yoHGFt/EX6LyNbxSEhPPXODPL+KB0VPk86UYfL0lMdy+KCnv+fmvIzySwaK5COwqVbWTQ==",
- "dev": true,
- "requires": {
- "safer-buffer": "~2.1.0"
- }
- },
- "assert-plus": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/assert-plus/-/assert-plus-1.0.0.tgz",
- "integrity": "sha512-NfJ4UzBCcQGLDlQq7nHxH+tv3kyZ0hHQqF5BO6J7tNJeP5do1llPr8dZ8zHonfhAu0PHAdMkSo+8o0wxg9lZWw==",
- "dev": true
- },
- "astral-regex": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/astral-regex/-/astral-regex-2.0.0.tgz",
- "integrity": "sha512-Z7tMw1ytTXt5jqMcOP+OQteU1VuNK9Y02uuJtKQ1Sv69jXQKKg5cibLwGJow8yzZP+eAc18EmLGPal0bp36rvQ==",
- "dev": true
- },
- "async": {
- "version": "3.2.4",
- "resolved": "https://registry.npmjs.org/async/-/async-3.2.4.tgz",
- "integrity": "sha512-iAB+JbDEGXhyIUavoDl9WP/Jj106Kz9DEn1DPgYw5ruDn0e3Wgi3sKFm55sASdGBNOQB8F59d9qQ7deqrHA8wQ==",
- "dev": true
- },
- "asynckit": {
- "version": "0.4.0",
- "resolved": "https://registry.npmjs.org/asynckit/-/asynckit-0.4.0.tgz",
- "integrity": "sha512-Oei9OH4tRh0YqU3GxhX79dM/mwVgvbZJaSNaRk+bshkj0S5cfHcgYakreBjrHwatXKbz+IoIdYLxrKim2MjW0Q==",
- "dev": true
- },
- "at-least-node": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/at-least-node/-/at-least-node-1.0.0.tgz",
- "integrity": "sha512-+q/t7Ekv1EDY2l6Gda6LLiX14rU9TV20Wa3ofeQmwPFZbOMo9DXrLbOjFaaclkXKWidIaopwAObQDqwWtGUjqg==",
- "dev": true
- },
- "aws-sign2": {
- "version": "0.7.0",
- "resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz",
- "integrity": "sha512-08kcGqnYf/YmjoRhfxyu+CLxBjUtHLXLXX/vUfx9l2LYzG3c1m61nrpyFUZI6zeS+Li/wWMMidD9KgrqtGq3mA==",
- "dev": true
- },
- "aws4": {
- "version": "1.12.0",
- "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.12.0.tgz",
- "integrity": "sha512-NmWvPnx0F1SfrQbYwOi7OeaNGokp9XhzNioJ/CSBs8Qa4vxug81mhJEAVZwxXuBmYB5KDRfMq/F3RR0BIU7sWg==",
- "dev": true
- },
- "balanced-match": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz",
- "integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw==",
- "dev": true
- },
- "base64-js": {
- "version": "1.5.1",
- "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz",
- "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==",
- "dev": true
- },
- "bcrypt-pbkdf": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/bcrypt-pbkdf/-/bcrypt-pbkdf-1.0.2.tgz",
- "integrity": "sha512-qeFIXtP4MSoi6NLqO12WfqARWWuCKi2Rn/9hJLEmtB5yTNr9DqFWkJRCf2qShWzPeAMRnOgCrq0sg/KLv5ES9w==",
- "dev": true,
- "requires": {
- "tweetnacl": "^0.14.3"
- }
- },
- "blob-util": {
- "version": "2.0.2",
- "resolved": "https://registry.npmjs.org/blob-util/-/blob-util-2.0.2.tgz",
- "integrity": "sha512-T7JQa+zsXXEa6/8ZhHcQEW1UFfVM49Ts65uBkFL6fz2QmrElqmbajIDJvuA0tEhRe5eIjpV9ZF+0RfZR9voJFQ==",
- "dev": true
- },
- "bluebird": {
- "version": "3.7.2",
- "resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz",
- "integrity": "sha512-XpNj6GDQzdfW+r2Wnn7xiSAd7TM3jzkxGXBGTtWKuSXv1xUV+azxAm8jdWZN06QTQk+2N2XB9jRDkvbmQmcRtg==",
- "dev": true
- },
- "brace-expansion": {
- "version": "1.1.11",
- "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-1.1.11.tgz",
- "integrity": "sha512-iCuPHDFgrHX7H2vEI/5xpz07zSHB00TpugqhmYtVmMO6518mCuRMoOYFldEBl0g187ufozdaHgWKcYFb61qGiA==",
- "dev": true,
- "requires": {
- "balanced-match": "^1.0.0",
- "concat-map": "0.0.1"
- }
- },
- "buffer": {
- "version": "5.7.1",
- "resolved": "https://registry.npmjs.org/buffer/-/buffer-5.7.1.tgz",
- "integrity": "sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ==",
- "dev": true,
- "requires": {
- "base64-js": "^1.3.1",
- "ieee754": "^1.1.13"
- }
- },
- "buffer-crc32": {
- "version": "0.2.13",
- "resolved": "https://registry.npmjs.org/buffer-crc32/-/buffer-crc32-0.2.13.tgz",
- "integrity": "sha512-VO9Ht/+p3SN7SKWqcrgEzjGbRSJYTx+Q1pTQC0wrWqHx0vpJraQ6GtHx8tvcg1rlK1byhU5gccxgOgj7B0TDkQ==",
- "dev": true
- },
- "cachedir": {
- "version": "2.4.0",
- "resolved": "https://registry.npmjs.org/cachedir/-/cachedir-2.4.0.tgz",
- "integrity": "sha512-9EtFOZR8g22CL7BWjJ9BUx1+A/djkofnyW3aOXZORNW2kxoUpx2h+uN2cOqwPmFhnpVmxg+KW2OjOSgChTEvsQ==",
- "dev": true
- },
- "call-bind": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/call-bind/-/call-bind-1.0.2.tgz",
- "integrity": "sha512-7O+FbCihrB5WGbFYesctwmTKae6rOiIzmz1icreWJ+0aA7LJfuqhEso2T9ncpcFtzMQtzXf2QGGueWJGTYsqrA==",
- "dev": true,
- "requires": {
- "function-bind": "^1.1.1",
- "get-intrinsic": "^1.0.2"
- }
- },
- "caseless": {
- "version": "0.12.0",
- "resolved": "https://registry.npmjs.org/caseless/-/caseless-0.12.0.tgz",
- "integrity": "sha512-4tYFyifaFfGacoiObjJegolkwSU4xQNGbVgUiNYVUxbQ2x2lUsFvY4hVgVzGiIe6WLOPqycWXA40l+PWsxthUw==",
- "dev": true
- },
- "chalk": {
- "version": "4.1.2",
- "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.2.tgz",
- "integrity": "sha512-oKnbhFyRIXpUuez8iBMmyEa4nbj4IOQyuhc/wy9kY7/WVPcwIO9VA668Pu8RkO7+0G76SLROeyw9CpQ061i4mA==",
- "dev": true,
- "requires": {
- "ansi-styles": "^4.1.0",
- "supports-color": "^7.1.0"
- },
- "dependencies": {
- "supports-color": {
- "version": "7.2.0",
- "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-7.2.0.tgz",
- "integrity": "sha512-qpCAvRl9stuOHveKsn7HncJRvv501qIacKzQlO/+Lwxc9+0q2wLyv4Dfvt80/DPn2pqOBsJdDiogXGR9+OvwRw==",
- "dev": true,
- "requires": {
- "has-flag": "^4.0.0"
- }
- }
- }
- },
- "check-more-types": {
- "version": "2.24.0",
- "resolved": "https://registry.npmjs.org/check-more-types/-/check-more-types-2.24.0.tgz",
- "integrity": "sha512-Pj779qHxV2tuapviy1bSZNEL1maXr13bPYpsvSDB68HlYcYuhlDrmGd63i0JHMCLKzc7rUSNIrpdJlhVlNwrxA==",
- "dev": true
- },
- "ci-info": {
- "version": "3.8.0",
- "resolved": "https://registry.npmjs.org/ci-info/-/ci-info-3.8.0.tgz",
- "integrity": "sha512-eXTggHWSooYhq49F2opQhuHWgzucfF2YgODK4e1566GQs5BIfP30B0oenwBJHfWxAs2fyPB1s7Mg949zLf61Yw==",
- "dev": true
- },
- "clean-stack": {
- "version": "2.2.0",
- "resolved": "https://registry.npmjs.org/clean-stack/-/clean-stack-2.2.0.tgz",
- "integrity": "sha512-4diC9HaTE+KRAMWhDhrGOECgWZxoevMc5TlkObMqNSsVU62PYzXZ/SMTjzyGAFF1YusgxGcSWTEXBhp0CPwQ1A==",
- "dev": true
- },
- "cli-cursor": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/cli-cursor/-/cli-cursor-3.1.0.tgz",
- "integrity": "sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw==",
- "dev": true,
- "requires": {
- "restore-cursor": "^3.1.0"
- }
- },
- "cli-table3": {
- "version": "0.6.3",
- "resolved": "https://registry.npmjs.org/cli-table3/-/cli-table3-0.6.3.tgz",
- "integrity": "sha512-w5Jac5SykAeZJKntOxJCrm63Eg5/4dhMWIcuTbo9rpE+brgaSZo0RuNJZeOyMgsUdhDeojvgyQLmjI+K50ZGyg==",
- "dev": true,
- "requires": {
- "@colors/colors": "1.5.0",
- "string-width": "^4.2.0"
- }
- },
- "cli-truncate": {
- "version": "2.1.0",
- "resolved": "https://registry.npmjs.org/cli-truncate/-/cli-truncate-2.1.0.tgz",
- "integrity": "sha512-n8fOixwDD6b/ObinzTrp1ZKFzbgvKZvuz/TvejnLn1aQfC6r52XEx85FmuC+3HI+JM7coBRXUvNqEU2PHVrHpg==",
- "dev": true,
- "requires": {
- "slice-ansi": "^3.0.0",
- "string-width": "^4.2.0"
- }
- },
- "color-convert": {
- "version": "2.0.1",
- "resolved": "https://registry.npmjs.org/color-convert/-/color-convert-2.0.1.tgz",
- "integrity": "sha512-RRECPsj7iu/xb5oKYcsFHSppFNnsj/52OVTRKb4zP5onXwVF3zVmmToNcOfGC+CRDpfK/U584fMg38ZHCaElKQ==",
- "dev": true,
- "requires": {
- "color-name": "~1.1.4"
- }
- },
- "color-name": {
- "version": "1.1.4",
- "resolved": "https://registry.npmjs.org/color-name/-/color-name-1.1.4.tgz",
- "integrity": "sha512-dOy+3AuW3a2wNbZHIuMZpTcgjGuLU/uBL/ubcZF9OXbDo8ff4O8yVp5Bf0efS8uEoYo5q4Fx7dY9OgQGXgAsQA==",
- "dev": true
- },
- "colorette": {
- "version": "2.0.20",
- "resolved": "https://registry.npmjs.org/colorette/-/colorette-2.0.20.tgz",
- "integrity": "sha512-IfEDxwoWIjkeXL1eXcDiow4UbKjhLdq6/EuSVR9GMN7KVH3r9gQ83e73hsz1Nd1T3ijd5xv1wcWRYO+D6kCI2w==",
- "dev": true
- },
- "combined-stream": {
- "version": "1.0.8",
- "resolved": "https://registry.npmjs.org/combined-stream/-/combined-stream-1.0.8.tgz",
- "integrity": "sha512-FQN4MRfuJeHf7cBbBMJFXhKSDq+2kAArBlmRBvcvFE5BB1HZKXtSFASDhdlz9zOYwxh8lDdnvmMOe/+5cdoEdg==",
- "dev": true,
- "requires": {
- "delayed-stream": "~1.0.0"
- }
- },
- "commander": {
- "version": "6.2.1",
- "resolved": "https://registry.npmjs.org/commander/-/commander-6.2.1.tgz",
- "integrity": "sha512-U7VdrJFnJgo4xjrHpTzu0yrHPGImdsmD95ZlgYSEajAn2JKzDhDTPG9kBTefmObL2w/ngeZnilk+OV9CG3d7UA==",
- "dev": true
- },
- "common-tags": {
- "version": "1.8.2",
- "resolved": "https://registry.npmjs.org/common-tags/-/common-tags-1.8.2.tgz",
- "integrity": "sha512-gk/Z852D2Wtb//0I+kRFNKKE9dIIVirjoqPoA1wJU+XePVXZfGeBpk45+A1rKO4Q43prqWBNY/MiIeRLbPWUaA==",
- "dev": true
- },
- "concat-map": {
- "version": "0.0.1",
- "resolved": "https://registry.npmjs.org/concat-map/-/concat-map-0.0.1.tgz",
- "integrity": "sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==",
- "dev": true
- },
- "core-util-is": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/core-util-is/-/core-util-is-1.0.2.tgz",
- "integrity": "sha512-3lqz5YjWTYnW6dlDa5TLaTCcShfar1e40rmcJVwCBJC6mWlFuj0eCHIElmG1g5kyuJ/GD+8Wn4FFCcz4gJPfaQ==",
- "dev": true
- },
- "cross-spawn": {
- "version": "7.0.3",
- "resolved": "https://registry.npmjs.org/cross-spawn/-/cross-spawn-7.0.3.tgz",
- "integrity": "sha512-iRDPJKUPVEND7dHPO8rkbOnPpyDygcDFtWjpeWNCgy8WP2rXcxXL8TskReQl6OrB2G7+UJrags1q15Fudc7G6w==",
- "dev": true,
- "requires": {
- "path-key": "^3.1.0",
- "shebang-command": "^2.0.0",
- "which": "^2.0.1"
- }
- },
- "cypress": {
- "version": "13.2.0",
- "resolved": "https://registry.npmjs.org/cypress/-/cypress-13.2.0.tgz",
- "integrity": "sha512-AvDQxBydE771GTq0TR4ZUBvv9m9ffXuB/ueEtpDF/6gOcvFR96amgwSJP16Yhqw6VhmwqspT5nAGzoxxB+D89g==",
- "dev": true,
- "requires": {
- "@cypress/request": "^3.0.0",
- "@cypress/xvfb": "^1.2.4",
- "@types/node": "^18.17.5",
- "@types/sinonjs__fake-timers": "8.1.1",
- "@types/sizzle": "^2.3.2",
- "arch": "^2.2.0",
- "blob-util": "^2.0.2",
- "bluebird": "^3.7.2",
- "buffer": "^5.6.0",
- "cachedir": "^2.3.0",
- "chalk": "^4.1.0",
- "check-more-types": "^2.24.0",
- "cli-cursor": "^3.1.0",
- "cli-table3": "~0.6.1",
- "commander": "^6.2.1",
- "common-tags": "^1.8.0",
- "dayjs": "^1.10.4",
- "debug": "^4.3.4",
- "enquirer": "^2.3.6",
- "eventemitter2": "6.4.7",
- "execa": "4.1.0",
- "executable": "^4.1.1",
- "extract-zip": "2.0.1",
- "figures": "^3.2.0",
- "fs-extra": "^9.1.0",
- "getos": "^3.2.1",
- "is-ci": "^3.0.0",
- "is-installed-globally": "~0.4.0",
- "lazy-ass": "^1.6.0",
- "listr2": "^3.8.3",
- "lodash": "^4.17.21",
- "log-symbols": "^4.0.0",
- "minimist": "^1.2.8",
- "ospath": "^1.2.2",
- "pretty-bytes": "^5.6.0",
- "process": "^0.11.10",
- "proxy-from-env": "1.0.0",
- "request-progress": "^3.0.0",
- "semver": "^7.5.3",
- "supports-color": "^8.1.1",
- "tmp": "~0.2.1",
- "untildify": "^4.0.0",
- "yauzl": "^2.10.0"
- }
- },
- "dashdash": {
- "version": "1.14.1",
- "resolved": "https://registry.npmjs.org/dashdash/-/dashdash-1.14.1.tgz",
- "integrity": "sha512-jRFi8UDGo6j+odZiEpjazZaWqEal3w/basFjQHQEwVtZJGDpxbH1MeYluwCS8Xq5wmLJooDlMgvVarmWfGM44g==",
- "dev": true,
- "requires": {
- "assert-plus": "^1.0.0"
- }
- },
- "dayjs": {
- "version": "1.11.10",
- "resolved": "https://registry.npmjs.org/dayjs/-/dayjs-1.11.10.tgz",
- "integrity": "sha512-vjAczensTgRcqDERK0SR2XMwsF/tSvnvlv6VcF2GIhg6Sx4yOIt/irsr1RDJsKiIyBzJDpCoXiWWq28MqH2cnQ==",
- "dev": true
- },
- "debug": {
- "version": "4.3.4",
- "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz",
- "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==",
- "dev": true,
- "requires": {
- "ms": "2.1.2"
- },
- "dependencies": {
- "ms": {
- "version": "2.1.2",
- "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz",
- "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==",
- "dev": true
- }
- }
- },
- "delayed-stream": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/delayed-stream/-/delayed-stream-1.0.0.tgz",
- "integrity": "sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==",
- "dev": true
- },
- "ecc-jsbn": {
- "version": "0.1.2",
- "resolved": "https://registry.npmjs.org/ecc-jsbn/-/ecc-jsbn-0.1.2.tgz",
- "integrity": "sha512-eh9O+hwRHNbG4BLTjEl3nw044CkGm5X6LoaCf7LPp7UU8Qrt47JYNi6nPX8xjW97TKGKm1ouctg0QSpZe9qrnw==",
- "dev": true,
- "requires": {
- "jsbn": "~0.1.0",
- "safer-buffer": "^2.1.0"
- }
- },
- "emoji-regex": {
- "version": "8.0.0",
- "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-8.0.0.tgz",
- "integrity": "sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==",
- "dev": true
- },
- "end-of-stream": {
- "version": "1.4.4",
- "resolved": "https://registry.npmjs.org/end-of-stream/-/end-of-stream-1.4.4.tgz",
- "integrity": "sha512-+uw1inIHVPQoaVuHzRyXd21icM+cnt4CzD5rW+NC1wjOUSTOs+Te7FOv7AhN7vS9x/oIyhLP5PR1H+phQAHu5Q==",
- "dev": true,
- "requires": {
- "once": "^1.4.0"
- }
- },
- "enquirer": {
- "version": "2.4.1",
- "resolved": "https://registry.npmjs.org/enquirer/-/enquirer-2.4.1.tgz",
- "integrity": "sha512-rRqJg/6gd538VHvR3PSrdRBb/1Vy2YfzHqzvbhGIQpDRKIa4FgV/54b5Q1xYSxOOwKvjXweS26E0Q+nAMwp2pQ==",
- "dev": true,
- "requires": {
- "ansi-colors": "^4.1.1",
- "strip-ansi": "^6.0.1"
- }
- },
- "escape-string-regexp": {
- "version": "1.0.5",
- "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz",
- "integrity": "sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg==",
- "dev": true
- },
- "eventemitter2": {
- "version": "6.4.7",
- "resolved": "https://registry.npmjs.org/eventemitter2/-/eventemitter2-6.4.7.tgz",
- "integrity": "sha512-tYUSVOGeQPKt/eC1ABfhHy5Xd96N3oIijJvN3O9+TsC28T5V9yX9oEfEK5faP0EFSNVOG97qtAS68GBrQB2hDg==",
- "dev": true
- },
- "execa": {
- "version": "4.1.0",
- "resolved": "https://registry.npmjs.org/execa/-/execa-4.1.0.tgz",
- "integrity": "sha512-j5W0//W7f8UxAn8hXVnwG8tLwdiUy4FJLcSupCg6maBYZDpyBvTApK7KyuI4bKj8KOh1r2YH+6ucuYtJv1bTZA==",
- "dev": true,
- "requires": {
- "cross-spawn": "^7.0.0",
- "get-stream": "^5.0.0",
- "human-signals": "^1.1.1",
- "is-stream": "^2.0.0",
- "merge-stream": "^2.0.0",
- "npm-run-path": "^4.0.0",
- "onetime": "^5.1.0",
- "signal-exit": "^3.0.2",
- "strip-final-newline": "^2.0.0"
- }
- },
- "executable": {
- "version": "4.1.1",
- "resolved": "https://registry.npmjs.org/executable/-/executable-4.1.1.tgz",
- "integrity": "sha512-8iA79xD3uAch729dUG8xaaBBFGaEa0wdD2VkYLFHwlqosEj/jT66AzcreRDSgV7ehnNLBW2WR5jIXwGKjVdTLg==",
- "dev": true,
- "requires": {
- "pify": "^2.2.0"
- }
- },
- "extend": {
- "version": "3.0.2",
- "resolved": "https://registry.npmjs.org/extend/-/extend-3.0.2.tgz",
- "integrity": "sha512-fjquC59cD7CyW6urNXK0FBufkZcoiGG80wTuPujX590cB5Ttln20E2UB4S/WARVqhXffZl2LNgS+gQdPIIim/g==",
- "dev": true
- },
- "extract-zip": {
- "version": "2.0.1",
- "resolved": "https://registry.npmjs.org/extract-zip/-/extract-zip-2.0.1.tgz",
- "integrity": "sha512-GDhU9ntwuKyGXdZBUgTIe+vXnWj0fppUEtMDL0+idd5Sta8TGpHssn/eusA9mrPr9qNDym6SxAYZjNvCn/9RBg==",
- "dev": true,
- "requires": {
- "@types/yauzl": "^2.9.1",
- "debug": "^4.1.1",
- "get-stream": "^5.1.0",
- "yauzl": "^2.10.0"
- }
- },
- "extsprintf": {
- "version": "1.3.0",
- "resolved": "https://registry.npmjs.org/extsprintf/-/extsprintf-1.3.0.tgz",
- "integrity": "sha512-11Ndz7Nv+mvAC1j0ktTa7fAb0vLyGGX+rMHNBYQviQDGU0Hw7lhctJANqbPhu9nV9/izT/IntTgZ7Im/9LJs9g==",
- "dev": true
- },
- "fd-slicer": {
- "version": "1.1.0",
- "resolved": "https://registry.npmjs.org/fd-slicer/-/fd-slicer-1.1.0.tgz",
- "integrity": "sha512-cE1qsB/VwyQozZ+q1dGxR8LBYNZeofhEdUNGSMbQD3Gw2lAzX9Zb3uIU6Ebc/Fmyjo9AWWfnn0AUCHqtevs/8g==",
- "dev": true,
- "requires": {
- "pend": "~1.2.0"
- }
- },
- "figures": {
- "version": "3.2.0",
- "resolved": "https://registry.npmjs.org/figures/-/figures-3.2.0.tgz",
- "integrity": "sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg==",
- "dev": true,
- "requires": {
- "escape-string-regexp": "^1.0.5"
- }
- },
- "forever-agent": {
- "version": "0.6.1",
- "resolved": "https://registry.npmjs.org/forever-agent/-/forever-agent-0.6.1.tgz",
- "integrity": "sha512-j0KLYPhm6zeac4lz3oJ3o65qvgQCcPubiyotZrXqEaG4hNagNYO8qdlUrX5vwqv9ohqeT/Z3j6+yW067yWWdUw==",
- "dev": true
- },
- "form-data": {
- "version": "2.3.3",
- "resolved": "https://registry.npmjs.org/form-data/-/form-data-2.3.3.tgz",
- "integrity": "sha512-1lLKB2Mu3aGP1Q/2eCOx0fNbRMe7XdwktwOruhfqqd0rIJWwN4Dh+E3hrPSlDCXnSR7UtZ1N38rVXm+6+MEhJQ==",
- "dev": true,
- "requires": {
- "asynckit": "^0.4.0",
- "combined-stream": "^1.0.6",
- "mime-types": "^2.1.12"
- }
- },
- "fs-extra": {
- "version": "9.1.0",
- "resolved": "https://registry.npmjs.org/fs-extra/-/fs-extra-9.1.0.tgz",
- "integrity": "sha512-hcg3ZmepS30/7BSFqRvoo3DOMQu7IjqxO5nCDt+zM9XWjb33Wg7ziNT+Qvqbuc3+gWpzO02JubVyk2G4Zvo1OQ==",
- "dev": true,
- "requires": {
- "at-least-node": "^1.0.0",
- "graceful-fs": "^4.2.0",
- "jsonfile": "^6.0.1",
- "universalify": "^2.0.0"
- },
- "dependencies": {
- "universalify": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/universalify/-/universalify-2.0.0.tgz",
- "integrity": "sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ==",
- "dev": true
- }
- }
- },
- "fs.realpath": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz",
- "integrity": "sha512-OO0pH2lK6a0hZnAdau5ItzHPI6pUlvI7jMVnxUQRtw4owF2wk8lOSabtGDCTP4Ggrg2MbGnWO9X8K1t4+fGMDw==",
- "dev": true
- },
- "function-bind": {
- "version": "1.1.1",
- "resolved": "https://registry.npmjs.org/function-bind/-/function-bind-1.1.1.tgz",
- "integrity": "sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==",
- "dev": true
- },
- "get-intrinsic": {
- "version": "1.2.1",
- "resolved": "https://registry.npmjs.org/get-intrinsic/-/get-intrinsic-1.2.1.tgz",
- "integrity": "sha512-2DcsyfABl+gVHEfCOaTrWgyt+tb6MSEGmKq+kI5HwLbIYgjgmMcV8KQ41uaKz1xxUcn9tJtgFbQUEVcEbd0FYw==",
- "dev": true,
- "requires": {
- "function-bind": "^1.1.1",
- "has": "^1.0.3",
- "has-proto": "^1.0.1",
- "has-symbols": "^1.0.3"
- }
- },
- "get-stream": {
- "version": "5.2.0",
- "resolved": "https://registry.npmjs.org/get-stream/-/get-stream-5.2.0.tgz",
- "integrity": "sha512-nBF+F1rAZVCu/p7rjzgA+Yb4lfYXrpl7a6VmJrU8wF9I1CKvP/QwPNZHnOlwbTkY6dvtFIzFMSyQXbLoTQPRpA==",
- "dev": true,
- "requires": {
- "pump": "^3.0.0"
- }
- },
- "getos": {
- "version": "3.2.1",
- "resolved": "https://registry.npmjs.org/getos/-/getos-3.2.1.tgz",
- "integrity": "sha512-U56CfOK17OKgTVqozZjUKNdkfEv6jk5WISBJ8SHoagjE6L69zOwl3Z+O8myjY9MEW3i2HPWQBt/LTbCgcC973Q==",
- "dev": true,
- "requires": {
- "async": "^3.2.0"
- }
- },
- "getpass": {
- "version": "0.1.7",
- "resolved": "https://registry.npmjs.org/getpass/-/getpass-0.1.7.tgz",
- "integrity": "sha512-0fzj9JxOLfJ+XGLhR8ze3unN0KZCgZwiSSDz168VERjK8Wl8kVSdcu2kspd4s4wtAa1y/qrVRiAA0WclVsu0ng==",
- "dev": true,
- "requires": {
- "assert-plus": "^1.0.0"
- }
- },
- "glob": {
- "version": "7.2.3",
- "resolved": "https://registry.npmjs.org/glob/-/glob-7.2.3.tgz",
- "integrity": "sha512-nFR0zLpU2YCaRxwoCJvL6UvCH2JFyFVIvwTLsIf21AuHlMskA1hhTdk+LlYJtOlYt9v6dvszD2BGRqBL+iQK9Q==",
- "dev": true,
- "requires": {
- "fs.realpath": "^1.0.0",
- "inflight": "^1.0.4",
- "inherits": "2",
- "minimatch": "^3.1.1",
- "once": "^1.3.0",
- "path-is-absolute": "^1.0.0"
- }
- },
- "global-dirs": {
- "version": "3.0.1",
- "resolved": "https://registry.npmjs.org/global-dirs/-/global-dirs-3.0.1.tgz",
- "integrity": "sha512-NBcGGFbBA9s1VzD41QXDG+3++t9Mn5t1FpLdhESY6oKY4gYTFpX4wO3sqGUa0Srjtbfj3szX0RnemmrVRUdULA==",
- "dev": true,
- "requires": {
- "ini": "2.0.0"
- }
- },
- "graceful-fs": {
- "version": "4.2.11",
- "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz",
- "integrity": "sha512-RbJ5/jmFcNNCcDV5o9eTnBLJ/HszWV0P73bc+Ff4nS/rJj+YaS6IGyiOL0VoBYX+l1Wrl3k63h/KrH+nhJ0XvQ==",
- "dev": true
- },
- "has": {
- "version": "1.0.3",
- "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz",
- "integrity": "sha512-f2dvO0VU6Oej7RkWJGrehjbzMAjFp5/VKPp5tTpWIV4JHHZK1/BxbFRtf/siA2SWTe09caDmVtYYzWEIbBS4zw==",
- "dev": true,
- "requires": {
- "function-bind": "^1.1.1"
- }
- },
- "has-flag": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/has-flag/-/has-flag-4.0.0.tgz",
- "integrity": "sha512-EykJT/Q1KjTWctppgIAgfSO0tKVuZUjhgMr17kqTumMl6Afv3EISleU7qZUzoXDFTAHTDC4NOoG/ZxU3EvlMPQ==",
- "dev": true
- },
- "has-proto": {
- "version": "1.0.1",
- "resolved": "https://registry.npmjs.org/has-proto/-/has-proto-1.0.1.tgz",
- "integrity": "sha512-7qE+iP+O+bgF9clE5+UoBFzE65mlBiVj3tKCrlNQ0Ogwm0BjpT/gK4SlLYDMybDh5I3TCTKnPPa0oMG7JDYrhg==",
- "dev": true
- },
- "has-symbols": {
- "version": "1.0.3",
- "resolved": "https://registry.npmjs.org/has-symbols/-/has-symbols-1.0.3.tgz",
- "integrity": "sha512-l3LCuF6MgDNwTDKkdYGEihYjt5pRPbEg46rtlmnSPlUbgmB8LOIrKJbYYFBSbnPaJexMKtiPO8hmeRjRz2Td+A==",
- "dev": true
- },
- "http-signature": {
- "version": "1.3.6",
- "resolved": "https://registry.npmjs.org/http-signature/-/http-signature-1.3.6.tgz",
- "integrity": "sha512-3adrsD6zqo4GsTqtO7FyrejHNv+NgiIfAfv68+jVlFmSr9OGy7zrxONceFRLKvnnZA5jbxQBX1u9PpB6Wi32Gw==",
- "dev": true,
- "requires": {
- "assert-plus": "^1.0.0",
- "jsprim": "^2.0.2",
- "sshpk": "^1.14.1"
- }
- },
- "human-signals": {
- "version": "1.1.1",
- "resolved": "https://registry.npmjs.org/human-signals/-/human-signals-1.1.1.tgz",
- "integrity": "sha512-SEQu7vl8KjNL2eoGBLF3+wAjpsNfA9XMlXAYj/3EdaNfAlxKthD1xjEQfGOUhllCGGJVNY34bRr6lPINhNjyZw==",
- "dev": true
- },
- "ieee754": {
- "version": "1.2.1",
- "resolved": "https://registry.npmjs.org/ieee754/-/ieee754-1.2.1.tgz",
- "integrity": "sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==",
- "dev": true
- },
- "indent-string": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/indent-string/-/indent-string-4.0.0.tgz",
- "integrity": "sha512-EdDDZu4A2OyIK7Lr/2zG+w5jmbuk1DVBnEwREQvBzspBJkCEbRa8GxU1lghYcaGJCnRWibjDXlq779X1/y5xwg==",
- "dev": true
- },
- "inflight": {
- "version": "1.0.6",
- "resolved": "https://registry.npmjs.org/inflight/-/inflight-1.0.6.tgz",
- "integrity": "sha512-k92I/b08q4wvFscXCLvqfsHCrjrF7yiXsQuIVvVE7N82W3+aqpzuUdBbfhWcy/FZR3/4IgflMgKLOsvPDrGCJA==",
- "dev": true,
- "requires": {
- "once": "^1.3.0",
- "wrappy": "1"
- }
- },
- "inherits": {
- "version": "2.0.4",
- "resolved": "https://registry.npmjs.org/inherits/-/inherits-2.0.4.tgz",
- "integrity": "sha512-k/vGaX4/Yla3WzyMCvTQOXYeIHvqOKtnqBduzTHpzpQZzAskKMhZ2K+EnBiSM9zGSoIFeMpXKxa4dYeZIQqewQ==",
- "dev": true
- },
- "ini": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/ini/-/ini-2.0.0.tgz",
- "integrity": "sha512-7PnF4oN3CvZF23ADhA5wRaYEQpJ8qygSkbtTXWBeXWXmEVRXK+1ITciHWwHhsjv1TmW0MgacIv6hEi5pX5NQdA==",
- "dev": true
- },
- "is-ci": {
- "version": "3.0.1",
- "resolved": "https://registry.npmjs.org/is-ci/-/is-ci-3.0.1.tgz",
- "integrity": "sha512-ZYvCgrefwqoQ6yTyYUbQu64HsITZ3NfKX1lzaEYdkTDcfKzzCI/wthRRYKkdjHKFVgNiXKAKm65Zo1pk2as/QQ==",
- "dev": true,
- "requires": {
- "ci-info": "^3.2.0"
- }
- },
- "is-fullwidth-code-point": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/is-fullwidth-code-point/-/is-fullwidth-code-point-3.0.0.tgz",
- "integrity": "sha512-zymm5+u+sCsSWyD9qNaejV3DFvhCKclKdizYaJUuHA83RLjb7nSuGnddCHGv0hk+KY7BMAlsWeK4Ueg6EV6XQg==",
- "dev": true
- },
- "is-installed-globally": {
- "version": "0.4.0",
- "resolved": "https://registry.npmjs.org/is-installed-globally/-/is-installed-globally-0.4.0.tgz",
- "integrity": "sha512-iwGqO3J21aaSkC7jWnHP/difazwS7SFeIqxv6wEtLU8Y5KlzFTjyqcSIT0d8s4+dDhKytsk9PJZ2BkS5eZwQRQ==",
- "dev": true,
- "requires": {
- "global-dirs": "^3.0.0",
- "is-path-inside": "^3.0.2"
- }
- },
- "is-path-inside": {
- "version": "3.0.3",
- "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-3.0.3.tgz",
- "integrity": "sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==",
- "dev": true
- },
- "is-stream": {
- "version": "2.0.1",
- "resolved": "https://registry.npmjs.org/is-stream/-/is-stream-2.0.1.tgz",
- "integrity": "sha512-hFoiJiTl63nn+kstHGBtewWSKnQLpyb155KHheA1l39uvtO9nWIop1p3udqPcUd/xbF1VLMO4n7OI6p7RbngDg==",
- "dev": true
- },
- "is-typedarray": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz",
- "integrity": "sha512-cyA56iCMHAh5CdzjJIa4aohJyeO1YbwLi3Jc35MmRU6poroFjIGZzUzupGiRPOjgHg9TLu43xbpwXk523fMxKA==",
- "dev": true
- },
- "is-unicode-supported": {
- "version": "0.1.0",
- "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz",
- "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==",
- "dev": true
- },
- "isexe": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/isexe/-/isexe-2.0.0.tgz",
- "integrity": "sha512-RHxMLp9lnKHGHRng9QFhRCMbYAcVpn69smSGcq3f36xjgVVWThj4qqLbTLlq7Ssj8B+fIQ1EuCEGI2lKsyQeIw==",
- "dev": true
- },
- "isstream": {
- "version": "0.1.2",
- "resolved": "https://registry.npmjs.org/isstream/-/isstream-0.1.2.tgz",
- "integrity": "sha512-Yljz7ffyPbrLpLngrMtZ7NduUgVvi6wG9RJ9IUcyCd59YQ911PBJphODUcbOVbqYfxe1wuYf/LJ8PauMRwsM/g==",
- "dev": true
- },
- "jsbn": {
- "version": "0.1.1",
- "resolved": "https://registry.npmjs.org/jsbn/-/jsbn-0.1.1.tgz",
- "integrity": "sha512-UVU9dibq2JcFWxQPA6KCqj5O42VOmAY3zQUfEKxU0KpTGXwNoCjkX1e13eHNvw/xPynt6pU0rZ1htjWTNTSXsg==",
- "dev": true
- },
- "json-schema": {
- "version": "0.4.0",
- "resolved": "https://registry.npmjs.org/json-schema/-/json-schema-0.4.0.tgz",
- "integrity": "sha512-es94M3nTIfsEPisRafak+HDLfHXnKBhV3vU5eqPcS3flIWqcxJWgXHXiey3YrpaNsanY5ei1VoYEbOzijuq9BA==",
- "dev": true
- },
- "json-stringify-safe": {
- "version": "5.0.1",
- "resolved": "https://registry.npmjs.org/json-stringify-safe/-/json-stringify-safe-5.0.1.tgz",
- "integrity": "sha512-ZClg6AaYvamvYEE82d3Iyd3vSSIjQ+odgjaTzRuO3s7toCdFKczob2i0zCh7JE8kWn17yvAWhUVxvqGwUalsRA==",
- "dev": true
- },
- "jsonfile": {
- "version": "6.1.0",
- "resolved": "https://registry.npmjs.org/jsonfile/-/jsonfile-6.1.0.tgz",
- "integrity": "sha512-5dgndWOriYSm5cnYaJNhalLNDKOqFwyDB/rr1E9ZsGciGvKPs8R2xYGCacuf3z6K1YKDz182fd+fY3cn3pMqXQ==",
- "dev": true,
- "requires": {
- "graceful-fs": "^4.1.6",
- "universalify": "^2.0.0"
- },
- "dependencies": {
- "universalify": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/universalify/-/universalify-2.0.0.tgz",
- "integrity": "sha512-hAZsKq7Yy11Zu1DE0OzWjw7nnLZmJZYTDZZyEFHZdUhV8FkH5MCfoU1XMaxXovpyW5nq5scPqq0ZDP9Zyl04oQ==",
- "dev": true
- }
- }
- },
- "jsprim": {
- "version": "2.0.2",
- "resolved": "https://registry.npmjs.org/jsprim/-/jsprim-2.0.2.tgz",
- "integrity": "sha512-gqXddjPqQ6G40VdnI6T6yObEC+pDNvyP95wdQhkWkg7crHH3km5qP1FsOXEkzEQwnz6gz5qGTn1c2Y52wP3OyQ==",
- "dev": true,
- "requires": {
- "assert-plus": "1.0.0",
- "extsprintf": "1.3.0",
- "json-schema": "0.4.0",
- "verror": "1.10.0"
- }
- },
- "lazy-ass": {
- "version": "1.6.0",
- "resolved": "https://registry.npmjs.org/lazy-ass/-/lazy-ass-1.6.0.tgz",
- "integrity": "sha512-cc8oEVoctTvsFZ/Oje/kGnHbpWHYBe8IAJe4C0QNc3t8uM/0Y8+erSz/7Y1ALuXTEZTMvxXwO6YbX1ey3ujiZw==",
- "dev": true
- },
- "listr2": {
- "version": "3.14.0",
- "resolved": "https://registry.npmjs.org/listr2/-/listr2-3.14.0.tgz",
- "integrity": "sha512-TyWI8G99GX9GjE54cJ+RrNMcIFBfwMPxc3XTFiAYGN4s10hWROGtOg7+O6u6LE3mNkyld7RSLE6nrKBvTfcs3g==",
- "dev": true,
- "requires": {
- "cli-truncate": "^2.1.0",
- "colorette": "^2.0.16",
- "log-update": "^4.0.0",
- "p-map": "^4.0.0",
- "rfdc": "^1.3.0",
- "rxjs": "^7.5.1",
- "through": "^2.3.8",
- "wrap-ansi": "^7.0.0"
- }
- },
- "lodash": {
- "version": "4.17.21",
- "resolved": "https://registry.npmjs.org/lodash/-/lodash-4.17.21.tgz",
- "integrity": "sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==",
- "dev": true
- },
- "lodash.once": {
- "version": "4.1.1",
- "resolved": "https://registry.npmjs.org/lodash.once/-/lodash.once-4.1.1.tgz",
- "integrity": "sha512-Sb487aTOCr9drQVL8pIxOzVhafOjZN9UU54hiN8PU3uAiSV7lx1yYNpbNmex2PK6dSJoNTSJUUswT651yww3Mg==",
- "dev": true
- },
- "log-symbols": {
- "version": "4.1.0",
- "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz",
- "integrity": "sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==",
- "dev": true,
- "requires": {
- "chalk": "^4.1.0",
- "is-unicode-supported": "^0.1.0"
- }
- },
- "log-update": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/log-update/-/log-update-4.0.0.tgz",
- "integrity": "sha512-9fkkDevMefjg0mmzWFBW8YkFP91OrizzkW3diF7CpG+S2EYdy4+TVfGwz1zeF8x7hCx1ovSPTOE9Ngib74qqUg==",
- "dev": true,
- "requires": {
- "ansi-escapes": "^4.3.0",
- "cli-cursor": "^3.1.0",
- "slice-ansi": "^4.0.0",
- "wrap-ansi": "^6.2.0"
- },
- "dependencies": {
- "slice-ansi": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/slice-ansi/-/slice-ansi-4.0.0.tgz",
- "integrity": "sha512-qMCMfhY040cVHT43K9BFygqYbUPFZKHOg7K73mtTWJRb8pyP3fzf4Ixd5SzdEJQ6MRUg/WBnOLxghZtKKurENQ==",
- "dev": true,
- "requires": {
- "ansi-styles": "^4.0.0",
- "astral-regex": "^2.0.0",
- "is-fullwidth-code-point": "^3.0.0"
- }
- },
- "wrap-ansi": {
- "version": "6.2.0",
- "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-6.2.0.tgz",
- "integrity": "sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA==",
- "dev": true,
- "requires": {
- "ansi-styles": "^4.0.0",
- "string-width": "^4.1.0",
- "strip-ansi": "^6.0.0"
- }
- }
- }
- },
- "lru-cache": {
- "version": "6.0.0",
- "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-6.0.0.tgz",
- "integrity": "sha512-Jo6dJ04CmSjuznwJSS3pUeWmd/H0ffTlkXXgwZi+eq1UCmqQwCh+eLsYOYCwY991i2Fah4h1BEMCx4qThGbsiA==",
- "dev": true,
- "requires": {
- "yallist": "^4.0.0"
- }
- },
- "merge-stream": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/merge-stream/-/merge-stream-2.0.0.tgz",
- "integrity": "sha512-abv/qOcuPfk3URPfDzmZU1LKmuw8kT+0nIHvKrKgFrwifol/doWcdA4ZqsWQ8ENrFKkd67Mfpo/LovbIUsbt3w==",
- "dev": true
- },
- "mime-db": {
- "version": "1.52.0",
- "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.52.0.tgz",
- "integrity": "sha512-sPU4uV7dYlvtWJxwwxHD0PuihVNiE7TyAbQ5SWxDCB9mUYvOgroQOwYQQOKPJ8CIbE+1ETVlOoK1UC2nU3gYvg==",
- "dev": true
- },
- "mime-types": {
- "version": "2.1.35",
- "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.35.tgz",
- "integrity": "sha512-ZDY+bPm5zTTF+YpCrAU9nK0UgICYPT0QtT1NZWFv4s++TNkcgVaT0g6+4R2uI4MjQjzysHB1zxuWL50hzaeXiw==",
- "dev": true,
- "requires": {
- "mime-db": "1.52.0"
- }
- },
- "mimic-fn": {
- "version": "2.1.0",
- "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-2.1.0.tgz",
- "integrity": "sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==",
- "dev": true
- },
- "minimatch": {
- "version": "3.1.2",
- "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-3.1.2.tgz",
- "integrity": "sha512-J7p63hRiAjw1NDEww1W7i37+ByIrOWO5XQQAzZ3VOcL0PNybwpfmV/N05zFAzwQ9USyEcX6t3UO+K5aqBQOIHw==",
- "dev": true,
- "requires": {
- "brace-expansion": "^1.1.7"
- }
- },
- "minimist": {
- "version": "1.2.8",
- "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.8.tgz",
- "integrity": "sha512-2yyAR8qBkN3YuheJanUpWC5U3bb5osDywNB8RzDVlDwDHbocAJveqqj1u8+SVD7jkWT4yvsHCpWqqWqAxb0zCA==",
- "dev": true
- },
- "ms": {
- "version": "2.1.3",
- "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz",
- "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==",
- "dev": true
- },
- "npm-run-path": {
- "version": "4.0.1",
- "resolved": "https://registry.npmjs.org/npm-run-path/-/npm-run-path-4.0.1.tgz",
- "integrity": "sha512-S48WzZW777zhNIrn7gxOlISNAqi9ZC/uQFnRdbeIHhZhCA6UqpkOT8T1G7BvfdgP4Er8gF4sUbaS0i7QvIfCWw==",
- "dev": true,
- "requires": {
- "path-key": "^3.0.0"
- }
- },
- "object-inspect": {
- "version": "1.12.3",
- "resolved": "https://registry.npmjs.org/object-inspect/-/object-inspect-1.12.3.tgz",
- "integrity": "sha512-geUvdk7c+eizMNUDkRpW1wJwgfOiOeHbxBR/hLXK1aT6zmVSO0jsQcs7fj6MGw89jC/cjGfLcNOrtMYtGqm81g==",
- "dev": true
- },
- "once": {
- "version": "1.4.0",
- "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz",
- "integrity": "sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==",
- "dev": true,
- "requires": {
- "wrappy": "1"
- }
- },
- "onetime": {
- "version": "5.1.2",
- "resolved": "https://registry.npmjs.org/onetime/-/onetime-5.1.2.tgz",
- "integrity": "sha512-kbpaSSGJTWdAY5KPVeMOKXSrPtr8C8C7wodJbcsd51jRnmD+GZu8Y0VoU6Dm5Z4vWr0Ig/1NKuWRKf7j5aaYSg==",
- "dev": true,
- "requires": {
- "mimic-fn": "^2.1.0"
- }
- },
- "ospath": {
- "version": "1.2.2",
- "resolved": "https://registry.npmjs.org/ospath/-/ospath-1.2.2.tgz",
- "integrity": "sha512-o6E5qJV5zkAbIDNhGSIlyOhScKXgQrSRMilfph0clDfM0nEnBOlKlH4sWDmG95BW/CvwNz0vmm7dJVtU2KlMiA==",
- "dev": true
- },
- "p-map": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/p-map/-/p-map-4.0.0.tgz",
- "integrity": "sha512-/bjOqmgETBYB5BoEeGVea8dmvHb2m9GLy1E9W43yeyfP6QQCZGFNa+XRceJEuDB6zqr+gKpIAmlLebMpykw/MQ==",
- "dev": true,
- "requires": {
- "aggregate-error": "^3.0.0"
- }
- },
- "path-is-absolute": {
- "version": "1.0.1",
- "resolved": "https://registry.npmjs.org/path-is-absolute/-/path-is-absolute-1.0.1.tgz",
- "integrity": "sha512-AVbw3UJ2e9bq64vSaS9Am0fje1Pa8pbGqTTsmXfaIiMpnr5DlDhfJOuLj9Sf95ZPVDAUerDfEk88MPmPe7UCQg==",
- "dev": true
- },
- "path-key": {
- "version": "3.1.1",
- "resolved": "https://registry.npmjs.org/path-key/-/path-key-3.1.1.tgz",
- "integrity": "sha512-ojmeN0qd+y0jszEtoY48r0Peq5dwMEkIlCOu6Q5f41lfkswXuKtYrhgoTpLnyIcHm24Uhqx+5Tqm2InSwLhE6Q==",
- "dev": true
- },
- "pend": {
- "version": "1.2.0",
- "resolved": "https://registry.npmjs.org/pend/-/pend-1.2.0.tgz",
- "integrity": "sha512-F3asv42UuXchdzt+xXqfW1OGlVBe+mxa2mqI0pg5yAHZPvFmY3Y6drSf/GQ1A86WgWEN9Kzh/WrgKa6iGcHXLg==",
- "dev": true
- },
- "performance-now": {
- "version": "2.1.0",
- "resolved": "https://registry.npmjs.org/performance-now/-/performance-now-2.1.0.tgz",
- "integrity": "sha512-7EAHlyLHI56VEIdK57uwHdHKIaAGbnXPiw0yWbarQZOKaKpvUIgW0jWRVLiatnM+XXlSwsanIBH/hzGMJulMow==",
- "dev": true
- },
- "pify": {
- "version": "2.3.0",
- "resolved": "https://registry.npmjs.org/pify/-/pify-2.3.0.tgz",
- "integrity": "sha512-udgsAY+fTnvv7kI7aaxbqwWNb0AHiB0qBO89PZKPkoTmGOgdbrHDKD+0B2X4uTfJ/FT1R09r9gTsjUjNJotuog==",
- "dev": true
- },
- "pretty-bytes": {
- "version": "5.6.0",
- "resolved": "https://registry.npmjs.org/pretty-bytes/-/pretty-bytes-5.6.0.tgz",
- "integrity": "sha512-FFw039TmrBqFK8ma/7OL3sDz/VytdtJr044/QUJtH0wK9lb9jLq9tJyIxUwtQJHwar2BqtiA4iCWSwo9JLkzFg==",
- "dev": true
- },
- "process": {
- "version": "0.11.10",
- "resolved": "https://registry.npmjs.org/process/-/process-0.11.10.tgz",
- "integrity": "sha512-cdGef/drWFoydD1JsMzuFf8100nZl+GT+yacc2bEced5f9Rjk4z+WtFUTBu9PhOi9j/jfmBPu0mMEY4wIdAF8A==",
- "dev": true
- },
- "proxy-from-env": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/proxy-from-env/-/proxy-from-env-1.0.0.tgz",
- "integrity": "sha512-F2JHgJQ1iqwnHDcQjVBsq3n/uoaFL+iPW/eAeL7kVxy/2RrWaN4WroKjjvbsoRtv0ftelNyC01bjRhn/bhcf4A==",
- "dev": true
- },
- "psl": {
- "version": "1.9.0",
- "resolved": "https://registry.npmjs.org/psl/-/psl-1.9.0.tgz",
- "integrity": "sha512-E/ZsdU4HLs/68gYzgGTkMicWTLPdAftJLfJFlLUAAKZGkStNU72sZjT66SnMDVOfOWY/YAoiD7Jxa9iHvngcag==",
- "dev": true
- },
- "pump": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/pump/-/pump-3.0.0.tgz",
- "integrity": "sha512-LwZy+p3SFs1Pytd/jYct4wpv49HiYCqd9Rlc5ZVdk0V+8Yzv6jR5Blk3TRmPL1ft69TxP0IMZGJ+WPFU2BFhww==",
- "dev": true,
- "requires": {
- "end-of-stream": "^1.1.0",
- "once": "^1.3.1"
- }
- },
- "punycode": {
- "version": "2.3.0",
- "resolved": "https://registry.npmjs.org/punycode/-/punycode-2.3.0.tgz",
- "integrity": "sha512-rRV+zQD8tVFys26lAGR9WUuS4iUAngJScM+ZRSKtvl5tKeZ2t5bvdNFdNHBW9FWR4guGHlgmsZ1G7BSm2wTbuA==",
- "dev": true
- },
- "qs": {
- "version": "6.10.4",
- "resolved": "https://registry.npmjs.org/qs/-/qs-6.10.4.tgz",
- "integrity": "sha512-OQiU+C+Ds5qiH91qh/mg0w+8nwQuLjM4F4M/PbmhDOoYehPh+Fb0bDjtR1sOvy7YKxvj28Y/M0PhP5uVX0kB+g==",
- "dev": true,
- "requires": {
- "side-channel": "^1.0.4"
- }
- },
- "querystringify": {
- "version": "2.2.0",
- "resolved": "https://registry.npmjs.org/querystringify/-/querystringify-2.2.0.tgz",
- "integrity": "sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ==",
- "dev": true
- },
- "request-progress": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/request-progress/-/request-progress-3.0.0.tgz",
- "integrity": "sha512-MnWzEHHaxHO2iWiQuHrUPBi/1WeBf5PkxQqNyNvLl9VAYSdXkP8tQ3pBSeCPD+yw0v0Aq1zosWLz0BdeXpWwZg==",
- "dev": true,
- "requires": {
- "throttleit": "^1.0.0"
- }
- },
- "requires-port": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/requires-port/-/requires-port-1.0.0.tgz",
- "integrity": "sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ==",
- "dev": true
- },
- "restore-cursor": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/restore-cursor/-/restore-cursor-3.1.0.tgz",
- "integrity": "sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA==",
- "dev": true,
- "requires": {
- "onetime": "^5.1.0",
- "signal-exit": "^3.0.2"
- }
- },
- "rfdc": {
- "version": "1.3.0",
- "resolved": "https://registry.npmjs.org/rfdc/-/rfdc-1.3.0.tgz",
- "integrity": "sha512-V2hovdzFbOi77/WajaSMXk2OLm+xNIeQdMMuB7icj7bk6zi2F8GGAxigcnDFpJHbNyNcgyJDiP+8nOrY5cZGrA==",
- "dev": true
- },
- "rimraf": {
- "version": "3.0.2",
- "resolved": "https://registry.npmjs.org/rimraf/-/rimraf-3.0.2.tgz",
- "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==",
- "dev": true,
- "requires": {
- "glob": "^7.1.3"
- }
- },
- "rxjs": {
- "version": "7.8.1",
- "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-7.8.1.tgz",
- "integrity": "sha512-AA3TVj+0A2iuIoQkWEK/tqFjBq2j+6PO6Y0zJcvzLAFhEFIO3HL0vls9hWLncZbAAbK0mar7oZ4V079I/qPMxg==",
- "dev": true,
- "requires": {
- "tslib": "^2.1.0"
- }
- },
- "safe-buffer": {
- "version": "5.2.1",
- "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz",
- "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==",
- "dev": true
- },
- "safer-buffer": {
- "version": "2.1.2",
- "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz",
- "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==",
- "dev": true
- },
- "semver": {
- "version": "7.5.4",
- "resolved": "https://registry.npmjs.org/semver/-/semver-7.5.4.tgz",
- "integrity": "sha512-1bCSESV6Pv+i21Hvpxp3Dx+pSD8lIPt8uVjRrxAUt/nbswYc+tK6Y2btiULjd4+fnq15PX+nqQDC7Oft7WkwcA==",
- "dev": true,
- "requires": {
- "lru-cache": "^6.0.0"
- }
- },
- "shebang-command": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/shebang-command/-/shebang-command-2.0.0.tgz",
- "integrity": "sha512-kHxr2zZpYtdmrN1qDjrrX/Z1rR1kG8Dx+gkpK1G4eXmvXswmcE1hTWBWYUzlraYw1/yZp6YuDY77YtvbN0dmDA==",
- "dev": true,
- "requires": {
- "shebang-regex": "^3.0.0"
- }
- },
- "shebang-regex": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/shebang-regex/-/shebang-regex-3.0.0.tgz",
- "integrity": "sha512-7++dFhtcx3353uBaq8DDR4NuxBetBzC7ZQOhmTQInHEd6bSrXdiEyzCvG07Z44UYdLShWUyXt5M/yhz8ekcb1A==",
- "dev": true
- },
- "side-channel": {
- "version": "1.0.4",
- "resolved": "https://registry.npmjs.org/side-channel/-/side-channel-1.0.4.tgz",
- "integrity": "sha512-q5XPytqFEIKHkGdiMIrY10mvLRvnQh42/+GoBlFW3b2LXLE2xxJpZFdm94we0BaoV3RwJyGqg5wS7epxTv0Zvw==",
- "dev": true,
- "requires": {
- "call-bind": "^1.0.0",
- "get-intrinsic": "^1.0.2",
- "object-inspect": "^1.9.0"
- }
- },
- "signal-exit": {
- "version": "3.0.7",
- "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-3.0.7.tgz",
- "integrity": "sha512-wnD2ZE+l+SPC/uoS0vXeE9L1+0wuaMqKlfz9AMUo38JsyLSBWSFcHR1Rri62LZc12vLr1gb3jl7iwQhgwpAbGQ==",
- "dev": true
- },
- "slice-ansi": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/slice-ansi/-/slice-ansi-3.0.0.tgz",
- "integrity": "sha512-pSyv7bSTC7ig9Dcgbw9AuRNUb5k5V6oDudjZoMBSr13qpLBG7tB+zgCkARjq7xIUgdz5P1Qe8u+rSGdouOOIyQ==",
- "dev": true,
- "requires": {
- "ansi-styles": "^4.0.0",
- "astral-regex": "^2.0.0",
- "is-fullwidth-code-point": "^3.0.0"
- }
- },
- "sshpk": {
- "version": "1.17.0",
- "resolved": "https://registry.npmjs.org/sshpk/-/sshpk-1.17.0.tgz",
- "integrity": "sha512-/9HIEs1ZXGhSPE8X6Ccm7Nam1z8KcoCqPdI7ecm1N33EzAetWahvQWVqLZtaZQ+IDKX4IyA2o0gBzqIMkAagHQ==",
- "dev": true,
- "requires": {
- "asn1": "~0.2.3",
- "assert-plus": "^1.0.0",
- "bcrypt-pbkdf": "^1.0.0",
- "dashdash": "^1.12.0",
- "ecc-jsbn": "~0.1.1",
- "getpass": "^0.1.1",
- "jsbn": "~0.1.0",
- "safer-buffer": "^2.0.2",
- "tweetnacl": "~0.14.0"
- }
- },
- "string-width": {
- "version": "4.2.3",
- "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz",
- "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==",
- "dev": true,
- "requires": {
- "emoji-regex": "^8.0.0",
- "is-fullwidth-code-point": "^3.0.0",
- "strip-ansi": "^6.0.1"
- }
- },
- "strip-ansi": {
- "version": "6.0.1",
- "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz",
- "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==",
- "dev": true,
- "requires": {
- "ansi-regex": "^5.0.1"
- }
- },
- "strip-final-newline": {
- "version": "2.0.0",
- "resolved": "https://registry.npmjs.org/strip-final-newline/-/strip-final-newline-2.0.0.tgz",
- "integrity": "sha512-BrpvfNAE3dcvq7ll3xVumzjKjZQ5tI1sEUIKr3Uoks0XUl45St3FlatVqef9prk4jRDzhW6WZg+3bk93y6pLjA==",
- "dev": true
- },
- "supports-color": {
- "version": "8.1.1",
- "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-8.1.1.tgz",
- "integrity": "sha512-MpUEN2OodtUzxvKQl72cUF7RQ5EiHsGvSsVG0ia9c5RbWGL2CI4C7EpPS8UTBIplnlzZiNuV56w+FuNxy3ty2Q==",
- "dev": true,
- "requires": {
- "has-flag": "^4.0.0"
- }
- },
- "throttleit": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/throttleit/-/throttleit-1.0.0.tgz",
- "integrity": "sha512-rkTVqu6IjfQ/6+uNuuc3sZek4CEYxTJom3IktzgdSxcZqdARuebbA/f4QmAxMQIxqq9ZLEUkSYqvuk1I6VKq4g==",
- "dev": true
- },
- "through": {
- "version": "2.3.8",
- "resolved": "https://registry.npmjs.org/through/-/through-2.3.8.tgz",
- "integrity": "sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg==",
- "dev": true
- },
- "tmp": {
- "version": "0.2.1",
- "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.2.1.tgz",
- "integrity": "sha512-76SUhtfqR2Ijn+xllcI5P1oyannHNHByD80W1q447gU3mp9G9PSpGdWmjUOHRDPiHYacIk66W7ubDTuPF3BEtQ==",
- "dev": true,
- "requires": {
- "rimraf": "^3.0.0"
- }
- },
- "tough-cookie": {
- "version": "4.1.3",
- "resolved": "https://registry.npmjs.org/tough-cookie/-/tough-cookie-4.1.3.tgz",
- "integrity": "sha512-aX/y5pVRkfRnfmuX+OdbSdXvPe6ieKX/G2s7e98f4poJHnqH3281gDPm/metm6E/WRamfx7WC4HUqkWHfQHprw==",
- "dev": true,
- "requires": {
- "psl": "^1.1.33",
- "punycode": "^2.1.1",
- "universalify": "^0.2.0",
- "url-parse": "^1.5.3"
- }
- },
- "tslib": {
- "version": "2.6.2",
- "resolved": "https://registry.npmjs.org/tslib/-/tslib-2.6.2.tgz",
- "integrity": "sha512-AEYxH93jGFPn/a2iVAwW87VuUIkR1FVUKB77NwMF7nBTDkDrrT/Hpt/IrCJ0QXhW27jTBDcf5ZY7w6RiqTMw2Q==",
- "dev": true
- },
- "tunnel-agent": {
- "version": "0.6.0",
- "resolved": "https://registry.npmjs.org/tunnel-agent/-/tunnel-agent-0.6.0.tgz",
- "integrity": "sha512-McnNiV1l8RYeY8tBgEpuodCC1mLUdbSN+CYBL7kJsJNInOP8UjDDEwdk6Mw60vdLLrr5NHKZhMAOSrR2NZuQ+w==",
- "dev": true,
- "requires": {
- "safe-buffer": "^5.0.1"
- }
- },
- "tweetnacl": {
- "version": "0.14.5",
- "resolved": "https://registry.npmjs.org/tweetnacl/-/tweetnacl-0.14.5.tgz",
- "integrity": "sha512-KXXFFdAbFXY4geFIwoyNK+f5Z1b7swfXABfL7HXCmoIWMKU3dmS26672A4EeQtDzLKy7SXmfBu51JolvEKwtGA==",
- "dev": true
- },
- "type-fest": {
- "version": "0.21.3",
- "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz",
- "integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==",
- "dev": true
- },
- "universalify": {
- "version": "0.2.0",
- "resolved": "https://registry.npmjs.org/universalify/-/universalify-0.2.0.tgz",
- "integrity": "sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg==",
- "dev": true
- },
- "untildify": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/untildify/-/untildify-4.0.0.tgz",
- "integrity": "sha512-KK8xQ1mkzZeg9inewmFVDNkg3l5LUhoq9kN6iWYB/CC9YMG8HA+c1Q8HwDe6dEX7kErrEVNVBO3fWsVq5iDgtw==",
- "dev": true
- },
- "url-parse": {
- "version": "1.5.10",
- "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.5.10.tgz",
- "integrity": "sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ==",
- "dev": true,
- "requires": {
- "querystringify": "^2.1.1",
- "requires-port": "^1.0.0"
- }
- },
- "uuid": {
- "version": "8.3.2",
- "resolved": "https://registry.npmjs.org/uuid/-/uuid-8.3.2.tgz",
- "integrity": "sha512-+NYs2QeMWy+GWFOEm9xnn6HCDp0l7QBD7ml8zLUmJ+93Q5NF0NocErnwkTkXVFNiX3/fpC6afS8Dhb/gz7R7eg==",
- "dev": true
- },
- "verror": {
- "version": "1.10.0",
- "resolved": "https://registry.npmjs.org/verror/-/verror-1.10.0.tgz",
- "integrity": "sha512-ZZKSmDAEFOijERBLkmYfJ+vmk3w+7hOLYDNkRCuRuMJGEmqYNCNLyBBFwWKVMhfwaEF3WOd0Zlw86U/WC/+nYw==",
- "dev": true,
- "requires": {
- "assert-plus": "^1.0.0",
- "core-util-is": "1.0.2",
- "extsprintf": "^1.2.0"
- }
- },
- "which": {
- "version": "2.0.2",
- "resolved": "https://registry.npmjs.org/which/-/which-2.0.2.tgz",
- "integrity": "sha512-BLI3Tl1TW3Pvl70l3yq3Y64i+awpwXqsGBYWkkqMtnbXgrMD+yj7rhW0kuEDxzJaYXGjEW5ogapKNMEKNMjibA==",
- "dev": true,
- "requires": {
- "isexe": "^2.0.0"
- }
- },
- "wrap-ansi": {
- "version": "7.0.0",
- "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz",
- "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==",
- "dev": true,
- "requires": {
- "ansi-styles": "^4.0.0",
- "string-width": "^4.1.0",
- "strip-ansi": "^6.0.0"
- }
- },
- "wrappy": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz",
- "integrity": "sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==",
- "dev": true
- },
- "yallist": {
- "version": "4.0.0",
- "resolved": "https://registry.npmjs.org/yallist/-/yallist-4.0.0.tgz",
- "integrity": "sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==",
- "dev": true
- },
- "yauzl": {
- "version": "2.10.0",
- "resolved": "https://registry.npmjs.org/yauzl/-/yauzl-2.10.0.tgz",
- "integrity": "sha512-p4a9I6X6nu6IhoGmBqAcbJy1mlC4j27vEPZX9F4L4/vZT3Lyq1VkFHw/V/PUcB9Buo+DG3iHkT0x3Qya58zc3g==",
- "dev": true,
- "requires": {
- "buffer-crc32": "~0.2.3",
- "fd-slicer": "~1.1.0"
- }
- }
- }
-}
diff --git a/vitess-mixin/e2e/package.json b/vitess-mixin/e2e/package.json
deleted file mode 100644
index ccc95ba93b0..00000000000
--- a/vitess-mixin/e2e/package.json
+++ /dev/null
@@ -1,6 +0,0 @@
-{
- "license": "Apache-2.0",
- "devDependencies": {
- "cypress": "^13.2.0"
- }
-}
diff --git a/vitess-mixin/e2e/prometheus/prometheus.yml b/vitess-mixin/e2e/prometheus/prometheus.yml
deleted file mode 100644
index bd8fe98dfa7..00000000000
--- a/vitess-mixin/e2e/prometheus/prometheus.yml
+++ /dev/null
@@ -1,61 +0,0 @@
-global:
- scrape_interval: 5s
- scrape_timeout: 2s
-
-rule_files:
- - prometheus_rules.yaml
-
-scrape_configs:
- - job_name: vitess-vtctld
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'vtctld:8080'
- - job_name: vitess-vtgate
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'vtgate:8080'
- - job_name: vitess-vttablet
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'vttablet101:8080'
- - 'vttablet102:8080'
- - 'vttablet201:8080'
- - 'vttablet202:8080'
- - 'vttablet301:8080'
- - 'vttablet302:8080'
- # Mock mysql exporter for vttablet
- - job_name: mysql
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'mysqld_exporter:9104'
- metric_relabel_configs:
- - source_labels: [ instance ]
- target_label: instance
- action: replace
- replacement: vttablet101:8080
- # Mock node exporter for vtgate
- - job_name: node-exporter-vitess-vtgate
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'node-exporter:9100'
- metric_relabel_configs:
- - source_labels: [ instance ]
- target_label: instance
- action: replace
- replacement: vtgate:8080
- # Mock node exporter for vttablet
- - job_name: node-exporter-vitess-vttablet
- metrics_path: /metrics
- static_configs:
- - targets:
- - 'node-exporter:9100'
- metric_relabel_configs:
- - source_labels: [ instance ]
- target_label: instance
- action: replace
- replacement: vttablet101:8080
diff --git a/vitess-mixin/e2e/schemaload.sh b/vitess-mixin/e2e/schemaload.sh
deleted file mode 100755
index b8f13ee8812..00000000000
--- a/vitess-mixin/e2e/schemaload.sh
+++ /dev/null
@@ -1,56 +0,0 @@
-#!/bin/bash -e
-
-# Copyright 2020 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.
-
-sleeptime=${SLEEPTIME:-0}
-targettab=${TARGETTAB:-"${CELL}-0000000101"}
-schema_files=${SCHEMA_FILES:-'create_messages.sql create_tokens.sql create'}
-vschema_file=${VSCHEMA_FILE:-'default_vschema.json'}
-load_file=${POST_LOAD_FILE:-''}
-external_db=${EXTERNAL_DB:-'0'}
-export PATH=/vt/bin:$PATH
-
-sleep $sleeptime
-
-if [ ! -f schema_run ]; then
- while true; do
- vtctlclient -server vtctld:$GRPC_PORT GetTablet $targettab && break
- sleep 1
- done
- if [ "$external_db" = "0" ]; then
- for schema_file in $schema_files; do
- echo "Applying Schema ${schema_file} to ${KEYSPACE}"
- vtctlclient -server vtctld:$GRPC_PORT ApplySchema -sql-file /script/tables/${schema_file} $KEYSPACE || \
- vtctlclient -server vtctld:$GRPC_PORT ApplySchema -sql "$(cat /script/tables/${schema_file})" $KEYSPACE || true
- done
- fi
- echo "Applying VSchema ${vschema_file} to ${KEYSPACE}"
-
- vtctlclient -server vtctld:$GRPC_PORT ApplyVSchema -vschema_file /script/${vschema_file} $KEYSPACE || \
- vtctlclient -server vtctld:$GRPC_PORT ApplyVSchema -vschema "$(cat /script/${vschema_file})" $KEYSPACE
-
- echo "List All Tablets"
- vtctlclient -server vtctld:$GRPC_PORT ListAllTablets
-
- if [ -n "$load_file" ]; then
- # vtgate can take a REALLY long time to come up fully
- sleep 60
- mysql --port=15306 --host=vtgate < /script/$load_file
- fi
-
- touch /vt/schema_run
- echo "Time: $(date). SchemaLoad completed at $(date "+%FT%T") " >> /vt/schema_run
- echo "Done Loading Schema at $(date "+%FT%T")"
-fi
diff --git a/vitess-mixin/e2e/tables/create_dinosaurs.sql b/vitess-mixin/e2e/tables/create_dinosaurs.sql
deleted file mode 100644
index ee9a5b624da..00000000000
--- a/vitess-mixin/e2e/tables/create_dinosaurs.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-CREATE TABLE dinosaurs (
- id BIGINT UNSIGNED,
- time_created_ns BIGINT UNSIGNED,
- name VARCHAR(255),
- PRIMARY KEY (id)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/create_eggs.sql b/vitess-mixin/e2e/tables/create_eggs.sql
deleted file mode 100644
index 7e74c50b5a1..00000000000
--- a/vitess-mixin/e2e/tables/create_eggs.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-CREATE TABLE eggs (
- id BIGINT UNSIGNED,
- time_created_ns BIGINT UNSIGNED,
- species VARCHAR(255),
- PRIMARY KEY (id)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/create_messages.sql b/vitess-mixin/e2e/tables/create_messages.sql
deleted file mode 100644
index 44c4cc16e87..00000000000
--- a/vitess-mixin/e2e/tables/create_messages.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-CREATE TABLE messages (
- page BIGINT(20) UNSIGNED,
- time_created_ns BIGINT(20) UNSIGNED,
- message VARCHAR(10000),
- PRIMARY KEY (page, time_created_ns)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/create_messages_message_lookup.sql b/vitess-mixin/e2e/tables/create_messages_message_lookup.sql
deleted file mode 100644
index 1eb667dac93..00000000000
--- a/vitess-mixin/e2e/tables/create_messages_message_lookup.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-CREATE TABLE messages_message_lookup (
- id BIGINT NOT NULL AUTO_INCREMENT,
- page BIGINT UNSIGNED,
- message VARCHAR(1000),
- PRIMARY KEY (id),
- UNIQUE KEY idx_message_page (`message`, `page`)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/create_tokens.sql b/vitess-mixin/e2e/tables/create_tokens.sql
deleted file mode 100644
index 69f1be03dec..00000000000
--- a/vitess-mixin/e2e/tables/create_tokens.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-CREATE TABLE tokens (
- page BIGINT(20) UNSIGNED,
- time_created_ns BIGINT(20) UNSIGNED,
- token VARCHAR(255) DEFAULT NULL,
- PRIMARY KEY (page, time_created_ns)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/create_tokens_token_lookup.sql b/vitess-mixin/e2e/tables/create_tokens_token_lookup.sql
deleted file mode 100644
index d3208fdfc41..00000000000
--- a/vitess-mixin/e2e/tables/create_tokens_token_lookup.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-CREATE TABLE tokens_token_lookup (
- id BIGINT NOT NULL AUTO_INCREMENT,
- page BIGINT UNSIGNED,
- token VARCHAR(255) DEFAULT NULL,
- PRIMARY KEY (id),
- UNIQUE KEY idx_token_page (`token`, `page`)
-) ENGINE=InnoDB;
\ No newline at end of file
diff --git a/vitess-mixin/e2e/tables/lookup_keyspace_schema_file.sql b/vitess-mixin/e2e/tables/lookup_keyspace_schema_file.sql
deleted file mode 100644
index e9c66144176..00000000000
--- a/vitess-mixin/e2e/tables/lookup_keyspace_schema_file.sql
+++ /dev/null
@@ -1,16 +0,0 @@
-CREATE TABLE tokens_token_lookup (
- id BIGINT NOT NULL AUTO_INCREMENT,
- page BIGINT UNSIGNED,
- token VARCHAR(255) DEFAULT NULL,
- PRIMARY KEY (id),
- UNIQUE KEY idx_token_page (`token`, `page`)
-) ENGINE=InnoDB;
-
-CREATE TABLE messages_message_lookup (
- id BIGINT NOT NULL AUTO_INCREMENT,
- page BIGINT UNSIGNED,
- message VARCHAR(1000),
- PRIMARY KEY (id),
- UNIQUE KEY idx_message_page (`message`, `page`)
-) ENGINE=InnoDB;
-
diff --git a/vitess-mixin/e2e/tables/test_keyspace_schema_file.sql b/vitess-mixin/e2e/tables/test_keyspace_schema_file.sql
deleted file mode 100644
index b049e4ef211..00000000000
--- a/vitess-mixin/e2e/tables/test_keyspace_schema_file.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-CREATE TABLE messages (
- page BIGINT(20) UNSIGNED,
- time_created_ns BIGINT(20) UNSIGNED,
- message VARCHAR(10000),
- PRIMARY KEY (page, time_created_ns)
-) ENGINE=InnoDB;
-
-CREATE TABLE tokens (
- page BIGINT(20) UNSIGNED,
- time_created_ns BIGINT(20) UNSIGNED,
- token VARCHAR(255) DEFAULT NULL,
- PRIMARY KEY (page, time_created_ns)
-) ENGINE=InnoDB;
-
diff --git a/vitess-mixin/e2e/tablet.yml b/vitess-mixin/e2e/tablet.yml
deleted file mode 100644
index 86cc8316a41..00000000000
--- a/vitess-mixin/e2e/tablet.yml
+++ /dev/null
@@ -1,24 +0,0 @@
-externalConnections:
- erpl:
- flavor: FilePos
- host: external_db_host
- port: 3306
- dbName: commerce
- app:
- user: external_db_user
- password: external_db_password
- dba:
- user: external_db_user
- password: external_db_password
- filtered:
- user: external_db_user
- password: external_db_password
- repl:
- user: external_db_user
- password: external_db_password
- appdebug:
- user: external_db_user
- password: external_db_password
- allprivs:
- user: external_db_user
- password: external_db_password
\ No newline at end of file
diff --git a/vitess-mixin/e2e/test_keyspace_vschema.json b/vitess-mixin/e2e/test_keyspace_vschema.json
deleted file mode 100644
index 55d0df96204..00000000000
--- a/vitess-mixin/e2e/test_keyspace_vschema.json
+++ /dev/null
@@ -1,54 +0,0 @@
-{
- "sharded": true,
- "tables": {
- "messages": {
- "column_vindexes": [
- {
- "column": "page",
- "name": "hash"
- },
- {
- "column": "message",
- "name": "messages_message_lookup"
- }
- ]
- },
- "tokens": {
- "column_vindexes": [
- {
- "column": "page",
- "name": "hash"
- },
- {
- "column": "token",
- "name": "tokens_token_lookup"
- }
- ]
- }
- },
- "vindexes": {
- "hash": {
- "type": "hash"
- },
- "messages_message_lookup": {
- "type": "lookup_hash",
- "params": {
- "table": "lookup_keyspace.messages_message_lookup",
- "from": "message",
- "to": "page",
- "autocommit": "true"
- },
- "owner": "messages"
- },
- "tokens_token_lookup": {
- "type": "lookup_hash",
- "params": {
- "table": "lookup_keyspace.tokens_token_lookup",
- "from": "token",
- "to": "page",
- "autocommit": "true"
- },
- "owner": "tokens"
- }
- }
-}
\ No newline at end of file
diff --git a/vitess-mixin/e2e/vtcompose/base_vschema.json b/vitess-mixin/e2e/vtcompose/base_vschema.json
deleted file mode 100644
index b867400e5ee..00000000000
--- a/vitess-mixin/e2e/vtcompose/base_vschema.json
+++ /dev/null
@@ -1,10 +0,0 @@
-{
- "sharded": true,
- "vindexes": {
- "hash": {
- "type": "hash"
- }
- },
- "tables": {
- }
-}
diff --git a/vitess-mixin/e2e/vtcompose/docker-compose.base.yml b/vitess-mixin/e2e/vtcompose/docker-compose.base.yml
deleted file mode 100644
index 2f2fbc12c70..00000000000
--- a/vitess-mixin/e2e/vtcompose/docker-compose.base.yml
+++ /dev/null
@@ -1,58 +0,0 @@
-version: "2.1"
-services:
- consul1:
- image: consul:latest
- hostname: "consul1"
- ports:
- - "8400:8400"
- - "8500:8500"
- - "8600:8600"
- command: "agent -server -bootstrap-expect 3 -ui -disable-host-node-id -client 0.0.0.0"
- consul2:
- image: consul:latest
- hostname: "consul2"
- expose:
- - "8400"
- - "8500"
- - "8600"
- command: "agent -server -retry-join consul1 -disable-host-node-id"
- depends_on:
- - consul1
- consul3:
- image: consul:latest
- hostname: "consul3"
- expose:
- - "8400"
- - "8500"
- - "8600"
- command: "agent -server -retry-join consul1 -disable-host-node-id"
- depends_on:
- - consul1
- # This is a convenience container to quickly test vitess against an external database.
- # In practice you will point Vitess to your existing database and migrate to a Vitess managed cluster.
- external_db_host:
- build:
- context: ./external_db/mysql
- dockerfile: Dockerfile
- restart: always
- environment:
- MYSQL_ROOT_PASSWORD: ${MYSQL_ROOT_PASSWORD:-pass}
- MYSQL_DATABASE: ${DB:-commerce}
- MYSQL_USER: ${DB_USER:-external_db_user}
- MYSQL_PASSWORD: ${DB_PASS:-external_db_password}
- volumes:
- - ./external_db/mysql/:/docker-entrypoint-initdb.d/
- - ./external_db/mysql/log:/var/log/mysql
- command:
- - --server-id=1
- - --log-bin=mysql-bin
- - --gtid_mode=ON
- - --enforce_gtid_consistency
- - --general_log=1
- - --slow_query_log=1
- healthcheck:
- test: "/usr/bin/mysql --user=root --password=$${MYSQL_ROOT_PASSWORD} --execute \"SHOW DATABASES;\""
- timeout: 10s
- retries: 10
- ports:
- - "3306"
\ No newline at end of file
diff --git a/vitess-mixin/e2e/vtcompose/docker-compose.test.yml b/vitess-mixin/e2e/vtcompose/docker-compose.test.yml
deleted file mode 100644
index e6928d73acc..00000000000
--- a/vitess-mixin/e2e/vtcompose/docker-compose.test.yml
+++ /dev/null
@@ -1,303 +0,0 @@
-# This file was generated by running `go run vtcompose/vtcompose.go` with no arguments.
-
-services:
- consul1:
- command: agent -server -bootstrap-expect 3 -ui -disable-host-node-id -client 0.0.0.0
- hostname: consul1
- image: consul:latest
- ports:
- - 8400:8400
- - 8500:8500
- - 8600:8600
- consul2:
- command: agent -server -retry-join consul1 -disable-host-node-id
- depends_on:
- - consul1
- expose:
- - "8400"
- - "8500"
- - "8600"
- hostname: consul2
- image: consul:latest
- consul3:
- command: agent -server -retry-join consul1 -disable-host-node-id
- depends_on:
- - consul1
- expose:
- - "8400"
- - "8500"
- - "8600"
- hostname: consul3
- image: consul:latest
- schemaload_test_keyspace:
- command:
- - sh
- - -c
- - /script/schemaload.sh
- depends_on:
- vttablet101:
- condition: service_healthy
- vttablet201:
- condition: service_healthy
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - TARGETTAB=test-0000000101
- - SLEEPTIME=15
- - VSCHEMA_FILE=test_keyspace_vschema.json
- - SCHEMA_FILES=test_keyspace_schema_file.sql
- - POST_LOAD_FILE=
- - EXTERNAL_DB=0
- image: vitess/lite
- volumes:
- - .:/script
- schemaload_unsharded_keyspace:
- command:
- - sh
- - -c
- - /script/schemaload.sh
- depends_on:
- vttablet301:
- condition: service_healthy
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=unsharded_keyspace
- - TARGETTAB=test-0000000301
- - SLEEPTIME=15
- - VSCHEMA_FILE=unsharded_keyspace_vschema.json
- - SCHEMA_FILES=unsharded_keyspace_schema_file.sql
- - POST_LOAD_FILE=
- - EXTERNAL_DB=0
- image: vitess/lite
- volumes:
- - .:/script
- vtctld:
- command:
- - sh
- - -c
- - ' $$VTROOT/bin/vtctld -topo_implementation consul -topo_global_server_address
- consul1:8500 -topo_global_root vitess/global -cell test ''grpc-vtctl'' -backup_storage_implementation
- file -file_backup_storage_root $$VTDATAROOT/backups -logtostderr=true -port
- 8080 -grpc_port 15999 -pid_file $$VTDATAROOT/tmp/vtctld.pid '
- depends_on:
- - consul1
- - consul2
- - consul3
- image: vitess/lite
- ports:
- - 15000:8080
- - "15999"
- volumes:
- - .:/script
- vtgate:
- command:
- - sh
- - -c
- - '/script/run-forever.sh $$VTROOT/bin/vtgate --topo_implementation consul --topo_global_server_address
- consul1:8500 --topo_global_root vitess/global --logtostderr=true --port 8080 --grpc_port
- 15999 --mysql_server_port 15306 --mysql_auth_server_impl none --cell test --cells_to_watch
- test --tablet_types_to_wait PRIMARY,REPLICA,RDONLY --service_map ''grpc-vtgateservice''
- --pid_file $$VTDATAROOT/tmp/vtgate.pid --normalize_queries=true '
- depends_on:
- - vtctld
- image: vitess/lite
- ports:
- - 15099:8080
- - "15999"
- - 15306:15306
- volumes:
- - .:/script
- vttablet101:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 101
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=-80
- - ROLE=primary
- - VTHOST=vttablet101
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite
- ports:
- - 15101:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet102:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 102
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=-80
- - ROLE=replica
- - VTHOST=vttablet102
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite
- ports:
- - 15102:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet201:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 201
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=80-
- - ROLE=primary
- - VTHOST=vttablet201
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite
- ports:
- - 15201:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet202:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 202
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=test_keyspace
- - SHARD=80-
- - ROLE=replica
- - VTHOST=vttablet202
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite
- ports:
- - 15202:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
- vttablet301:
- command:
- - sh
- - -c
- - /script/vttablet-up.sh 301
- depends_on:
- - vtctld
- environment:
- - TOPOLOGY_FLAGS=-topo_implementation consul -topo_global_server_address consul1:8500
- -topo_global_root vitess/global
- - WEB_PORT=8080
- - GRPC_PORT=15999
- - CELL=test
- - KEYSPACE=unsharded_keyspace
- - SHARD=-
- - ROLE=primary
- - VTHOST=vttablet301
- - EXTERNAL_DB=0
- - DB_PORT=
- - DB_HOST=
- - DB_USER=
- - DB_PASS=
- - DB_CHARSET=
- healthcheck:
- interval: 30s
- retries: 15
- test:
- - CMD-SHELL
- - curl localhost:8080/debug/health
- timeout: 10s
- image: vitess/lite
- ports:
- - 15301:8080
- - "15999"
- - "3306"
- volumes:
- - .:/script
-version: "2.1"
diff --git a/vitess-mixin/e2e/vtcompose/vtcompose.go b/vitess-mixin/e2e/vtcompose/vtcompose.go
deleted file mode 100644
index d272c7330ff..00000000000
--- a/vitess-mixin/e2e/vtcompose/vtcompose.go
+++ /dev/null
@@ -1,857 +0,0 @@
-/*
- * Copyright 2020 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 main
-
-import (
- "bytes"
- "encoding/json"
- "flag"
- "fmt"
- "math"
- "os"
- "regexp"
- "strconv"
- "strings"
-
- jsonpatch "github.com/evanphx/json-patch"
- yamlpatch "github.com/krishicks/yaml-patch"
-
- "vitess.io/vitess/go/vt/log"
-)
-
-const (
- DefaultWebPort = 8080
- webPortUsage = "Web port to be used"
- DefaultGrpcPort = 15999
- gRpcPortUsage = "gRPC port to be used"
- DefaultMysqlPort = 15306
- mySqlPortUsage = "mySql port to be used"
- DefaultKeyspaceData = "test_keyspace:2:1:create_messages.sql,create_tokens.sql;unsharded_keyspace:0:0:create_dinosaurs.sql,create_eggs.sql"
- keyspaceDataUsage = "List of keyspace_name/external_db_name:num_of_shards:num_of_replica_tablets:schema_files:lookup_keyspace_name separated by ';'"
- DefaultCell = "test"
- cellUsage = "Vitess Cell name"
- DefaultExternalDbData = ""
- externalDbDataUsage = "List of Data corresponding to external DBs. List of ,,,,, separated by ';'"
- DefaultTopologyFlags = "-topo_implementation consul -topo_global_server_address consul1:8500 -topo_global_root vitess/global"
- topologyFlagsUsage = "Vitess Topology Flags config"
-)
-
-var (
- tabletsUsed = 0
- tablesPath = "tables/"
- baseDockerComposeFile = flag.String("base_yaml", "vtcompose/docker-compose.base.yml", "Starting docker-compose yaml")
- baseVschemaFile = flag.String("base_vschema", "vtcompose/base_vschema.json", "Starting vschema json")
-
- topologyFlags = flag.String("topologyFlags", DefaultTopologyFlags, topologyFlagsUsage)
- webPort = flag.Int("webPort", DefaultWebPort, webPortUsage)
- gRpcPort = flag.Int("gRpcPort", DefaultGrpcPort, gRpcPortUsage)
- mySqlPort = flag.Int("mySqlPort", DefaultMysqlPort, mySqlPortUsage)
- cell = flag.String("cell", DefaultCell, cellUsage)
- keyspaceData = flag.String("keyspaceData", DefaultKeyspaceData, keyspaceDataUsage)
- externalDbData = flag.String("externalDbData", DefaultExternalDbData, externalDbDataUsage)
-)
-
-type vtOptions struct {
- webPort int
- gRpcPort int
- mySqlPort int
- topologyFlags string
- cell string
-}
-
-type keyspaceInfo struct {
- keyspace string
- shards int
- replicaTablets int
- lookupKeyspace string
- useLookups bool
- schemaFile *os.File
- schemaFileNames []string
-}
-
-type externalDbInfo struct {
- dbName string
- dbHost string
- dbPort string
- dbUser string
- dbPass string
- dbCharset string
-}
-
-func newKeyspaceInfo(
- keyspace string,
- shards int,
- replicaTablets int,
- schemaFiles []string,
- lookupKeyspace string,
-) keyspaceInfo {
- k := keyspaceInfo{
- keyspace: keyspace,
- shards: shards,
- replicaTablets: replicaTablets,
- schemaFileNames: schemaFiles,
- lookupKeyspace: lookupKeyspace,
- }
- if len(strings.TrimSpace(lookupKeyspace)) == 0 {
- k.useLookups = false
- } else {
- k.useLookups = true
- }
-
- k.schemaFile = nil
- return k
-}
-
-func newExternalDbInfo(dbName, dbHost, dbPort, dbUser, dbPass, dbCharset string) externalDbInfo {
- return externalDbInfo{
- dbName: dbName,
- dbHost: dbHost,
- dbPort: dbPort,
- dbUser: dbUser,
- dbPass: dbPass,
- dbCharset: dbCharset,
- }
-}
-
-func parseKeyspaceInfo(keyspaceData string) map[string]keyspaceInfo {
- keyspaceInfoMap := make(map[string]keyspaceInfo)
-
- for _, v := range strings.Split(keyspaceData, ";") {
- tokens := strings.Split(v, ":")
- shards, _ := strconv.Atoi(tokens[1])
- replicaTablets, _ := strconv.Atoi(tokens[2])
- schemaFileNames := []string{}
- // Make schemafiles argument optional
- if len(tokens) > 3 {
- f := func(c rune) bool {
- return c == ','
- }
- schemaFileNames = strings.FieldsFunc(tokens[3], f)
- }
-
- if len(tokens) > 4 {
- keyspaceInfoMap[tokens[0]] = newKeyspaceInfo(tokens[0], shards, replicaTablets, schemaFileNames, tokens[4])
- } else {
- keyspaceInfoMap[tokens[0]] = newKeyspaceInfo(tokens[0], shards, replicaTablets, schemaFileNames, "")
- }
- }
-
- return keyspaceInfoMap
-}
-
-func parseExternalDbData(externalDbData string) map[string]externalDbInfo {
- externalDbInfoMap := make(map[string]externalDbInfo)
- for _, v := range strings.Split(externalDbData, ";") {
- tokens := strings.Split(v, ":")
- if len(tokens) > 1 {
- externalDbInfoMap[tokens[0]] =
- newExternalDbInfo(tokens[0], tokens[1], tokens[2], tokens[3], tokens[4], tokens[5])
- }
- }
-
- return externalDbInfoMap
-}
-
-func main() {
- flag.Parse()
- keyspaceInfoMap := parseKeyspaceInfo(*keyspaceData)
- externalDbInfoMap := parseExternalDbData(*externalDbData)
- vtOpts := vtOptions{
- webPort: *webPort,
- gRpcPort: *gRpcPort,
- mySqlPort: *mySqlPort,
- topologyFlags: *topologyFlags,
- cell: *cell,
- }
-
- // Write schemaFile.
- for k, v := range keyspaceInfoMap {
- if _, ok := externalDbInfoMap[k]; !ok {
- v.schemaFile = createFile(fmt.Sprintf("%s%s_schema_file.sql", tablesPath, v.keyspace))
- appendToSqlFile(v.schemaFileNames, v.schemaFile)
- closeFile(v.schemaFile)
- }
- }
-
- // Vschema Patching
- for k, keyspaceData := range keyspaceInfoMap {
- vSchemaFile := readFile(*baseVschemaFile)
- if keyspaceData.shards == 0 {
- vSchemaFile = applyJsonInMemoryPatch(vSchemaFile, `[{"op": "replace","path": "/sharded", "value": false}]`)
- }
-
- // Check if it is an external_db
- if _, ok := externalDbInfoMap[k]; ok {
- //This is no longer necessary, but we'll keep it for reference
- //https://github.com/vitessio/vitess/pull/4868, https://github.com/vitessio/vitess/pull/5010
- //vSchemaFile = applyJsonInMemoryPatch(vSchemaFile,`[{"op": "add","path": "/tables/*", "value": {}}]`)
- } else {
- var primaryTableColumns map[string]string
- vSchemaFile, primaryTableColumns = addTablesVschemaPatch(vSchemaFile, keyspaceData.schemaFileNames)
-
- if keyspaceData.useLookups {
- lookup := keyspaceInfoMap[keyspaceData.lookupKeyspace]
- vSchemaFile = addLookupDataToVschema(vSchemaFile, lookup.schemaFileNames, primaryTableColumns, lookup.keyspace)
- }
- }
-
- writeVschemaFile(vSchemaFile, fmt.Sprintf("%s_vschema.json", keyspaceData.keyspace))
- }
-
- // Docker Compose File Patches
- dockerComposeFile := readFile(*baseDockerComposeFile)
- dockerComposeFile = applyDockerComposePatches(dockerComposeFile, keyspaceInfoMap, externalDbInfoMap, vtOpts)
- writeFile(dockerComposeFile, "docker-compose.yml")
-}
-
-func applyFilePatch(dockerYaml []byte, patchFile string) []byte {
- yamlPatch, err := os.ReadFile(patchFile)
- if err != nil {
- log.Fatalf("reading yaml patch file %s: %s", patchFile, err)
- }
-
- patch, err := yamlpatch.DecodePatch(yamlPatch)
- if err != nil {
- log.Fatalf("decoding patch failed: %s", err)
- }
-
- bs, err := patch.Apply(dockerYaml)
- if err != nil {
- log.Fatalf("applying patch failed: %s", err)
- }
- return bs
-}
-
-func applyJsonInMemoryPatch(vSchemaFile []byte, patchString string) []byte {
- patch, err := jsonpatch.DecodePatch([]byte(patchString))
- if err != nil {
- log.Fatalf("decoding vschema patch failed: %s", err)
- }
-
- modified, err := patch.Apply(vSchemaFile)
- if err != nil {
- log.Fatalf("applying vschema patch failed: %s", err)
- }
- return modified
-}
-
-func applyInMemoryPatch(dockerYaml []byte, patchString string) []byte {
- patch, err := yamlpatch.DecodePatch([]byte(patchString))
- if err != nil {
- log.Fatalf("decoding patch failed: %s", err)
- }
-
- bs, err := patch.Apply(dockerYaml)
- if err != nil {
- log.Fatalf("applying patch failed: %s", err)
- }
- return bs
-}
-
-func createFile(filePath string) *os.File {
- f, err := os.Create(filePath)
- if err != nil {
- log.Fatalf("creating %s %s", filePath, err)
- }
- return f
-}
-
-func readFile(filePath string) []byte {
- file, err := os.ReadFile(filePath)
-
- if err != nil {
- log.Fatalf("reading %s: %s", filePath, err)
- }
-
- return file
-}
-
-func closeFile(file *os.File) {
- err := file.Close()
- if err != nil {
- log.Fatalf("Closing schema_file.sql %s", err)
- }
-}
-
-func handleError(err error) {
- if err != nil {
- log.Fatalf("Error: %s", err)
- }
-}
-
-func appendToSqlFile(schemaFileNames []string, f *os.File) {
- for _, file := range schemaFileNames {
- data, err := os.ReadFile(tablesPath + file)
- if err != nil {
- log.Fatalf("reading %s: %s", tablesPath+file, err)
- }
-
- _, err = f.Write(data)
- handleError(err)
-
- _, err = f.WriteString("\n\n")
- handleError(err)
-
- err = f.Sync()
- handleError(err)
- }
-}
-
-func getTableName(sqlFile string) string {
- sqlFileData, err := os.ReadFile(sqlFile)
- if err != nil {
- log.Fatalf("reading sqlFile file %s: %s", sqlFile, err)
- }
-
- r, _ := regexp.Compile("CREATE TABLE ([a-z_-]*) \\(")
- rs := r.FindStringSubmatch(string(sqlFileData))
- // replace all ` from table name if exists
- return strings.ReplaceAll(rs[1], "`", "")
-}
-
-func getPrimaryKey(sqlFile string) string {
- sqlFileData, err := os.ReadFile(sqlFile)
- if err != nil {
- log.Fatalf("reading sqlFile file %s: %s", sqlFile, err)
- }
-
- r, _ := regexp.Compile("PRIMARY KEY \\((.*)\\).*")
- rs := r.FindStringSubmatch(string(sqlFileData))
-
- return rs[1]
-}
-
-func getKeyColumns(sqlFile string) string {
- sqlFileData, err := os.ReadFile(sqlFile)
- if err != nil {
- log.Fatalf("reading sqlFile file %s: %s", sqlFile, err)
- }
-
- r, _ := regexp.Compile("[^PRIMARY] (KEY|UNIQUE KEY) .*\\((.*)\\).*")
- rs := r.FindStringSubmatch(string(sqlFileData))
- // replace all ` from column names if exists
- return strings.ReplaceAll(rs[2], "`", "")
-}
-
-func addTablesVschemaPatch(vSchemaFile []byte, schemaFileNames []string) ([]byte, map[string]string) {
- indexedColumns := ""
- primaryTableColumns := make(map[string]string)
- for _, fileName := range schemaFileNames {
- tableName := getTableName(tablesPath + fileName)
- indexedColumns = getPrimaryKey(tablesPath + fileName)
- firstColumnName := strings.Split(indexedColumns, ", ")[0]
- vSchemaFile = applyJsonInMemoryPatch(vSchemaFile, generatePrimaryVIndex(tableName, firstColumnName, "hash"))
- primaryTableColumns[tableName] = firstColumnName
- }
-
- return vSchemaFile, primaryTableColumns
-}
-
-func addLookupDataToVschema(
- vSchemaFile []byte,
- schemaFileNames []string,
- primaryTableColumns map[string]string,
- keyspace string,
-) []byte {
- for _, fileName := range schemaFileNames {
- tableName := fileName[7 : len(fileName)-4]
- lookupTableOwner := ""
-
- // Find owner of lookup table
- for primaryTableName := range primaryTableColumns {
- if strings.HasPrefix(tableName, primaryTableName) && len(primaryTableName) > len(lookupTableOwner) {
- lookupTableOwner = primaryTableName
- }
- }
-
- indexedColumns := getKeyColumns(tablesPath + fileName)
- firstColumnName := strings.Split(indexedColumns, ", ")[0]
-
- // Lookup patch under "tables"
- vSchemaFile = applyJsonInMemoryPatch(vSchemaFile, addToColumnVIndexes(lookupTableOwner, firstColumnName, tableName))
-
- // Generate Vschema lookup hash types
- lookupHash := generateVschemaLookupHash(tableName, keyspace, firstColumnName, primaryTableColumns[lookupTableOwner], lookupTableOwner)
- vSchemaFile = applyJsonInMemoryPatch(vSchemaFile, lookupHash)
- }
-
- return vSchemaFile
-}
-
-func writeVschemaFile(file []byte, fileName string) {
- // Format json file
- var buf bytes.Buffer
- err := json.Indent(&buf, file, "", "\t")
- handleError(err)
- file = buf.Bytes()
-
- writeFile(file, fileName)
-}
-
-func writeFile(file []byte, fileName string) {
- err := os.WriteFile(fileName, file, 0644)
- if err != nil {
- log.Fatalf("writing %s %s", fileName, err)
- }
-}
-
-func applyKeyspaceDependentPatches(
- dockerComposeFile []byte,
- keyspaceData keyspaceInfo,
- externalDbInfoMap map[string]externalDbInfo,
- opts vtOptions,
-) []byte {
- var externalDbInfo externalDbInfo
- if val, ok := externalDbInfoMap[keyspaceData.keyspace]; ok {
- externalDbInfo = val
- }
- tabAlias := 0 + tabletsUsed*100
- shard := "-"
- var primaryTablets []string
- if tabletsUsed == 0 {
- primaryTablets = append(primaryTablets, "101")
- } else {
- primaryTablets = append(primaryTablets, strconv.Itoa((tabletsUsed+1)*100+1))
- }
- interval := int(math.Floor(256 / float64(keyspaceData.shards)))
-
- for i := 1; i < keyspaceData.shards; i++ {
- primaryTablets = append(primaryTablets, strconv.Itoa((i+1)*100+1))
- }
-
- schemaLoad := generateSchemaload(primaryTablets, "", keyspaceData.keyspace, externalDbInfo, opts)
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, schemaLoad)
-
- // Append Primary and Replica Tablets
- if keyspaceData.shards < 2 {
- tabAlias = tabAlias + 100
- dockerComposeFile = applyTabletPatches(dockerComposeFile, tabAlias, shard, keyspaceData, externalDbInfoMap, opts)
- dockerComposeFile = applyShardPatches(dockerComposeFile, tabAlias, shard, keyspaceData, externalDbInfoMap, opts)
- } else {
- // Determine shard range
- for i := 0; i < keyspaceData.shards; i++ {
- if i == 0 {
- shard = fmt.Sprintf("-%x", interval)
- } else if i == (keyspaceData.shards - 1) {
- shard = fmt.Sprintf("%x-", interval*i)
- } else {
- shard = fmt.Sprintf("%x-%x", interval*(i), interval*(i+1))
- }
- tabAlias = tabAlias + 100
- dockerComposeFile = applyTabletPatches(dockerComposeFile, tabAlias, shard, keyspaceData, externalDbInfoMap, opts)
- dockerComposeFile = applyShardPatches(dockerComposeFile, tabAlias, shard, keyspaceData, externalDbInfoMap, opts)
- }
- }
-
- tabletsUsed += len(primaryTablets)
- return dockerComposeFile
-}
-
-func applyDefaultDockerPatches(
- dockerComposeFile []byte,
- keyspaceInfoMap map[string]keyspaceInfo,
- externalDbInfoMap map[string]externalDbInfo,
- opts vtOptions,
-) []byte {
-
- var dbInfo externalDbInfo
- // This is a workaround to check if there are any externalDBs defined
- for _, keyspaceData := range keyspaceInfoMap {
- if val, ok := externalDbInfoMap[keyspaceData.keyspace]; ok {
- dbInfo = val
- }
- }
-
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateVtctld(opts))
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateVtgate(opts))
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateVreplication(dbInfo, opts))
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateVTOrc(dbInfo, opts))
- return dockerComposeFile
-}
-
-func applyDockerComposePatches(
- dockerComposeFile []byte,
- keyspaceInfoMap map[string]keyspaceInfo,
- externalDbInfoMap map[string]externalDbInfo,
- vtOpts vtOptions,
-) []byte {
- // Vtctld, vtgate, vtwork patches.
- dockerComposeFile = applyDefaultDockerPatches(dockerComposeFile, keyspaceInfoMap, externalDbInfoMap, vtOpts)
- for _, keyspaceData := range keyspaceInfoMap {
- dockerComposeFile = applyKeyspaceDependentPatches(dockerComposeFile, keyspaceData, externalDbInfoMap, vtOpts)
- }
-
- return dockerComposeFile
-}
-
-func applyShardPatches(
- dockerComposeFile []byte,
- tabAlias int,
- shard string,
- keyspaceData keyspaceInfo,
- externalDbInfoMap map[string]externalDbInfo,
- opts vtOptions,
-) []byte {
- var dbInfo externalDbInfo
- if val, ok := externalDbInfoMap[keyspaceData.keyspace]; ok {
- dbInfo = val
- }
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateExternalPrimary(tabAlias, shard, keyspaceData, dbInfo, opts))
- return dockerComposeFile
-}
-
-func generateDefaultShard(tabAlias int, shard string, keyspaceData keyspaceInfo, opts vtOptions) string {
- aliases := []int{tabAlias + 1} // primary alias, e.g. 201
- for i := 0; i < keyspaceData.replicaTablets; i++ {
- aliases = append(aliases, tabAlias+2+i) // replica aliases, e.g. 202, 203, ...
- }
- tabletDepends := make([]string, len(aliases))
- for i, tabletId := range aliases {
- tabletDepends[i] = fmt.Sprintf("vttablet%d: {condition : service_healthy}", tabletId)
- }
- // Wait on all shard tablets to be healthy
- dependsOn := "depends_on: {" + strings.Join(tabletDepends, ", ") + "}"
-
- return fmt.Sprintf(`
-- op: add
- path: /services/init_shard_primary%[2]d
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- command: ["sh", "-c", "/vt/bin/vtctlclient %[5]s InitShardPrimary -force %[4]s/%[3]s %[6]s-%[2]d "]
- %[1]s
-`, dependsOn, aliases[0], shard, keyspaceData.keyspace, opts.topologyFlags, opts.cell)
-}
-
-func generateExternalPrimary(
- tabAlias int,
- shard string,
- keyspaceData keyspaceInfo,
- dbInfo externalDbInfo,
- opts vtOptions,
-) string {
-
- aliases := []int{tabAlias + 1} // primary alias, e.g. 201
- for i := 0; i < keyspaceData.replicaTablets; i++ {
- aliases = append(aliases, tabAlias+2+i) // replica aliases, e.g. 202, 203, ...
- }
-
- externalPrimaryTab := tabAlias
- externalDb := "0"
-
- if dbInfo.dbName != "" {
- externalDb = "1"
- } else {
- return fmt.Sprintf(``)
- }
-
- return fmt.Sprintf(`
-- op: add
- path: /services/vttablet%[1]d
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15%[1]d:%[3]d"
- - "%[4]d"
- - "3306"
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS=%[2]s
- - WEB_PORT=%[3]d
- - GRPC_PORT=%[4]d
- - CELL=%[5]s
- - SHARD=%[6]s
- - KEYSPACE=%[7]s
- - ROLE=primary
- - VTHOST=vttablet%[1]d
- - EXTERNAL_DB=%[8]s
- - DB_PORT=%[9]s
- - DB_HOST=%[10]s
- - DB_USER=%[11]s
- - DB_PASS=%[12]s
- - DB_CHARSET=%[13]s
- command: ["sh", "-c", "[ $$EXTERNAL_DB -eq 1 ] && /script/vttablet-up.sh %[1]d || exit 0"]
- depends_on:
- - vtctld
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:%[3]d/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-`, externalPrimaryTab, opts.topologyFlags, opts.webPort, opts.gRpcPort, opts.cell, shard, keyspaceData.keyspace, externalDb, dbInfo.dbPort, dbInfo.dbHost, dbInfo.dbUser, dbInfo.dbPass, dbInfo.dbCharset)
-}
-
-func applyTabletPatches(
- dockerComposeFile []byte,
- tabAlias int,
- shard string,
- keyspaceData keyspaceInfo,
- externalDbInfoMap map[string]externalDbInfo,
- opts vtOptions,
-) []byte {
- var dbInfo externalDbInfo
- if val, ok := externalDbInfoMap[keyspaceData.keyspace]; ok {
- dbInfo = val
- }
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateDefaultTablet(tabAlias+1, shard, "primary", keyspaceData.keyspace, dbInfo, opts))
- for i := 0; i < keyspaceData.replicaTablets; i++ {
- dockerComposeFile = applyInMemoryPatch(dockerComposeFile, generateDefaultTablet(tabAlias+2+i, shard, "replica", keyspaceData.keyspace, dbInfo, opts))
- }
- return dockerComposeFile
-}
-
-func generateDefaultTablet(tabAlias int, shard, role, keyspace string, dbInfo externalDbInfo, opts vtOptions) string {
- externalDb := "0"
- if dbInfo.dbName != "" {
- externalDb = "1"
- }
-
- return fmt.Sprintf(`
-- op: add
- path: /services/vttablet%[1]d
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15%[1]d:%[4]d"
- - "%[5]d"
- - "3306"
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS=%[7]s
- - WEB_PORT=%[4]d
- - GRPC_PORT=%[5]d
- - CELL=%[8]s
- - KEYSPACE=%[6]s
- - SHARD=%[2]s
- - ROLE=%[3]s
- - VTHOST=vttablet%[1]d
- - EXTERNAL_DB=%[9]s
- - DB_PORT=%[10]s
- - DB_HOST=%[11]s
- - DB_USER=%[12]s
- - DB_PASS=%[13]s
- - DB_CHARSET=%[14]s
- command: ["sh", "-c", "/script/vttablet-up.sh %[1]d"]
- depends_on:
- - vtctld
- healthcheck:
- test: ["CMD-SHELL","curl -s --fail --show-error localhost:%[4]d/debug/health"]
- interval: 30s
- timeout: 10s
- retries: 15
-`, tabAlias, shard, role, opts.webPort, opts.gRpcPort, keyspace, opts.topologyFlags, opts.cell, externalDb, dbInfo.dbPort, dbInfo.dbHost, dbInfo.dbUser, dbInfo.dbPass, dbInfo.dbCharset)
-}
-
-func generateVtctld(opts vtOptions) string {
- return fmt.Sprintf(`
-- op: add
- path: /services/vtctld
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15000:%[1]d"
- - "%[2]d"
- command: ["sh", "-c", " /vt/bin/vtctld \
- %[3]s \
- -cell %[4]s \
- -service_map 'grpc-vtctl' \
- -backup_storage_implementation file \
- -file_backup_storage_root /vt/vtdataroot/backups \
- -logtostderr=true \
- -port %[1]d \
- -grpc_port %[2]d \
- "]
- volumes:
- - .:/script
- depends_on:
- - consul1
- - consul2
- - consul3
- depends_on:
- external_db_host:
- condition: service_healthy
-`, opts.webPort, opts.gRpcPort, opts.topologyFlags, opts.cell)
-}
-
-func generateVtgate(opts vtOptions) string {
- return fmt.Sprintf(`
-- op: add
- path: /services/vtgate
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- ports:
- - "15099:%[1]d"
- - "%[2]d"
- - "15306:%[3]d"
- command: ["sh", "-c", "/script/run-forever.sh /vt/bin/vtgate \
- %[4]s \
- --logtostderr=true \
- --port %[1]d \
- --grpc_port %[2]d \
- --mysql_server_port %[3]d \
- --mysql_auth_server_impl none \
- --cell %[5]s \
- --cells_to_watch %[5]s \
- --tablet_types_to_wait PRIMARY,REPLICA,RDONLY \
- --service_map 'grpc-vtgateservice' \
- --normalize_queries=true \
- "]
- volumes:
- - .:/script
- depends_on:
- - vtctld
-`, opts.webPort, opts.gRpcPort, opts.mySqlPort, opts.topologyFlags, opts.cell)
-}
-
-func generateVTOrc(dbInfo externalDbInfo, opts vtOptions) string {
- externalDb := "0"
- if dbInfo.dbName != "" {
- externalDb = "1"
- }
- return fmt.Sprintf(`
-- op: add
- path: /services/vtorc
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS=%[1]s
- - EXTERNAL_DB=%[2]s
- - DB_USER=%[3]s
- - DB_PASS=%[4]s
- ports:
- - "13000:3000"
- command: ["sh", "-c", "/script/vtorc-up.sh"]
- depends_on:
- - vtctld
-`, opts.topologyFlags, externalDb, dbInfo.dbUser, dbInfo.dbPass)
-}
-
-func generateVreplication(dbInfo externalDbInfo, opts vtOptions) string {
- externalDb := "0"
- if dbInfo.dbName != "" {
- externalDb = "1"
- }
- return fmt.Sprintf(`
-- op: add
- path: /services/vreplication
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS=%[1]s
- - EXTERNAL_DB=%[2]s
- command: ["sh", "-c", "[ $$EXTERNAL_DB -eq 1 ] && /script/externaldb_vreplication.sh || exit 0"]
- depends_on:
- - vtctld
-`, opts.topologyFlags, externalDb)
-}
-
-func generateSchemaload(
- tabletAliases []string,
- postLoadFile string,
- keyspace string,
- dbInfo externalDbInfo,
- opts vtOptions,
-) string {
- targetTab := tabletAliases[0]
- schemaFileName := fmt.Sprintf("%s_schema_file.sql", keyspace)
- externalDb := "0"
-
- if dbInfo.dbName != "" {
- schemaFileName = ""
- externalDb = "1"
- }
-
- // Formatting for list in yaml
- for i, tabletId := range tabletAliases {
- tabletAliases[i] = "vttablet" + tabletId + ": " + "{condition : service_healthy}"
- }
- dependsOn := "depends_on: {" + strings.Join(tabletAliases, ", ") + "}"
-
- return fmt.Sprintf(`
-- op: add
- path: /services/schemaload_%[7]s
- value:
- image: vitess/lite:${VITESS_TAG:-latest}
- volumes:
- - ".:/script"
- environment:
- - TOPOLOGY_FLAGS=%[3]s
- - WEB_PORT=%[4]d
- - GRPC_PORT=%[5]d
- - CELL=%[6]s
- - KEYSPACE=%[7]s
- - TARGETTAB=%[6]s-0000000%[2]s
- - SLEEPTIME=15
- - VSCHEMA_FILE=%[7]s_vschema.json
- - SCHEMA_FILES=%[9]s
- - POST_LOAD_FILE=%[8]s
- - EXTERNAL_DB=%[10]s
- command: ["sh", "-c", "/script/schemaload.sh"]
- %[1]s
-`, dependsOn, targetTab, opts.topologyFlags, opts.webPort, opts.gRpcPort, opts.cell, keyspace, postLoadFile, schemaFileName, externalDb)
-}
-
-func generatePrimaryVIndex(tableName, column, name string) string {
- return fmt.Sprintf(`
-[{"op": "add",
-"path": "/tables/%[1]s",
-"value":
- {"column_vindexes": [
- {
- "column": "%[2]s",
- "name": "%[3]s"
- }
- ]}
-}]
-`, tableName, column, name)
-}
-
-func generateVschemaLookupHash(tableName, tableKeyspace, from, to, owner string) string {
- return fmt.Sprintf(`
-[{"op": "add",
-"path": "/vindexes/%[1]s",
-"value":
- {"type": "lookup_hash",
- "params": {
- "table": "%[2]s.%[1]s",
- "from": "%[3]s",
- "to": "%[4]s",
- "autocommit": "true"
- },
- "owner": "%[5]s"
- }
-}]
-`, tableName, tableKeyspace, from, to, owner)
-}
-
-func addToColumnVIndexes(tableName, column, referenceName string) string {
- return fmt.Sprintf(`
-[{"op": "add",
-"path": "/tables/%[1]s/column_vindexes/-",
-"value":
- {
- "column": "%[2]s",
- "name": "%[3]s"
- }
-}]
-`, tableName, column, referenceName)
-}
diff --git a/vitess-mixin/e2e/vtcompose/vtcompose_test.go b/vitess-mixin/e2e/vtcompose/vtcompose_test.go
deleted file mode 100644
index 8e5cfaa2b89..00000000000
--- a/vitess-mixin/e2e/vtcompose/vtcompose_test.go
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Copyright 2019 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 main
-
-import (
- "fmt"
- "strconv"
- "testing"
-
- "github.com/stretchr/testify/assert"
-)
-
-var (
- referenceYaml = string(readFile("./docker-compose.test.yml"))
- testComposeFile = readFile("./docker-compose.base.yml")
- testKeyspaceInfoMap = parseKeyspaceInfo(DefaultKeyspaceData)
- testExternalDbInfoMap = parseExternalDbData(DefaultExternalDbData)
-
- testVtOpts = vtOptions{
- webPort: DefaultWebPort,
- gRpcPort: DefaultGrpcPort,
- mySqlPort: DefaultMysqlPort,
- topologyFlags: DefaultTopologyFlags,
- cell: DefaultCell,
- }
-)
-
-func TestGenerateCorrectFileWithDefaultOpts(t *testing.T) {
- baseFile := testComposeFile
- finalFile := applyDockerComposePatches(baseFile, testKeyspaceInfoMap, testExternalDbInfoMap, testVtOpts)
-
- yamlString := string(finalFile)
- assert.YAMLEq(t, referenceYaml, yamlString)
-}
-
-func TestOptsAppliedThroughoutGeneratedFile(t *testing.T) {
- baseFile := testComposeFile
- options := vtOptions{
- webPort: 55_555,
- gRpcPort: 66_666,
- mySqlPort: 77_777,
- topologyFlags: "-custom -flags",
- cell: "custom cell",
- }
- finalFile := applyDockerComposePatches(baseFile, testKeyspaceInfoMap, testExternalDbInfoMap, options)
- yamlString := string(finalFile)
-
- // These asserts are not exhaustive, but should cover most cases.
- assert.NotContains(t, yamlString, strconv.Itoa(DefaultWebPort))
- assert.Contains(t, yamlString, strconv.Itoa(options.webPort))
-
- assert.NotContains(t, yamlString, strconv.Itoa(DefaultGrpcPort))
- assert.Contains(t, yamlString, strconv.Itoa(options.gRpcPort))
-
- assert.NotContains(t, yamlString, ":"+strconv.Itoa(DefaultMysqlPort))
- assert.Contains(t, yamlString, ":"+strconv.Itoa(options.webPort))
-
- assert.NotContains(t, yamlString, fmt.Sprintf("-cell %s", DefaultCell))
- assert.Contains(t, yamlString, fmt.Sprintf("-cell %s", options.cell))
-
- assert.Contains(t, yamlString, fmt.Sprintf("- TOPOLOGY_FLAGS=%s", options.topologyFlags))
- assert.NotContains(t, yamlString, DefaultTopologyFlags)
-}
diff --git a/vitess-mixin/e2e/vtorc-up.sh b/vitess-mixin/e2e/vtorc-up.sh
deleted file mode 100755
index 9d71831c3ea..00000000000
--- a/vitess-mixin/e2e/vtorc-up.sh
+++ /dev/null
@@ -1,43 +0,0 @@
-#!/bin/bash
-
-# Copyright 2020 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.
-
-set -u
-
-external=${EXTERNAL_DB:-0}
-config=${VTORC_CONFIG:-/vt/vtorc/config.json}
-# Copy config directory
-cp -R /script/vtorc /vt
-# Update credentials
-if [ $external = 1 ] ; then
- # Terrible substitution but we don't have jq in this image
- # This can be overridden by passing VTORC_CONFIG env variable
- echo "Updating $config..."
- cp /vt/vtorc/default.json /vt/vtorc/tmp.json
- sed -i '/MySQLTopologyUser/c\ \"MySQLTopologyUser\" : \"'"$DB_USER"'\",' /vt/vtorc/tmp.json
- sed -i '/MySQLTopologyPassword/c\ \"MySQLTopologyPassword\" : \"'"$DB_PASS"'\",' /vt/vtorc/tmp.json
- sed -i '/MySQLReplicaUser/c\ \"MySQLReplicaUser\" : \"'"$DB_USER"'\",' /vt/vtorc/tmp.json
- sed -i '/MySQLReplicaPassword/c\ \"MySQLReplicaPassword\" : \"'"$DB_PASS"'\",' /vt/vtorc/tmp.json
- cat /vt/vtorc/tmp.json
- cp /vt/vtorc/tmp.json /vt/vtorc/config.json
-else
- cp /vt/vtorc/default.json /vt/vtorc/config.json
-fi
-
-echo "Starting vtorc..."
-exec /vt/bin/vtorc \
-$TOPOLOGY_FLAGS \
--logtostderr=true \
--config $config
diff --git a/vitess-mixin/e2e/vtorc/default.json b/vitess-mixin/e2e/vtorc/default.json
deleted file mode 100644
index e0a03884c99..00000000000
--- a/vitess-mixin/e2e/vtorc/default.json
+++ /dev/null
@@ -1,13 +0,0 @@
-{
- "Debug": true,
- "EnableSyslog": false,
- "MySQLTopologyUser": "orc_client_user",
- "MySQLTopologyPassword": "orc_client_user_password",
- "MySQLReplicaUser": "vt_repl",
- "MySQLReplicaPassword": "",
- "BackendDB": "sqlite",
- "SQLite3DataFile": "/tmp/vtorc.sqlite3",
- "RecoverMasterClusterFilters": ["*"],
- "RecoveryPeriodBlockSeconds": 5,
- "DelayMasterPromotionIfSQLThreadNotUpToDate": true
-}
diff --git a/vitess-mixin/e2e/vttablet-up.sh b/vitess-mixin/e2e/vttablet-up.sh
deleted file mode 100755
index 0bc9d7a629d..00000000000
--- a/vitess-mixin/e2e/vttablet-up.sh
+++ /dev/null
@@ -1,161 +0,0 @@
-#!/bin/bash
-
-# Copyright 2020 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.
-
-set -u
-export VTROOT=/vt
-export VTDATAROOT=/vt/vtdataroot
-
-keyspace=${KEYSPACE:-'test_keyspace'}
-shard=${SHARD:-'0'}
-grpc_port=${GRPC_PORT:-'15999'}
-web_port=${WEB_PORT:-'8080'}
-role=${ROLE:-'replica'}
-vthost=${VTHOST:-`hostname -i`}
-sleeptime=${SLEEPTIME:-'0'}
-uid=$1
-external=${EXTERNAL_DB:-0}
-
-# If DB is not explicitly set, we default to behaviour of prefixing with vt_
-# If there is an external db, the db_nmae will always match the keyspace name
-[ $external = 0 ] && db_name=${DB:-"vt_$keyspace"} || db_name=${DB:-"$keyspace"}
-db_charset=${DB_CHARSET:-''}
-tablet_hostname=''
-
-# Use IPs to simplify connections when testing in docker.
-# Otherwise, blank hostname means the tablet auto-detects FQDN.
-# This is now set further up
-
-printf -v alias '%s-%010d' $CELL $uid
-printf -v tablet_dir 'vt_%010d' $uid
-
-tablet_role=$role
-tablet_type='replica'
-
-# Make every 3rd tablet rdonly
-if (( $uid % 100 % 3 == 0 )) ; then
- tablet_type='rdonly'
-fi
-
-# Consider every tablet with %d00 as external primary
-if [ $external = 1 ] && (( $uid % 100 == 0 )) ; then
- tablet_type='replica'
- tablet_role='externalprimary'
- keyspace="ext_$keyspace"
-fi
-
-# Copy config directory
-cp -R /script/config $VTROOT
-init_db_sql_file="$VTROOT/config/init_db.sql"
-# Clear in-place edits of init_db_sql_file if any exist
-sed -i '/##\[CUSTOM_SQL/{:a;N;/END\]##/!ba};//d' $init_db_sql_file
-
-echo "##[CUSTOM_SQL_START]##" >> $init_db_sql_file
-
-if [ "$external" = "1" ]; then
- # We need a common user for the unmanaged and managed tablets else tools like orchestrator will not function correctly
- echo "Creating matching user for managed tablets..."
- echo "CREATE USER IF NOT EXISTS '$DB_USER'@'%' IDENTIFIED BY '$DB_PASS';" >> $init_db_sql_file
- echo "GRANT ALL ON *.* TO '$DB_USER'@'%';" >> $init_db_sql_file
-fi
-echo "##[CUSTOM_SQL_END]##" >> $init_db_sql_file
-
-echo "##[CUSTOM_SQL_END]##" >> $init_db_sql_file
-
-mkdir -p $VTDATAROOT/backups
-
-
-export KEYSPACE=$keyspace
-export SHARD=$shard
-export TABLET_ID=$alias
-export TABLET_DIR=$tablet_dir
-export MYSQL_PORT=3306
-export TABLET_ROLE=$tablet_role
-export DB_PORT=${DB_PORT:-3306}
-export DB_HOST=${DB_HOST:-""}
-export DB_NAME=$db_name
-
-# Delete socket files before running mysqlctld if exists.
-# This is the primary reason for unhealthy state on restart.
-# https://github.com/vitessio/vitess/pull/5115/files
-echo "Removing $VTDATAROOT/$tablet_dir/{mysql.sock,mysql.sock.lock}..."
-rm -rf $VTDATAROOT/$tablet_dir/{mysql.sock,mysql.sock.lock}
-
-# Create mysql instances
-# Do not create mysql instance for primary if connecting to external mysql database
-if [[ $tablet_role != "externalprimary" ]]; then
- echo "Initing mysql for tablet: $uid role: $role external: $external.. "
- $VTROOT/bin/mysqlctld \
- --init_db_sql_file=$init_db_sql_file \
- --logtostderr=true \
- --tablet_uid=$uid \
- &
-fi
-
-sleep $sleeptime
-
-# Create the cell
-# https://vitess.io/blog/2020-04-27-life-of-a-cluster/
-$VTROOT/bin/vtctlclient --server vtctld:$GRPC_PORT -- AddCellInfo --root vitess/$CELL --server_address consul1:8500 $CELL || true
-
-#Populate external db conditional args
-if [ $tablet_role = "externalprimary" ]; then
- echo "Setting external db args for primary: $DB_NAME"
- external_db_args="--db_host $DB_HOST \
- --db_port $DB_PORT \
- --init_db_name_override $DB_NAME \
- --init_tablet_type $tablet_type \
- --mycnf_server_id $uid \
- --db_app_user $DB_USER \
- --db_app_password $DB_PASS \
- --db_allprivs_user $DB_USER \
- --db_allprivs_password $DB_PASS \
- --db_appdebug_user $DB_USER \
- --db_appdebug_password $DB_PASS \
- --db_dba_user $DB_USER \
- --db_dba_password $DB_PASS \
- --db_filtered_user $DB_USER \
- --db_filtered_password $DB_PASS \
- --db_repl_user $DB_USER \
- --db_repl_password $DB_PASS \
- --enable_replication_reporter=false \
- --enforce_strict_trans_tables=false \
- --track_schema_versions=true \
- --watch_replication_stream=true"
-else
- external_db_args="--init_db_name_override $DB_NAME \
- --init_tablet_type $tablet_type \
- --enable_replication_reporter=true \
- --restore_from_backup"
-fi
-
-
-echo "Starting vttablet..."
-exec $VTROOT/bin/vttablet \
- $TOPOLOGY_FLAGS \
- --logtostderr=true \
- --tablet-path $alias \
- --tablet_hostname "$vthost" \
- --health_check_interval 5s \
- --disable_active_reparents=true \
- --port $web_port \
- --grpc_port $grpc_port \
- --service_map 'grpc-queryservice,grpc-tabletmanager,grpc-updatestream' \
- --init_keyspace $keyspace \
- --init_shard $shard \
- --backup_storage_implementation file \
- --file_backup_storage_root $VTDATAROOT/backups \
- --queryserver-config-schema-reload-time 60 \
- $external_db_args
diff --git a/vitess-mixin/go.mod b/vitess-mixin/go.mod
deleted file mode 100644
index 20d3f33eed8..00000000000
--- a/vitess-mixin/go.mod
+++ /dev/null
@@ -1,132 +0,0 @@
-module vitess-mixin
-
-go 1.22.5
-
-require (
- github.com/evanphx/json-patch v5.9.0+incompatible
- github.com/google/go-jsonnet v0.16.0
- github.com/jsonnet-bundler/jsonnet-bundler v0.4.0
- github.com/krishicks/yaml-patch v0.0.10
- // Believe it or not, this is actually version 2.13.1
- // See https://github.com/prometheus/prometheus/issues/5590#issuecomment-546368944
- github.com/prometheus/prometheus v1.8.2-0.20191017095924-6f92ce560538
- github.com/stretchr/testify v1.9.0
- vitess.io/vitess v0.19.4
-)
-
-require (
- cloud.google.com/go/compute v1.25.0 // indirect
- cloud.google.com/go/compute/metadata v0.2.3 // indirect
- github.com/Azure/azure-sdk-for-go v23.2.0+incompatible // indirect
- github.com/Azure/go-autorest v14.2.0+incompatible // indirect
- github.com/Azure/go-autorest/autorest v0.11.1 // indirect
- github.com/Azure/go-autorest/autorest/adal v0.9.23 // indirect
- github.com/Azure/go-autorest/autorest/date v0.3.0 // indirect
- github.com/Azure/go-autorest/autorest/to v0.4.0 // indirect
- github.com/Azure/go-autorest/autorest/validation v0.3.1 // indirect
- github.com/Azure/go-autorest/logger v0.2.1 // indirect
- github.com/Azure/go-autorest/tracing v0.6.0 // indirect
- github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 // indirect
- github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 // indirect
- github.com/armon/go-metrics v0.4.1 // indirect
- github.com/aws/aws-sdk-go v1.50.32 // indirect
- github.com/beorn7/perks v1.0.1 // indirect
- github.com/cespare/xxhash v1.1.0 // indirect
- github.com/cespare/xxhash/v2 v2.2.0 // indirect
- github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect
- github.com/edsrzf/mmap-go v1.0.0 // indirect
- github.com/emicklei/go-restful/v3 v3.10.1 // indirect
- github.com/fatih/color v1.16.0 // indirect
- github.com/felixge/httpsnoop v1.0.4 // indirect
- github.com/go-kit/kit v0.9.0 // indirect
- github.com/go-logfmt/logfmt v0.5.1 // indirect
- github.com/go-logr/logr v1.4.1 // indirect
- github.com/go-logr/stdr v1.2.2 // indirect
- github.com/go-openapi/jsonpointer v0.19.6 // indirect
- github.com/go-openapi/jsonreference v0.20.2 // indirect
- github.com/go-openapi/swag v0.22.3 // indirect
- github.com/gogo/protobuf v1.3.2 // indirect
- github.com/golang-jwt/jwt/v4 v4.5.0 // indirect
- github.com/golang/glog v1.2.1 // indirect
- github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect
- github.com/golang/protobuf v1.5.4 // indirect
- github.com/golang/snappy v0.0.4 // indirect
- github.com/google/gnostic v0.6.9 // indirect
- github.com/google/go-cmp v0.6.0 // indirect
- github.com/google/gofuzz v1.2.0 // indirect
- github.com/google/pprof v0.0.0-20190723021845-34ac40c74b70 // indirect
- github.com/google/s2a-go v0.1.7 // indirect
- github.com/google/uuid v1.6.0 // indirect
- github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect
- github.com/googleapis/gax-go/v2 v2.12.2 // indirect
- github.com/gophercloud/gophercloud v0.3.0 // indirect
- github.com/hashicorp/consul/api v1.28.2 // indirect
- github.com/hashicorp/errwrap v1.1.0 // indirect
- github.com/hashicorp/go-cleanhttp v0.5.2 // indirect
- github.com/hashicorp/go-hclog v1.6.2 // indirect
- github.com/hashicorp/go-immutable-radix v1.3.1 // indirect
- github.com/hashicorp/go-multierror v1.1.1 // indirect
- github.com/hashicorp/go-rootcerts v1.0.2 // indirect
- github.com/hashicorp/golang-lru v1.0.2 // indirect
- github.com/hashicorp/serf v0.10.1 // indirect
- github.com/jmespath/go-jmespath v0.4.0 // indirect
- github.com/josharian/intern v1.0.0 // indirect
- github.com/jpillora/backoff v1.0.0 // indirect
- github.com/json-iterator/go v1.1.12 // indirect
- github.com/mailru/easyjson v0.7.7 // indirect
- github.com/mattn/go-colorable v0.1.13 // indirect
- github.com/mattn/go-isatty v0.0.20 // indirect
- github.com/miekg/dns v1.1.41 // indirect
- github.com/mitchellh/go-homedir v1.1.0 // indirect
- github.com/mitchellh/mapstructure v1.5.0 // indirect
- github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect
- github.com/modern-go/reflect2 v1.0.2 // indirect
- github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect
- github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect
- github.com/nxadm/tail v1.4.11 // indirect
- github.com/oklog/ulid v1.3.1 // indirect
- github.com/opentracing/opentracing-go v1.2.0 // indirect
- github.com/pkg/errors v0.9.1 // indirect
- github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
- github.com/prometheus/client_golang v1.19.0 // indirect
- github.com/prometheus/client_model v0.6.0 // indirect
- github.com/prometheus/common v0.49.0 // indirect
- github.com/prometheus/procfs v0.12.0 // indirect
- github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75 // indirect
- github.com/spf13/pflag v1.0.5 // indirect
- go.opencensus.io v0.24.0 // indirect
- go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect
- go.opentelemetry.io/otel v1.24.0 // indirect
- go.opentelemetry.io/otel/metric v1.24.0 // indirect
- go.opentelemetry.io/otel/trace v1.24.0 // indirect
- golang.org/x/crypto v0.23.0 // indirect
- golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 // indirect
- golang.org/x/net v0.25.0 // indirect
- golang.org/x/oauth2 v0.18.0 // indirect
- golang.org/x/sync v0.6.0 // indirect
- golang.org/x/sys v0.20.0 // indirect
- golang.org/x/term v0.20.0 // indirect
- golang.org/x/text v0.15.0 // indirect
- golang.org/x/time v0.5.0 // indirect
- google.golang.org/api v0.168.0 // indirect
- google.golang.org/appengine v1.6.8 // indirect
- google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8 // indirect
- google.golang.org/grpc v1.62.1 // indirect
- google.golang.org/protobuf v1.33.0 // indirect
- gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect
- gopkg.in/fsnotify/fsnotify.v1 v1.4.7 // indirect
- gopkg.in/inf.v0 v0.9.1 // indirect
- gopkg.in/yaml.v2 v2.4.0 // indirect
- gopkg.in/yaml.v3 v3.0.1 // indirect
- k8s.io/api v0.26.1 // indirect
- k8s.io/apimachinery v0.26.1 // indirect
- k8s.io/client-go v0.26.1 // indirect
- k8s.io/klog/v2 v2.90.0 // indirect
- k8s.io/kube-openapi v0.0.0-20230202010329-39b3636cbaa3 // indirect
- k8s.io/utils v0.0.0-20230115233650-391b47cb4029 // indirect
- sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect
- sigs.k8s.io/structured-merge-diff/v4 v4.2.3 // indirect
- sigs.k8s.io/yaml v1.4.0 // indirect
-)
-
-replace k8s.io/client-go v2.0.0-alpha.0.0.20181121191925-a47917edff34+incompatible => k8s.io/client-go v2.0.0-alpha.1+incompatible
diff --git a/vitess-mixin/go.sum b/vitess-mixin/go.sum
deleted file mode 100644
index 73171b17f6a..00000000000
--- a/vitess-mixin/go.sum
+++ /dev/null
@@ -1,908 +0,0 @@
-cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw=
-cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU=
-cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU=
-cloud.google.com/go/compute v1.25.0 h1:H1/4SqSUhjPFE7L5ddzHOfY2bCAvjwNRZPNl6Ni5oYU=
-cloud.google.com/go/compute v1.25.0/go.mod h1:GR7F0ZPZH8EhChlMo9FkLd7eUTwEymjqQagxzilIxIE=
-cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY=
-cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA=
-contrib.go.opencensus.io/exporter/ocagent v0.6.0/go.mod h1:zmKjrJcdo0aYcVS7bmEeSEBLPA9YJp5bjrofdU3pIXs=
-github.com/Azure/azure-sdk-for-go v23.2.0+incompatible h1:bch1RS060vGpHpY3zvQDV4rOiRw25J1zmR/B9a76aSA=
-github.com/Azure/azure-sdk-for-go v23.2.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc=
-github.com/Azure/go-autorest v11.1.2+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24=
-github.com/Azure/go-autorest v11.2.8+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24=
-github.com/Azure/go-autorest v14.2.0+incompatible h1:V5VMDjClD3GiElqLWO7mz2MxNAK/vTfRHdAubSIPRgs=
-github.com/Azure/go-autorest v14.2.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24=
-github.com/Azure/go-autorest/autorest v0.11.1 h1:eVvIXUKiTgv++6YnWb42DUA1YL7qDugnKP0HljexdnQ=
-github.com/Azure/go-autorest/autorest v0.11.1/go.mod h1:JFgpikqFJ/MleTTxwepExTKnFUKKszPS8UavbQYUMuw=
-github.com/Azure/go-autorest/autorest/adal v0.9.0/go.mod h1:/c022QCutn2P7uY+/oQWWNcK9YU+MH96NgK+jErpbcg=
-github.com/Azure/go-autorest/autorest/adal v0.9.23 h1:Yepx8CvFxwNKpH6ja7RZ+sKX+DWYNldbLiALMC3BTz8=
-github.com/Azure/go-autorest/autorest/adal v0.9.23/go.mod h1:5pcMqFkdPhviJdlEy3kC/v1ZLnQl0MH6XA5YCcMhy4c=
-github.com/Azure/go-autorest/autorest/date v0.3.0 h1:7gUk1U5M/CQbp9WoqinNzJar+8KY+LPI6wiWrP/myHw=
-github.com/Azure/go-autorest/autorest/date v0.3.0/go.mod h1:BI0uouVdmngYNUzGWeSYnokU+TrmwEsOqdt8Y6sso74=
-github.com/Azure/go-autorest/autorest/mocks v0.4.0/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k=
-github.com/Azure/go-autorest/autorest/mocks v0.4.1 h1:K0laFcLE6VLTOwNgSxaGbUcLPuGXlNkbVvq4cW4nIHk=
-github.com/Azure/go-autorest/autorest/mocks v0.4.1/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k=
-github.com/Azure/go-autorest/autorest/to v0.4.0 h1:oXVqrxakqqV1UZdSazDOPOLvOIz+XA683u8EctwboHk=
-github.com/Azure/go-autorest/autorest/to v0.4.0/go.mod h1:fE8iZBn7LQR7zH/9XU2NcPR4o9jEImooCeWJcYV/zLE=
-github.com/Azure/go-autorest/autorest/validation v0.3.1 h1:AgyqjAd94fwNAoTjl/WQXg4VvFeRFpO+UhNyRXqF1ac=
-github.com/Azure/go-autorest/autorest/validation v0.3.1/go.mod h1:yhLgjC0Wda5DYXl6JAsWyUe4KVNffhoDhG0zVzUMo3E=
-github.com/Azure/go-autorest/logger v0.2.0/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8=
-github.com/Azure/go-autorest/logger v0.2.1 h1:IG7i4p/mDa2Ce4TRyAO8IHnVhAVF3RFU+ZtXWSmf4Tg=
-github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZmbF5NWuPV8+WeEW8=
-github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo=
-github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU=
-github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
-github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
-github.com/DataDog/datadog-go v2.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
-github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
-github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ=
-github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
-github.com/OneOfOne/xxhash v1.2.5 h1:zl/OfRA6nftbBK9qTohYBJ5xvw6C/oNKizR7cZGl3cI=
-github.com/OneOfOne/xxhash v1.2.5/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q=
-github.com/PuerkitoBio/purell v1.0.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
-github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
-github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
-github.com/PuerkitoBio/urlesc v0.0.0-20160726150825-5bd2802263f2/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
-github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
-github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
-github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 h1:JYp7IbQjafoB+tBA3gMyHYHrpOtNuDiK/uB5uXxq5wM=
-github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc=
-github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
-github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
-github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 h1:s6gZFSlWYmbqAuRjVTiNNhvNRfY2Wxp9nhfyel4rklc=
-github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE=
-github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
-github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
-github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY=
-github.com/armon/go-metrics v0.0.0-20190430140413-ec5e00d3c878/go.mod h1:3AMJUQhVx52RsWOnlkpikZr01T/yAVN2gn0861vByNg=
-github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA=
-github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4=
-github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
-github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
-github.com/asaskevich/govalidator v0.0.0-20180720115003-f9ffefc3facf/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY=
-github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY=
-github.com/aws/aws-sdk-go v1.23.12/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
-github.com/aws/aws-sdk-go v1.50.32 h1:POt81DvegnpQKM4DMDLlHz1CO6OBnEoQ1gRhYFd7QRY=
-github.com/aws/aws-sdk-go v1.50.32/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk=
-github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
-github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
-github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
-github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw=
-github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
-github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0=
-github.com/campoy/embedmd v1.0.0/go.mod h1:oxyr9RCiSXg0M3VJ3ks0UGfp98BpSSGr0kpiX3MzVl8=
-github.com/cenkalti/backoff v0.0.0-20181003080854-62661b46c409/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
-github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
-github.com/cespare/xxhash v0.0.0-20181017004759-096ff4a8a059/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
-github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko=
-github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc=
-github.com/cespare/xxhash/v2 v2.1.0/go.mod h1:dgIUBU3pDso/gPgZ1osOZ0iQf77oPR28Tjxl5dIMyVM=
-github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
-github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44=
-github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
-github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag=
-github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I=
-github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
-github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
-github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
-github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
-github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
-github.com/davecgh/go-spew v0.0.0-20151105211317-5215b55f46b2/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
-github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
-github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
-github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM=
-github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
-github.com/dgrijalva/jwt-go v0.0.0-20160705203006-01aeca54ebda/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
-github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
-github.com/dgryski/go-sip13 v0.0.0-20190329191031-25c5027a8c7b/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
-github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
-github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
-github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM=
-github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE=
-github.com/edsrzf/mmap-go v1.0.0 h1:CEBF7HpRnUCSJgGUb5h1Gm7e3VkmVDrR8lvWVLtrOFw=
-github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M=
-github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc=
-github.com/emicklei/go-restful v0.0.0-20170410110728-ff4f55a20633/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs=
-github.com/emicklei/go-restful/v3 v3.10.1 h1:rc42Y5YTp7Am7CS630D7JmhRjq4UlEUuEKfrDac4bSQ=
-github.com/emicklei/go-restful/v3 v3.10.1/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc=
-github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
-github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
-github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
-github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
-github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ=
-github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/evanphx/json-patch v0.0.0-20190203023257-5858425f7550/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
-github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
-github.com/evanphx/json-patch v4.5.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
-github.com/evanphx/json-patch v5.9.0+incompatible h1:fBXyNpNMuTTDdquAq/uisOr2lShz4oaXpDTX2bLe7ls=
-github.com/evanphx/json-patch v5.9.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk=
-github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
-github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU=
-github.com/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk=
-github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM=
-github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE=
-github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg=
-github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
-github.com/flowstack/go-jsonschema v0.1.1/go.mod h1:yL7fNggx1o8rm9RlgXv7hTBWxdBM0rVwpMwimd3F3N0=
-github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
-github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw=
-github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA=
-github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM=
-github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
-github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
-github.com/globalsign/mgo v0.0.0-20180905125535-1ca0a4f7cbcb/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q=
-github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q=
-github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
-github.com/go-kit/kit v0.9.0 h1:wDJmvq38kDhkVxi50ni9ykkdUr1PKgqKOoi01fa0Mdk=
-github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
-github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
-github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
-github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA=
-github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs=
-github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas=
-github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
-github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
-github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ=
-github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
-github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag=
-github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE=
-github.com/go-openapi/analysis v0.0.0-20180825180245-b006789cd277/go.mod h1:k70tL6pCuVxPJOHXQ+wIac1FUrvNkHolPie/cLEU6hI=
-github.com/go-openapi/analysis v0.17.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik=
-github.com/go-openapi/analysis v0.17.2/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik=
-github.com/go-openapi/analysis v0.18.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik=
-github.com/go-openapi/analysis v0.19.2/go.mod h1:3P1osvZa9jKjb8ed2TPng3f0i/UY9snX6gxi44djMjk=
-github.com/go-openapi/analysis v0.19.4/go.mod h1:3P1osvZa9jKjb8ed2TPng3f0i/UY9snX6gxi44djMjk=
-github.com/go-openapi/errors v0.17.0/go.mod h1:LcZQpmvG4wyF5j4IhA73wkLFQg+QJXOQHVjmcZxhka0=
-github.com/go-openapi/errors v0.17.2/go.mod h1:LcZQpmvG4wyF5j4IhA73wkLFQg+QJXOQHVjmcZxhka0=
-github.com/go-openapi/errors v0.18.0/go.mod h1:LcZQpmvG4wyF5j4IhA73wkLFQg+QJXOQHVjmcZxhka0=
-github.com/go-openapi/errors v0.19.2/go.mod h1:qX0BLWsyaKfvhluLejVpVNwNRdXZhEbTA4kxxpKBC94=
-github.com/go-openapi/jsonpointer v0.0.0-20160704185906-46af16f9f7b1/go.mod h1:+35s3my2LFTysnkMfxsJBAMHj/DoqoB9knIWoYG/Vk0=
-github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M=
-github.com/go-openapi/jsonpointer v0.17.2/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M=
-github.com/go-openapi/jsonpointer v0.18.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M=
-github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg=
-github.com/go-openapi/jsonpointer v0.19.6 h1:eCs3fxoIi3Wh6vtgmLTOjdhSpiqphQ+DaPn38N2ZdrE=
-github.com/go-openapi/jsonpointer v0.19.6/go.mod h1:osyAmYz/mB/C3I+WsTTSgw1ONzaLJoLCyoi6/zppojs=
-github.com/go-openapi/jsonreference v0.0.0-20160704190145-13c6e3589ad9/go.mod h1:W3Z9FmVs9qj+KR4zFKmDPGiLdk1D9Rlm7cyMvf57TTg=
-github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I=
-github.com/go-openapi/jsonreference v0.17.2/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I=
-github.com/go-openapi/jsonreference v0.18.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I=
-github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc=
-github.com/go-openapi/jsonreference v0.20.2 h1:3sVjiK66+uXK/6oQ8xgcRKcFgQ5KXa2KvnJRumpMGbE=
-github.com/go-openapi/jsonreference v0.20.2/go.mod h1:Bl1zwGIM8/wsvqjsOQLJ/SH+En5Ap4rVB5KVcIDZG2k=
-github.com/go-openapi/loads v0.17.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU=
-github.com/go-openapi/loads v0.17.2/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU=
-github.com/go-openapi/loads v0.18.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU=
-github.com/go-openapi/loads v0.19.0/go.mod h1:72tmFy5wsWx89uEVddd0RjRWPZm92WRLhf7AC+0+OOU=
-github.com/go-openapi/loads v0.19.2/go.mod h1:QAskZPMX5V0C2gvfkGZzJlINuP7Hx/4+ix5jWFxsNPs=
-github.com/go-openapi/runtime v0.0.0-20180920151709-4f900dc2ade9/go.mod h1:6v9a6LTXWQCdL8k1AO3cvqx5OtZY/Y9wKTgaoP6YRfA=
-github.com/go-openapi/runtime v0.18.0/go.mod h1:uI6pHuxWYTy94zZxgcwJkUWa9wbIlhteGfloI10GD4U=
-github.com/go-openapi/runtime v0.19.0/go.mod h1:OwNfisksmmaZse4+gpV3Ne9AyMOlP1lt4sK4FXt0O64=
-github.com/go-openapi/runtime v0.19.3/go.mod h1:X277bwSUBxVlCYR3r7xgZZGKVvBd/29gLDlFGtJ8NL4=
-github.com/go-openapi/spec v0.0.0-20160808142527-6aced65f8501/go.mod h1:J8+jY1nAiCcj+friV/PDoE1/3eeccG9LYBs0tYvLOWc=
-github.com/go-openapi/spec v0.17.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI=
-github.com/go-openapi/spec v0.17.2/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI=
-github.com/go-openapi/spec v0.18.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI=
-github.com/go-openapi/spec v0.19.2/go.mod h1:sCxk3jxKgioEJikev4fgkNmwS+3kuYdJtcsZsD5zxMY=
-github.com/go-openapi/strfmt v0.17.0/go.mod h1:P82hnJI0CXkErkXi8IKjPbNBM6lV6+5pLP5l494TcyU=
-github.com/go-openapi/strfmt v0.17.2/go.mod h1:P82hnJI0CXkErkXi8IKjPbNBM6lV6+5pLP5l494TcyU=
-github.com/go-openapi/strfmt v0.18.0/go.mod h1:P82hnJI0CXkErkXi8IKjPbNBM6lV6+5pLP5l494TcyU=
-github.com/go-openapi/strfmt v0.19.0/go.mod h1:+uW+93UVvGGq2qGaZxdDeJqSAqBqBdl+ZPMF/cC8nDY=
-github.com/go-openapi/strfmt v0.19.2/go.mod h1:0yX7dbo8mKIvc3XSKp7MNfxw4JytCfCD6+bY1AVL9LU=
-github.com/go-openapi/swag v0.0.0-20160704191624-1d0bd113de87/go.mod h1:DXUve3Dpr1UfpPtxFw+EFuQ41HhCWZfha5jSVRG7C7I=
-github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg=
-github.com/go-openapi/swag v0.17.2/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg=
-github.com/go-openapi/swag v0.18.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg=
-github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk=
-github.com/go-openapi/swag v0.19.4/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk=
-github.com/go-openapi/swag v0.22.3 h1:yMBqmnQ0gyZvEb/+KzuWZOXgllrXT4SADYbvDaXHv/g=
-github.com/go-openapi/swag v0.22.3/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14=
-github.com/go-openapi/validate v0.17.2/go.mod h1:Uh4HdOzKt19xGIGm1qHf/ofbX1YQ4Y+MYsct2VUrAJ4=
-github.com/go-openapi/validate v0.18.0/go.mod h1:Uh4HdOzKt19xGIGm1qHf/ofbX1YQ4Y+MYsct2VUrAJ4=
-github.com/go-openapi/validate v0.19.2/go.mod h1:1tRCw7m3jtI8eNWEEliiAqUIcBztB2KDnRCRMUi7GTA=
-github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk=
-github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
-github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
-github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
-github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
-github.com/gogo/protobuf v1.2.2-0.20190723190241-65acae22fc9d/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
-github.com/gogo/protobuf v1.2.2-0.20190730201129-28a6bbf47e48/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
-github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q=
-github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q=
-github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg=
-github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0=
-github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
-github.com/golang/glog v1.2.1 h1:OptwRhECazUx5ix5TTWC3EZhsZEHWcYWY4FQHTIubm4=
-github.com/golang/glog v1.2.1/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w=
-github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
-github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
-github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
-github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE=
-github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
-github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
-github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
-github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y=
-github.com/golang/protobuf v0.0.0-20161109072736-4bd1920723d7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
-github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw=
-github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8=
-github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA=
-github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs=
-github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
-github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
-github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
-github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
-github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
-github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
-github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
-github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek=
-github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps=
-github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM=
-github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
-github.com/google/btree v0.0.0-20160524151835-7d79101e329e/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
-github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
-github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
-github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4=
-github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA=
-github.com/google/gnostic v0.6.9 h1:ZK/5VhkoX835RikCHpSUJV9a+S3e1zLh59YnyWeBW+0=
-github.com/google/gnostic v0.6.9/go.mod h1:Nm8234We1lq6iB9OmlgNv3nH91XLLVZHCDayfA3xq+E=
-github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M=
-github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
-github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU=
-github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
-github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
-github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI=
-github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY=
-github.com/google/go-jsonnet v0.16.0 h1:Nb4EEOp+rdeGGyB1rQ5eisgSAqrTnhf9ip+X6lzZbY0=
-github.com/google/go-jsonnet v0.16.0/go.mod h1:sOcuej3UW1vpPTZOr8L7RQimqai1a57bt5j22LzGZCw=
-github.com/google/gofuzz v0.0.0-20161122191042-44d81051d367/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
-github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
-github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
-github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0=
-github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
-github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
-github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
-github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
-github.com/google/pprof v0.0.0-20190723021845-34ac40c74b70 h1:XTnP8fJpa4Kvpw2qARB4KS9izqxPS0Sd92cDlY3uk+w=
-github.com/google/pprof v0.0.0-20190723021845-34ac40c74b70/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc=
-github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o=
-github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw=
-github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0=
-github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
-github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs=
-github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0=
-github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
-github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk=
-github.com/googleapis/gax-go/v2 v2.12.2 h1:mhN09QQW1jEWeMF74zGR81R30z4VJzjZsfkUhuHF+DA=
-github.com/googleapis/gax-go/v2 v2.12.2/go.mod h1:61M8vcyyXR2kqKFxKrfA22jaA8JGF7Dc8App1U3H6jc=
-github.com/googleapis/gnostic v0.0.0-20170426233943-68f4ded48ba9/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
-github.com/googleapis/gnostic v0.0.0-20170729233727-0c5108395e2d/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
-github.com/googleapis/gnostic v0.3.0/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
-github.com/gophercloud/gophercloud v0.0.0-20190126172459-c818fa66e4c8/go.mod h1:3WdhXV3rUYy9p6AUW8d94kr+HS62Y4VL9mBnFxsD8q4=
-github.com/gophercloud/gophercloud v0.3.0 h1:6sjpKIpVwRIIwmcEGp+WwNovNsem+c+2vm6oxshRpL8=
-github.com/gophercloud/gophercloud v0.3.0/go.mod h1:vxM41WHh5uqHVBMZHzuwNOHh8XEoIEcSTewFxm1c5g8=
-github.com/gregjones/httpcache v0.0.0-20170728041850-787624de3eb7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA=
-github.com/grpc-ecosystem/grpc-gateway v1.9.4/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
-github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
-github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
-github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q=
-github.com/hashicorp/consul/api v1.28.2 h1:mXfkRHrpHN4YY3RqL09nXU1eHKLNiuAN4kHvDQ16k/8=
-github.com/hashicorp/consul/api v1.28.2/go.mod h1:KyzqzgMEya+IZPcD65YFoOVAgPpbfERu4I/tzG6/ueE=
-github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8=
-github.com/hashicorp/consul/sdk v0.16.0 h1:SE9m0W6DEfgIVCJX7xU+iv/hUl4m/nxqMTnCdMxDpJ8=
-github.com/hashicorp/consul/sdk v0.16.0/go.mod h1:7pxqqhqoaPqnBnzXD1StKed62LqJeClzVsUEy85Zr0A=
-github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
-github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I=
-github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4=
-github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
-github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80=
-github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ=
-github.com/hashicorp/go-cleanhttp v0.5.2/go.mod h1:kO/YDlP8L1346E6Sodw+PrpBSV4/SoxCXGY6BqNFT48=
-github.com/hashicorp/go-hclog v1.6.2 h1:NOtoftovWkDheyUM/8JW3QMiXyxJK3uHRK7wV04nD2I=
-github.com/hashicorp/go-hclog v1.6.2/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M=
-github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
-github.com/hashicorp/go-immutable-radix v1.1.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
-github.com/hashicorp/go-immutable-radix v1.3.1 h1:DKHmCUm2hRBK510BaiZlwvpD40f8bJFeZnpfm2KLowc=
-github.com/hashicorp/go-immutable-radix v1.3.1/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60=
-github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
-github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI=
-github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM=
-github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk=
-github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA=
-github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo=
-github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM=
-github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs=
-github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU=
-github.com/hashicorp/go-rootcerts v1.0.1/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8=
-github.com/hashicorp/go-rootcerts v1.0.2 h1:jzhAVGtqPKbwpyCPELlgNWhE1znq+qwJtW5Oi2viEzc=
-github.com/hashicorp/go-rootcerts v1.0.2/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8=
-github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU=
-github.com/hashicorp/go-sockaddr v1.0.2 h1:ztczhD1jLxIRjVejw8gFomI1BQZOe2WoVOu0SyteCQc=
-github.com/hashicorp/go-sockaddr v1.0.2/go.mod h1:rB4wwRAUzs07qva3c5SdrY/NEtAUjGlgmH/UkBUC97A=
-github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4=
-github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8=
-github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
-github.com/hashicorp/go-version v1.6.0 h1:feTTfFNnjP967rlCxM/I9g701jU+RN74YKx2mOkIeek=
-github.com/hashicorp/go-version v1.6.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA=
-github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90=
-github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
-github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
-github.com/hashicorp/golang-lru v0.5.3/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
-github.com/hashicorp/golang-lru v1.0.2 h1:dV3g9Z/unq5DpblPpw+Oqcv4dU/1omnb4Ok8iPY6p1c=
-github.com/hashicorp/golang-lru v1.0.2/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4=
-github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64=
-github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
-github.com/hashicorp/mdns v1.0.4/go.mod h1:mtBihi+LeNXGtG8L9dX59gAEa12BDtBQSp4v/YAJqrc=
-github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
-github.com/hashicorp/memberlist v0.1.4/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
-github.com/hashicorp/memberlist v0.5.0 h1:EtYPN8DpAURiapus508I4n9CzHs2W+8NZGbmmR/prTM=
-github.com/hashicorp/memberlist v0.5.0/go.mod h1:yvyXLpo0QaGE59Y7hDTsTzDD25JYBZ4mHgHUZ8lrOI0=
-github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
-github.com/hashicorp/serf v0.8.3/go.mod h1:UpNcs7fFbpKIyZaUuSW6EPiH+eZC7OuyFD+wc1oal+k=
-github.com/hashicorp/serf v0.10.1 h1:Z1H2J60yRKvfDYAOZLd2MU0ND4AH/WDz7xYHDWQsIPY=
-github.com/hashicorp/serf v0.10.1/go.mod h1:yL2t6BqATOLGc5HF7qbFkTfXoPIY0WZdWHfEvMqbG+4=
-github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU=
-github.com/imdario/mergo v0.3.5/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA=
-github.com/influxdata/influxdb v1.7.7/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY=
-github.com/jessevdk/go-flags v0.0.0-20180331124232-1c38ed7ad0cc/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
-github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
-github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg=
-github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
-github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8=
-github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U=
-github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY=
-github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y=
-github.com/jpillora/backoff v0.0.0-20180909062703-3050d21c67d7/go.mod h1:2iMrUgbbvHEiQClaW2NsSzMyGHqN+rDFqY705q49KG0=
-github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA=
-github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4=
-github.com/json-iterator/go v0.0.0-20180612202835-f2b4162afba3/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
-github.com/json-iterator/go v0.0.0-20180701071628-ab8a2e0c74be/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
-github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU=
-github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
-github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4=
-github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM=
-github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo=
-github.com/jsonnet-bundler/jsonnet-bundler v0.4.0 h1:4BKZ6LDqPc2wJDmaKnmYD/vDjUptJtnUpai802MibFc=
-github.com/jsonnet-bundler/jsonnet-bundler v0.4.0/go.mod h1:/by7P/OoohkI3q4CgSFqcoFsVY+IaNbzOVDknEsKDeU=
-github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU=
-github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w=
-github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
-github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
-github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
-github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
-github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
-github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc=
-github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
-github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
-github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
-github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
-github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk=
-github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
-github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA=
-github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI=
-github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
-github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
-github.com/krishicks/yaml-patch v0.0.10 h1:H4FcHpnNwVmw8u0MjPRjWyIXtco6zM2F78t+57oNM3E=
-github.com/krishicks/yaml-patch v0.0.10/go.mod h1:Sm5TchwZS6sm7RJoyg87tzxm2ZcKzdRE4Q7TjNhPrME=
-github.com/kylelemons/godebug v0.0.0-20160406211939-eadb3ce320cb/go.mod h1:B69LEHPfb2qLo0BaaOLcbitczOKLWTsrBG9LczfCD4k=
-github.com/mailru/easyjson v0.0.0-20160728113105-d5b7844b561a/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
-github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
-github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
-github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
-github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc=
-github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0=
-github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc=
-github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU=
-github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE=
-github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
-github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc=
-github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4=
-github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA=
-github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg=
-github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4=
-github.com/mattn/go-isatty v0.0.6/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
-github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
-github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE=
-github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
-github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
-github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM=
-github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY=
-github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y=
-github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
-github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
-github.com/miekg/dns v1.1.15/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
-github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
-github.com/miekg/dns v1.1.41 h1:WMszZWJG0XmzbK9FEmzH2TVcqYzFesusSIB41b8KHxY=
-github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI=
-github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
-github.com/mitchellh/cli v1.1.0/go.mod h1:xcISNoH86gajksDmfB23e/pu+B+GeFRMYmoHXxx3xhI=
-github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
-github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y=
-github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
-github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
-github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo=
-github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
-github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
-github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
-github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
-github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY=
-github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo=
-github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
-github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg=
-github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
-github.com/modern-go/reflect2 v0.0.0-20180320133207-05fbef0ca5da/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
-github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
-github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
-github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M=
-github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk=
-github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ=
-github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA=
-github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ=
-github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
-github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU=
-github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U=
-github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw=
-github.com/nxadm/tail v1.4.11 h1:8feyoE3OzPrcshW5/MJ4sGESc5cqmGkGCWlco4l0bqY=
-github.com/nxadm/tail v1.4.11/go.mod h1:OTaG3NK980DZzxbRq6lEuzgU+mug70nY11sMd4JXXHc=
-github.com/oklog/run v1.0.0/go.mod h1:dlhp/R75TPv97u0XWUtDeV/lRKWPKSdTuV0TZvrmrQA=
-github.com/oklog/ulid v0.0.0-20170117200651-66bb6560562f/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
-github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4=
-github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U=
-github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
-github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
-github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
-github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE=
-github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU=
-github.com/onsi/ginkgo/v2 v2.4.0 h1:+Ig9nvqgS5OBSACXNk15PLdp0U9XPYROt9CFzVdFGIs=
-github.com/onsi/ginkgo/v2 v2.4.0/go.mod h1:iHkDK1fKGcBoEHT5W7YBq4RFWaQulw+caOMkAt4OrFo=
-github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
-github.com/onsi/gomega v0.0.0-20190113212917-5533ce8a0da3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
-github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
-github.com/onsi/gomega v1.23.0 h1:/oxKu9c2HVap+F3PfKort2Hw5DEU+HGlW8n+tguWsys=
-github.com/onsi/gomega v1.23.0/go.mod h1:Z/NWtiqwBrwUt4/2loMmHL63EDLnYHmVbuBpDr2vQAg=
-github.com/opentracing-contrib/go-stdlib v0.0.0-20190519235532-cf7a6c988dc9/go.mod h1:PLldrQSroqzH70Xl+1DQcGnefIbqsKR7UDaiux3zV+w=
-github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
-github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs=
-github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc=
-github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
-github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY=
-github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
-github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k=
-github.com/peterbourgon/diskv v2.0.1+incompatible/go.mod h1:uqqh8zWWbv1HBMNONnaR/tNboyR3/BZd58JJSHlUSCU=
-github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
-github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
-github.com/pmezard/go-difflib v0.0.0-20151028094244-d8ed2627bdf0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
-github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
-github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U=
-github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
-github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI=
-github.com/posener/complete v1.2.3/go.mod h1:WZIdtGGp+qx0sLrYKtIRAruyNpv6hFCicSgv7Sy7s/s=
-github.com/prometheus/alertmanager v0.18.0/go.mod h1:WcxHBl40VSPuOaqWae6l6HpnEOVRIycEJ7i9iYkadEE=
-github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw=
-github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM=
-github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo=
-github.com/prometheus/client_golang v1.2.0/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U=
-github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU=
-github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU=
-github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k=
-github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
-github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
-github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos=
-github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8=
-github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
-github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4=
-github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA=
-github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4=
-github.com/prometheus/common v0.49.0 h1:ToNTdK4zSnPVJmh698mGFkDor9wBI/iGaJy5dbH1EgI=
-github.com/prometheus/common v0.49.0/go.mod h1:Kxm+EULxRbUkjGU6WFsQqo3ORzB4tyKvlWFOE9mB2sE=
-github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
-github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA=
-github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ=
-github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A=
-github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo=
-github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo=
-github.com/prometheus/prometheus v0.0.0-20180315085919-58e2a31db8de/go.mod h1:oAIUtOny2rjMX0OWN5vPR5/q/twIROJvdqnQKDdil/s=
-github.com/prometheus/prometheus v1.8.2-0.20191017095924-6f92ce560538 h1:iyerK9/VU1F02ASqYyIXp60gKxo7ualRoEezXPqbQZE=
-github.com/prometheus/prometheus v1.8.2-0.20191017095924-6f92ce560538/go.mod h1:SgN99nHQ/tVJyAuyLKKz6i2j5cJx3eLy9MCRCPOXqUI=
-github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
-github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
-github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8=
-github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4=
-github.com/rs/cors v1.6.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU=
-github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
-github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
-github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75 h1:cA+Ubq9qEVIQhIWvP2kNuSZ2CmnfBJFSRq+kO1pu2cc=
-github.com/samuel/go-zookeeper v0.0.0-20190810000440-0ceca61e4d75/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
-github.com/satori/go.uuid v0.0.0-20160603004225-b111a074d5ef/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
-github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I=
-github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
-github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0=
-github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM=
-github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg=
-github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg=
-github.com/shurcooL/vfsgen v0.0.0-20180825020608-02ddb050ef6b/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw=
-github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw=
-github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
-github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE=
-github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
-github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
-github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI=
-github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
-github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk=
-github.com/spf13/pflag v0.0.0-20170130214245-9ff6c6923cff/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
-github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA=
-github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
-github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8=
-github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
-github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
-github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE=
-github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
-github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
-github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY=
-github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA=
-github.com/stretchr/testify v0.0.0-20151208002404-e3a8ff8ce365/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
-github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
-github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
-github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
-github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA=
-github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
-github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals=
-github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
-github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
-github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
-github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg=
-github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY=
-github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk=
-github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM=
-github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU=
-github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ=
-github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y=
-github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg=
-github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
-github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
-github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY=
-go.mongodb.org/mongo-driver v1.0.3/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM=
-go.mongodb.org/mongo-driver v1.0.4/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM=
-go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
-go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
-go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0=
-go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo=
-go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 h1:jq9TW8u3so/bN+JPT166wjOI6/vQPF6Xe7nMNIltagk=
-go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1:p8pYQP+m5XfbZm9fxtSKAbM6oIllS7s2AfxrChvc7iw=
-go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo=
-go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo=
-go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI=
-go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco=
-go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI=
-go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU=
-go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI=
-golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
-golang.org/x/crypto v0.0.0-20181025213731-e84da0312774/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
-golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
-golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
-golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190320223903-b7391e95e576/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
-golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20190617133340-57b3e21c3d56/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY=
-golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
-golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
-golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
-golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58=
-golang.org/x/crypto v0.23.0 h1:dIJU/v2J8Mdglj/8rJ6UUOM3Zc9zLZxVZwwxMooUSAI=
-golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8=
-golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
-golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8=
-golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUFH7PGP+OQ6mgDYo3yuQ=
-golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc=
-golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js=
-golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
-golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU=
-golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
-golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
-golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc=
-golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE=
-golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
-golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
-golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4=
-golang.org/x/net v0.0.0-20170114055629-f2499483f923/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20190206173232-65e2d4e15006/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
-golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
-golang.org/x/net v0.0.0-20190320064053-1272bf9dcd53/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
-golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
-golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
-golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
-golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks=
-golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
-golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
-golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
-golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
-golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM=
-golang.org/x/net v0.0.0-20210410081132-afb366fc7cd1/go.mod h1:9tjilg8BloeKEkVJvy7fQ90B1CfIiPueXVOjqfkSzI8=
-golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
-golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
-golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs=
-golang.org/x/net v0.25.0 h1:d/OCCoBEUq33pjydKrGQhw7IlUPI2Oylr+8qLx49kac=
-golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM=
-golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
-golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
-golang.org/x/oauth2 v0.0.0-20190402181905-9f3314589c9a/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
-golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
-golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
-golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI=
-golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8=
-golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
-golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ=
-golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk=
-golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190209173611-3b5209105503/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
-golang.org/x/sys v0.0.0-20190310054646-10058d7d4faa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190321052220-f7bb7a8bee54/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190712062909-fae7ac547cb7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
-golang.org/x/sys v0.20.0 h1:Od9JTbYCk261bKm4M/mw7AklTlFYIa0bIp9BgSm1S8Y=
-golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA=
-golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
-golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
-golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k=
-golang.org/x/term v0.20.0 h1:VnkxpohqXaOBYJtBmEppKUG6mXpi+4O6purfc2+sMhw=
-golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY=
-golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
-golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
-golang.org/x/text v0.3.1-0.20180805044716-cb6730876b98/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
-golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
-golang.org/x/text v0.3.1-0.20181227161524-e6919f6577db/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
-golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
-golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
-golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ=
-golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ=
-golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
-golang.org/x/text v0.15.0 h1:h1V/4gjBv8v9cjcR6+AR5+/cIYK5N/WAgiv4xlsEtAk=
-golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU=
-golang.org/x/time v0.0.0-20161028155119-f51c12702a4d/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
-golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
-golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
-golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk=
-golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM=
-golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20181011042414-1f849cf54d09/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20190118193359-16909d206f00/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
-golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY=
-golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
-golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
-golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
-golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
-golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
-golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q=
-golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
-golang.org/x/tools v0.0.0-20190614205625-5aca471b1d59/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
-golang.org/x/tools v0.0.0-20190617190820-da514acc4774/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
-golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc=
-golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20190918214516-5a1a30219888/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
-golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
-golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
-golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
-golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE=
-google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M=
-google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg=
-google.golang.org/api v0.168.0 h1:MBRe+Ki4mMN93jhDDbpuRLjRddooArz4FeSObvUMmjY=
-google.golang.org/api v0.168.0/go.mod h1:gpNOiMA2tZ4mf5R9Iwf4rK/Dcz0fbdIgWYWVoxmsyLg=
-google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM=
-google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
-google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4=
-google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0=
-google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM=
-google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds=
-google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
-google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
-google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
-google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
-google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
-google.golang.org/genproto v0.0.0-20190716160619-c506a9f90610/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
-google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
-google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
-google.golang.org/genproto v0.0.0-20220107163113-42d7afdf6368/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc=
-google.golang.org/genproto v0.0.0-20240304212257-790db918fca8 h1:Fe8QycXyEd9mJgnwB9kmw00WgB43eQ/xYO5C6gceybQ=
-google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 h1:8eadJkXbwDEMNwcB5O0s5Y5eCfyuCLdvaiOIaGTrWmQ=
-google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8/go.mod h1:O1cOfN1Cy6QEYr7VxtjOyP5AdAuR0aJ/MYZaaof623Y=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8 h1:IR+hp6ypxjH24bkMfEJ0yHR21+gwPWdV+/IBrPQyn3k=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8/go.mod h1:UCOku4NytXMJuLQE5VuqA5lX3PcHCBo8pxNyvkf4xBs=
-google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c=
-google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38=
-google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM=
-google.golang.org/grpc v1.22.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.22.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg=
-google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY=
-google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
-google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0=
-google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc=
-google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU=
-google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34=
-google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk=
-google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE=
-google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8=
-google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0=
-google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
-google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE=
-google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo=
-google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
-google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
-google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU=
-google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c=
-google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
-google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
-google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
-google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI=
-google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos=
-gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc=
-gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
-gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
-gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
-gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q=
-gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys=
-gopkg.in/fsnotify/fsnotify.v1 v1.4.7 h1:XNNYLJHt73EyYiCZi6+xjupS9CpvmiDgjPTAjrBlQbo=
-gopkg.in/fsnotify/fsnotify.v1 v1.4.7/go.mod h1:Fyux9zXlo4rWoMSIzpn9fDAYjalPqJ/K1qJ27s+7ltE=
-gopkg.in/inf.v0 v0.9.0/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
-gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc=
-gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
-gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo=
-gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ=
-gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw=
-gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74=
-gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
-gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
-gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
-gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
-gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
-gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
-honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4=
-k8s.io/api v0.0.0-20190620084959-7cf5895f2711/go.mod h1:TBhBqb1AWbBQbW3XRusr7n7E4v2+5ZY8r8sAMnyFC5A=
-k8s.io/api v0.0.0-20190813020757-36bff7324fb7/go.mod h1:3Iy+myeAORNCLgjd/Xu9ebwN7Vh59Bw0vh9jhoX+V58=
-k8s.io/api v0.26.1 h1:f+SWYiPd/GsiWwVRz+NbFyCgvv75Pk9NK6dlkZgpCRQ=
-k8s.io/api v0.26.1/go.mod h1:xd/GBNgR0f707+ATNyPmQ1oyKSgndzXij81FzWGsejg=
-k8s.io/apimachinery v0.0.0-20190612205821-1799e75a0719/go.mod h1:I4A+glKBHiTgiEjQiCCQfCAIcIMFGt291SmsvcrFzJA=
-k8s.io/apimachinery v0.0.0-20190809020650-423f5d784010/go.mod h1:Waf/xTS2FGRrgXCkO5FP3XxTOWh0qLf2QhL1qFZZ/R8=
-k8s.io/apimachinery v0.26.1 h1:8EZ/eGJL+hY/MYCNwhmDzVqq2lPl3N3Bo8rvweJwXUQ=
-k8s.io/apimachinery v0.26.1/go.mod h1:tnPmbONNJ7ByJNz9+n9kMjNP8ON+1qoAIIC70lztu74=
-k8s.io/client-go v0.0.0-20190620085101-78d2af792bab/go.mod h1:E95RaSlHr79aHaX0aGSwcPNfygDiPKOVXdmivCIZT0k=
-k8s.io/client-go v0.26.1 h1:87CXzYJnAMGaa/IDDfRdhTzxk/wzGZ+/HUQpqgVSZXU=
-k8s.io/client-go v0.26.1/go.mod h1:IWNSglg+rQ3OcvDkhY6+QLeasV4OYHDjdqeWkDQZwGE=
-k8s.io/gengo v0.0.0-20190128074634-0689ccc1d7d6/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0=
-k8s.io/klog v0.0.0-20181102134211-b9b56d5dfc92/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk=
-k8s.io/klog v0.3.0/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk=
-k8s.io/klog v0.3.1/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk=
-k8s.io/klog v0.4.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I=
-k8s.io/klog/v2 v2.90.0 h1:VkTxIV/FjRXn1fgNNcKGM8cfmL1Z33ZjXRTVxKCoF5M=
-k8s.io/klog/v2 v2.90.0/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0=
-k8s.io/kube-openapi v0.0.0-20190228160746-b3a7cee44a30/go.mod h1:BXM9ceUBTj2QnfH2MK1odQs778ajze1RxcmP6S8RVVc=
-k8s.io/kube-openapi v0.0.0-20190709113604-33be087ad058/go.mod h1:nfDlWeOsu3pUf4yWGL+ERqohP4YsZcBJXWMK+gkzOA4=
-k8s.io/kube-openapi v0.0.0-20190722073852-5e22f3d471e6/go.mod h1:RZvgC8MSN6DjiMV6oIfEE9pDL9CYXokkfaCKZeHm3nc=
-k8s.io/kube-openapi v0.0.0-20230202010329-39b3636cbaa3 h1:vV3ZKAUX0nMjTflyfVea98dTfROpIxDaEsQws0FT2Ts=
-k8s.io/kube-openapi v0.0.0-20230202010329-39b3636cbaa3/go.mod h1:/BYxry62FuDzmI+i9B+X2pqfySRmSOW2ARmj5Zbqhj0=
-k8s.io/utils v0.0.0-20190221042446-c2654d5206da/go.mod h1:8k8uAuAQ0rXslZKaEWd0c3oVhZz7sSzSiPnVZayjIX0=
-k8s.io/utils v0.0.0-20190809000727-6c36bc71fc4a/go.mod h1:sZAwmy6armz5eXlNoLmJcl4F1QuKu7sr+mFQ0byX7Ew=
-k8s.io/utils v0.0.0-20230115233650-391b47cb4029 h1:L8zDtT4jrxj+TaQYD0k8KNlr556WaVQylDXswKmX+dE=
-k8s.io/utils v0.0.0-20230115233650-391b47cb4029/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0=
-rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8=
-sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo=
-sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0=
-sigs.k8s.io/structured-merge-diff v0.0.0-20190525122527-15d366b2352e/go.mod h1:wWxsB5ozmmv/SG7nM11ayaAW51xMvak/t1r0CSlcokI=
-sigs.k8s.io/structured-merge-diff/v4 v4.2.3 h1:PRbqxJClWWYMNV1dhaG4NsibJbArud9kFxnAMREiWFE=
-sigs.k8s.io/structured-merge-diff/v4 v4.2.3/go.mod h1:qjx8mGObPmV2aSZepjQjbmb2ihdVs8cGKBraizNC69E=
-sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o=
-sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E=
-sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY=
-vitess.io/vitess v0.19.4 h1:fJhZm5RknYbF7YUBXBmcvST/mvA4CeevlI9N0TLlYXA=
-vitess.io/vitess v0.19.4/go.mod h1:15uIi9x/Gu+BdDhUR80yP0M1v7aMG2mZQXSd56CE2t0=
diff --git a/vitess-mixin/jsonnetfile.json b/vitess-mixin/jsonnetfile.json
deleted file mode 100644
index 93f3316ec38..00000000000
--- a/vitess-mixin/jsonnetfile.json
+++ /dev/null
@@ -1,15 +0,0 @@
-{
- "version": 1,
- "dependencies": [
- {
- "source": {
- "git": {
- "remote": "https://github.com/grafana/grafonnet-lib.git",
- "subdir": "grafonnet"
- }
- },
- "version": "master"
- }
- ],
- "legacyImports": true
-}
diff --git a/vitess-mixin/lib/alerts.jsonnet b/vitess-mixin/lib/alerts.jsonnet
deleted file mode 100644
index d396a38cd71..00000000000
--- a/vitess-mixin/lib/alerts.jsonnet
+++ /dev/null
@@ -1 +0,0 @@
-std.manifestYamlDoc((import '../mixin.libsonnet').prometheusAlerts)
diff --git a/vitess-mixin/lib/dashboards.jsonnet b/vitess-mixin/lib/dashboards.jsonnet
deleted file mode 100644
index c260feb80e3..00000000000
--- a/vitess-mixin/lib/dashboards.jsonnet
+++ /dev/null
@@ -1,10 +0,0 @@
-local dashboards = (import '../mixin.libsonnet').grafanaDashboards;
-
-local config = (import '../config.libsonnet');
-local deploy_list = config._config.deploy_list;
-
-{
- [name]: dashboards[name]
- for name in std.objectFields(dashboards)
- if std.member(dashboards[name].environments, std.extVar('env'))
-}
diff --git a/vitess-mixin/lib/rules.jsonnet b/vitess-mixin/lib/rules.jsonnet
deleted file mode 100644
index 2d7fa91f7ca..00000000000
--- a/vitess-mixin/lib/rules.jsonnet
+++ /dev/null
@@ -1 +0,0 @@
-std.manifestYamlDoc((import '../mixin.libsonnet').prometheusRules)
diff --git a/vitess-mixin/mixin.libsonnet b/vitess-mixin/mixin.libsonnet
deleted file mode 100644
index b9831f9380b..00000000000
--- a/vitess-mixin/mixin.libsonnet
+++ /dev/null
@@ -1,4 +0,0 @@
-(import 'config.libsonnet') +
-(import 'alerts/alerts.libsonnet') +
-(import 'dashboards/dashboards.libsonnet') +
-(import 'rules/rules.libsonnet')
diff --git a/vitess-mixin/rules/rules.libsonnet b/vitess-mixin/rules/rules.libsonnet
deleted file mode 100644
index 903212ca351..00000000000
--- a/vitess-mixin/rules/rules.libsonnet
+++ /dev/null
@@ -1,221 +0,0 @@
-// Recording rules can be used to continually evaluate queries and
-// store their results as new timeseries. This is commonly used to
-// calculate expensive aggregates prior to querying them. You can have
-// any number of rules, split across any number of files.
-//
-// Reference: https://prometheus.io/docs/prometheus/latest/configuration/recording_rules/
-local config = import '../config.libsonnet';
-
-{
- prometheusRules+:: {
- groups: [
- {
- name: 'vitess_mixin_1',
- rules: [
- {
- record: 'vitess_mixin:vttablet_errors:rate1m',
- expr: 'sum (rate(vttablet_errors[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_2',
- rules: [
- {
- record: 'vitess_mixin:vttablet_query_counts:rate1m',
- expr: 'sum (rate(vttablet_query_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_3',
- rules: [
- {
- record: 'vitess_mixin:mysql_global_status_queries:rate1m',
- expr: 'sum (rate(mysql_global_status_queries[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_4',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_bucket:rate1m',
- expr: 'sum by(le)(rate(vtgate_api_bucket[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_5',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_bucket_by_keyspace:rate1m',
- expr: 'sum by(le,keyspace)(rate(vtgate_api_bucket[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_6',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_error_counts:rate1m',
- expr: 'sum (rate(vtgate_api_error_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_7',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_count:rate1m',
- expr: 'sum (rate(vtgate_api_count[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_8',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_error_counts_by_keyspace:rate1m',
- expr: 'sum by(keyspace)(rate(vtgate_api_error_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_9',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_count_by_keyspace:rate1m',
- expr: 'sum by(keyspace)(rate(vtgate_api_count[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_10',
- rules: [
- {
- record: 'vitess_mixin:vttablet_kills:rate1m',
- expr: 'sum by (keyspace,shard)(rate(vttablet_kills[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_11',
- rules: [
- {
- record: 'vitess_mixin:vtgate_vttablet_call_error_count_byinstance:rate1m',
- expr: 'sum by(instance)(rate(vtgate_vttablet_call_error_count[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_12',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_error_counts_by_db_type:rate1m',
- expr: 'sum by(db_type)(rate(vtgate_api_error_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_13',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_count_by_db_type:rate1m',
- expr: 'sum by(db_type)(rate(vtgate_api_count[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_14',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_bucket_by_db_type:rate1m',
- expr: 'sum by(le,db_type)(rate(vtgate_api_bucket[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_15',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_error_counts_by_operation:rate1m',
- expr: 'sum by(operation)(rate(vtgate_api_error_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_16',
- rules: [
- {
- record: 'vitess_mixin:vtgate_api_error_counts_by_code:rate1m',
- expr: 'sum by(code)(rate(vtgate_api_error_counts[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_17',
- rules: [
- {
- record: 'vitess_mixin:vttablet_queries_sum_by_keyspace_shard:rate1m',
- expr: 'sum by(keyspace,shard)(rate(vttablet_queries_sum[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_18',
- rules: [
- {
- record: 'vitess_mixin:vttablet_queries_count_by_keyspace_shard:rate1m',
- expr: 'sum by(keyspace,shard)(rate(vttablet_queries_count[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_19',
- rules: [
- {
- record: 'vitess_mixin:vttablet_transactions_bucket_by_keyspace_shard:rate1m',
- expr: 'sum by(keyspace,shard,le)(rate(vttablet_transactions_bucket[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_20',
- rules: [
- {
- record: 'vitess_mixin:process_start_time_seconds_by_instance_job:sum5m',
- expr: 'sum by (instance,job) (changes (process_start_time_seconds[5m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_21',
- rules: [
- {
- record: 'vitess_mixin:vttablet_kills_by_instance:rate1m',
- expr: 'sum by(instance)(rate(vttablet_kills[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_22',
- rules: [
- {
- record: 'vitess_mixin:vttablet_errors:rate1m',
- expr: 'sum by(keyspace,shard,instance,error_code)(rate(vttablet_errors[1m]))',
- },
- ],
- },
- {
- name: 'vitess_mixin_23',
- rules: [
- {
- record: 'vitess_mixin:vtgate_queries_processed_by_table:rate1m',
- expr: 'sum by(keyspace,plan,table) (rate(vtgate_queries_processed_by_table{plan!="Rollback"}[1m]))',
- },
- ],
- },
- ],
- },
-}
diff --git a/vitess-mixin/scripts/fmt.sh b/vitess-mixin/scripts/fmt.sh
deleted file mode 100755
index e1a5976ee8e..00000000000
--- a/vitess-mixin/scripts/fmt.sh
+++ /dev/null
@@ -1,67 +0,0 @@
-#!/usr/bin/env bash
-#/ Usage: fmt [--check]
-#/ --check Fail if changed files need changes, do not format in place.
-
-set -eu
-
-usage() {
- grep '^#/' "$0" | cut -c'4-' >&2
- exit "$1"
-}
-
-if ! [ -x "$(command -v jsonnet)" ] && ! [ -x "$(command -v jsonnetfmt)" ]; then
- echo 'jsonnet or jsonnetfmt executable not found'
- exit 1
-fi
-
-if [ -x "$(command -v jsonnetfmt)" ]; then
- JSONNET_COMMAND='jsonnetfmt'
-else
- JSONNET_COMMAND='jsonnet fmt'
-fi
-
-CHECK="false"
-EXIT_STATUS=0
-FILES_CHANGED="$(git diff --name-only origin/main --diff-filter=d | grep -E '(jsonnet|TEMPLATE|libsonnet)$' || true)"
-
-while [ "$#" -gt 0 ]; do
- case "$1" in
- -c|--check) CHECK="true"; shift;;
- *) usage ;;
- esac
-done
-
-if [ "${CHECK}" == "true" ]; then
- if [ -n "${FILES_CHANGED}" ]; then
- for FILE in ${FILES_CHANGED}; do
- set +e
- echo -n "Checking $FILE: "
- ${JSONNET_COMMAND} --test "${FILE}"
- EC=$?
- if [ ${EC} -ne 0 ]; then
- echo "⚠️"
- EXIT_STATUS=1
- else
- echo "✅"
- fi
- set -e
- done
- echo ""
- fi
-
- echo -n "STATUS:"
- if [ "$EXIT_STATUS" -eq 0 ]; then
- echo "✅"
- else
- echo "❌"
- fi
-
- exit $EXIT_STATUS
-
-else
- for FILE in $FILES_CHANGED; do
- echo -n "Formatting $FILE: "
- $JSONNET_COMMAND -n 2 --max-blank-lines 2 --string-style s --comment-style s -i ../$FILE
- echo "✅"
- done
-fi
diff --git a/vitess-mixin/scripts/vitess-mixin-plan.sh b/vitess-mixin/scripts/vitess-mixin-plan.sh
deleted file mode 100755
index 8c305601956..00000000000
--- a/vitess-mixin/scripts/vitess-mixin-plan.sh
+++ /dev/null
@@ -1,89 +0,0 @@
-#!/bin/bash
-if [[ -z "$ENV" || ("${ENV}" != 'dev' && "${ENV}" != 'prod') ]]; then
- echo -e "##ERROR\nPlease specify ENV (prod or dev)\nUsage: ENV='prod' ./vitess-mixin-plan.sh" 1>&2
- exit 1
-fi
-
-export ENVIRONMENT=$ENV
-
-EXIT_STATUS=0;
-
-echo "#### Building origin/main"
-REPODIR="$(pwd)"
-TEMPDIR="$(mktemp -d)"
-
-cd $TEMPDIR
-git clone git@github.com:vitessio/vitess.git > /dev/null 2>&1
-cd vitess/vitess-mixin
-jb install > /dev/null 2>&1
-make dashboards_out > /dev/null 2>&1
-make prometheus_rules.yaml > /dev/null 2>&1
-# TODO enalbe when alerts are not empty.
-# make prometheus_alerts.yaml > /dev/null 2>&1
-
-echo -e "\nDone!\n"
-
-cd $REPODIR
-make dashboards_out > /dev/null 2>&1
-make prometheus_rules.yaml > /dev/null 2>&1
-# TODO enalbe when alerts are not empty.
-# make prometheus_alerts.yaml > /dev/null 2>&1
-
-branch=$(git rev-parse --abbrev-ref HEAD)
-echo -e "#### Diff origin/main with $branch:\n"
-
-# TODO check prometheus_alerts.yaml
-
-t="# Checking prometheus_rules.yaml...";
-d=$(diff -urt --label origin/main/prometheus_rules.yaml "$TEMPDIR/vitess/vitess-mixin/prometheus_rules.yaml" --label $branch/prometheus_rules.yaml "prometheus_rules.yaml" 2>/dev/null)
-if [ "$?" = "0" ];
-then
- echo $t OK
-else
- echo $t NOK
- echo "$d"
- EXIT_STATUS=2
-fi
-
-DASHBOARDS=()
-for filename in $(ls dashboards_out)
-do
- t="# Checking $filename..."
- DASHBOARDS+=($filename)
- d=$(diff -urt --label origin/main/$filename "$TEMPDIR/vitess/vitess-mixin/dashboards_out/"$filename --label $branch/$filename "dashboards_out/"$filename 2>/dev/null)
- if [ "$?" = "0" ];
- then
- echo $t OK
- else
- if [ -e "$TEMPDIR/vitess-mixin/dashboards_out/"$filename ];
- then
- echo $t NOK
- echo "$d"
- else
- echo $t "This is a new dashboard not present in origin/main" NOK
- fi
- EXIT_STATUS=2
- fi
-done
-
-for filename in $(ls $TEMPDIR/vitess/vitess-mixin/dashboards_out)
-do
- t="# Checking $filename..."
- if [[ ! " ${DASHBOARDS[*]} " == *"$filename"* ]];
- then
- echo $t This dashboard has been removed NOK ;
- EXIT_STATUS=2
- fi
-done
-
-echo -e "\nEXIT STATUS:"
-if [ "$EXIT_STATUS" -eq 0 ]; then
- echo -e "✅ Your dashboards local version matches the origin/main version"
-elif [ "$EXIT_STATUS" -eq 2 ]; then
- echo -e "👀 If you are happy with your changes open a PR"
-else
- echo "❌"
- exit 1
-fi
-
-rm -rf $TEMPDIR
diff --git a/vitess-mixin/tools.go b/vitess-mixin/tools.go
deleted file mode 100644
index 63cc73e0bbb..00000000000
--- a/vitess-mixin/tools.go
+++ /dev/null
@@ -1,11 +0,0 @@
-//go:build tools
-// +build tools
-
-package main
-
-import (
- _ "github.com/google/go-jsonnet/cmd/jsonnet"
- _ "github.com/google/go-jsonnet/cmd/jsonnetfmt"
- _ "github.com/jsonnet-bundler/jsonnet-bundler/cmd/jb"
- _ "github.com/prometheus/prometheus/cmd/promtool"
-)
diff --git a/vitess-mixin/vitess-mixin-quickstart.md b/vitess-mixin/vitess-mixin-quickstart.md
deleted file mode 100644
index e15e6126211..00000000000
--- a/vitess-mixin/vitess-mixin-quickstart.md
+++ /dev/null
@@ -1,323 +0,0 @@
-# Vitess Mixin QuickStart
-
-## Table of Contents
-
-- [Vitess Mixin QuickStart](#vitess-mixin-quickstart)
- - [Table of Contents](#table-of-contents)
- - [What is the Vitess Mixin](#what-is-the-vitess-mixin)
- - [Development](#development)
- - [Building](#building)
- - [1. Prometheus Recording Rules](#1-prometheus-recording-rules)
- - [2. Grafana Dashboards](#2-grafana-dashboards)
- - [Formatting](#formatting)
- - [Linting](#linting)
- - [Cleaning up](#cleaning-up)
- - [Define a new Dashboard](#define-a-new-dashboard)
- - [Update Existing Dashboards](#update-existing-dashboards)
- - [Edit existing resources](#edit-existing-resources)
- - [Adding new resources](#adding-new-resources)
- - [Test and Deploy](#test-and-deploy)
- - [Local Unit Tests with `make all`](#local-unit-tests-with-make-all)
- - [Local e2e tests using Cypress (Alpha)](#local-e2e-tests-using-cypress-alpha)
- - [Local e2e manual testing using compose (Alpha)](#local-e2e-manual-testing-using-compose-alpha)
-
-## What is the Vitess Mixin
-
-Vitess [mixin](https://en.wikipedia.org/wiki/Mixin) is a monitoring mixin defined using [jsonnet](https://jsonnet.org/) data templating language. The grafana `jsonnet` library for creating dashboards is [grafonnet](https://github.com/grafana/grafonnet-lib). While the package manager for easily reusing and extending the mixin is [jsonnet-bundler](https://github.com/jsonnet-bundler/jsonnet-bundler).
-
-## Development
-
-If you want to see all available targets run `make help`.
-
-Choose an IDE of your preference. We use VSCode with `heptio.jsonnet` plugin which includes some nice features like on mouse over documentation and (ctrl+space) autocomplete.
-
-### Building
-
-#### 1. Prometheus Recording Rules
-
- ```shell
- $ make prometheus_rules.yaml
- # Building 'prometheus_rules.yaml'...
-
- Done!
- ```
-
- Once generated the Vitess Mixin Prometheus recording rules are available at `prometheus_rules.yaml`.
-
- **Note:** The Vitess Mixin only includes recording rules for Prometheus Vitess metrics. Other metric like Prometheus Node are not currently included in the rules.
-
-#### 2. Grafana Dashboards
-
-The vitess mixin supports DEV/PROD configurations (Currently it is possible to enable/disable dashboards. And set different alert configurations for DEV/PROD).
-
- **Prod dashboards**:
-
- ```shell
- $ ENV='prod' make dashboards_out
- # Building Grafana dashboards...
-
- dashboards_out/cluster_overview.json
- dashboards_out/vtgate_host_view.json
- dashboards_out/vtgate_overview.json
- dashboards_out/vttablet_host_view.json
-
- Done!
- ```
-
- **Dev dashboards**:
-
- ```shell
- $ ENV=dev make dashboards_out
-
- # Building Grafana dashboards...
-
- dashboards_out/cluster_overview.json
- dashboards_out/vtgate_host_view.json
- dashboards_out/vtgate_overview.json
- dashboards_out/vttablet_host_view.json
-
- Done!
- ```
-
- Voila!! the generated dashboards definitions in `.json` format are now available under `/dashboards_out`. (**Note**: dev and prod dashboards have the same names and they are stored in the same output folder. It is a good practice to run `make clean` after each build to make sure we are working with a clean `/dashboards_out` folder).
-
-### Formatting
-
- ```shell
- $ make fmt
- # Formatting all .libsonnet and .jsonnet files...
-
- Formatting dashboards/alerts_memory_events.libsonnet: ✅
-
- Done!
- ```
-
-### Linting
-
- ```shell
- $ make lint
- # Linting all .libsonnet and .jsonnet files...
-
- Checking dashboards/alerts_memory_events.libsonnet: ✅
-
- STATUS:✅
-
- Done!
-
- # Linting 'prometheus_rules.yaml' and 'prometheus_alerts.yaml'...
- TO BE IMPLEMENTED
- Done!
- ```
-
-### Cleaning up
-
- ```shell
- $ make clean
- # Cleaning up all generated files...
-
- rm -rf dashboards_out prometheus_alerts.yaml prometheus_rules.yaml
-
- Done!
- ```
-
-## Define a new Dashboard
-
- Dashboards are defined in `.libsonnet` format under the `/dashboards` folder.
- **Note** Files designed for import by convention end with `.libsonnet`.
-
- 1. Update the Mixin config to include the new **dashboard metadata** `config.libsonnet`.
- The metadata needs to be added under `grafanaDashboardMetadata`. For example for `vtgate-overview dashboard` it looks like this:
-
- ```bash
- vtgateOverview: {
- environments: ['dev', 'prod'],
- uid: 'vitess-vtgate-overview',
- title: 'vtgate - overview %(dashboardNameSuffix)s' % $._config.grafanaDashboardMetadataDefault,
- description: 'General vtgate overview',
- dashboardTags: $._config.grafanaDashboardMetadataDefault.dashboardTags + ['overview', 'vtgate'],
- }
- ```
-
- **Note**: It is possible to add/remove a dashboard from specific environments using the `environments` array field.
-
- 2. Define the **dashboard layout**: Create a new `.libsonnet` file following _Naming Convention_ (**TODO** Define a naming convention) in the `/dashboards/layouts` folder.
- The dashboard skeleton should look something like this:
-
- ```js
- local helpers = import '../resources/helpers.libsonnet';
-
- local config = import '../../config.libsonnet';
-
- {
- grafanaDashboards+:: {
- 'DASHBOARD_NAME.json':
-
- helpers.dashboard.getDashboard(config._config.grafanaDashboardMetadata.DASHBOARD_NAME)
- .addTemplates(
- [
-
- ]
- ).addLink(helpers.default.getDashboardLink(config._config.dashborardLinks))
- .addPanels(
- [
-
- ],
- ),
- },
- }
- ```
-
- 3. Import the new dashboard in the `dashboard.libsonnet` library.
-
- 4. Add/Edit dashboard resources.
-
- **Note:** Some of the resources have been grouped in the following categories `vtgate`, `vttablet`, `vtorc`, `webapp`, `os` for convenience. The resources under this categories follow a helper/config pattern. The `.*_helper.libsonnet` libraries within `grafonnet` folder functions to retrieve resources such as `getPanel(panel_name_config)`, reading the configuration by name for the desired resource from the corresponding `.*_config.libsonnet` library .
-
- **Note:** To isolate **PROD** and **DEV** the configuration files have a section for each environment where we can define specific `notifications`(ex: pagerduty, vitess-alerts...) and `conditions` (alert thresholds).
-
- In this example the alert has a different notification config. For **PROD** it pages and **DEV** it doesn't:
-
- ```js
- alert: {
- name: 'vitess-mixin - # of tablets with MySQL replication lag > 300s',
- frequency: '1m',
- forDuration: '3m',
- noDataState: 'ok',
- executionErrorState: 'keep_state',
- message: |||
- The number of tablets with MySQL replication lag > 300s is above the threshold.
- This usually indicate that replicas are not able to keep up with the write loads on the primary. Please take a look.
- |||,
-
- prod+: {
- notifications: [
- { uid: 'alerts-vitess' },
- { uid: 'pagerduty-vitess' },
- ],
- conditions: [
- alert_condition.new(
- evaluatorParams=[1],
- evaluatorType='gt',
- operatorType='and',
- queryRefId='A',
- queryTimeStart='1m',
- reducerType='max'),
- ],
- },
- //override PROD notifications.
- dev+: self.prod {
- notifications: [
- { uid: 'alerts-vitess' },
- ],
- },
- },
- ```
-
- **Warning:** Editing existing resources can change other dashboards behavior. Since basic resources like `panels`, `rows`, `texts`... can be shared across dashboards. This can be easily checked running `make all` as explained bellow.
-
-## Update Existing Dashboards
-
- Same steps as **Define a new Dashboard** starting from step `.4`.
-
-### Edit existing resources
-
- 1. Look for the dashboard you want to edit in the `dashboards/layouts` folder.
- 2. Find the resource you want to update in the `dashboards/resources` folder.
-
-### Adding new resources
-
- 1. Create the resource in the corresponding location under `dashboards/resources` folder.
- 2. Add the resource to the desired layout in `dashboards/layouts`.
-
-## Test and Deploy
-
-### Local Unit Tests with `make all`
-
-These tests attempt to assert truth to the following:
-
-- Does this mixin generate valid dashboard JSON?
-- Does this mixin generate valid recording rules YAML?
-- Do the configured elements do what they are expected to do?
-
-This make target will format, lint, build all generated dashboards and recording rules using `origin/main` and diff with them your `local branch`. Printing a report to the `stdout`.
-
-```shell
- $ pwd
- /manfontan/vitess-mixin
- $ ENV='prod' make all
- #### Building origin/main
-
- Done!
-
- #### Diff origin/main with main:
-
- # Checking prometheus_rules.yaml... OK
- # Checking cluster_overview.json... OK
- # Checking vtgate_host_view.json... OK
- # Checking vtgate_overview.json... OK
- # Checking vttablet_host_view.json... OK
-
- EXIT STATUS:
- ✅ Your dashboards local version matches the origin/main version
-```
-
-The above execution shows the report for a `local branch` matching `origin/main`. Any changes will be reported as **NOK** along with the diff report. This doesn't mean something is wrong it just points that there are changes in your local branch compared to `origin/main` which is expected. Review the diff report and once you are happy with your changes create a PR.
-
-### Local e2e tests using Cypress (Alpha)
-
-These tests attempt to assert truth to the following:
-
-- Are dashboard elements displayed as expected?
-
-The spec for each dashboard can be found at `/e2e/cypress/integration`.
-
-`docker-compose` is used to run Cypress and Grafana. There are two targets in
-[Makefile](../Makefile) to help run it.
-
-- `make e2e`: runs tests headless and exits.
-
-- `make e2e-dev`: opens the [the test
-runner](https://docs.cypress.io/guides/core-concepts/test-runner.html#Overview)
-and exposes Grafana to the host machine - http://localhost:3030. This requires
-an X11 server to work. [This
-post](https://www.cypress.io/blog/2019/05/02/run-cypress-with-a-single-docker-command/#Interactive-mode)
-describes how to set this up with [XQuartz](https://www.xquartz.org/).
-
-**Note** The dummy Grafana server is not connected to a Prometheus Backend for this reason dashboards will not display any data, templates will fail to load etc... If you don't have a Dev Prometheus server. Replacing prometheus datasources(`Prometheus_Vitess`, `Prometheus_Node` ...) with an empty string in the generated JSON file will default to the dummy datasource displaying dummy data. This is useful when testing using interactive mode `make e2e-dev`.
-
-### Local e2e manual testing using compose (Alpha)
-
-**Note**: This targets have been tested using docker for Mac. You may need to change the IPs and configurations for your specific setup.
-
-Before we run the local environment using compose it is necessary to generate the dashboards and recording rules using `ENV=dev make all` (Note: choose the environment you want to test).
-
-Once our dashboards are available simply run:
-
-- `make e2e-compose-up`: spin up the cluster
-
-Changes to the dashboards are not dynamically loaded so you will need to bring down the cluster and initialized it again to load your changes.
-
-If you are done testing or the cluster gets in a bad state quit and clean up using:
-
-- `make e2e-compose-down`: cleanup compose resources
-
-In order to generate some metrics we can use the following commands:
-
-```shell
-## INSERT TEST DATA
-mysql --port=15306 --host=127.0.0.1 < load_test.sql
-## SIMULATED QUERIES
-mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=test_keyspace:80-@primary --query="SELECT * FROM messages;"
-mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=test_keyspace:80-@replica --query="SELECT * FROM messages;"
-mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=lookup_keyspace:-@primary --query="SELECT * FROM messages_message_lookup;"
-mysqlslap -c 5 --port=15306 --host=127.0.0.1 --iterations=1000 --create-schema=lookup_keyspace:-@replica --query="SELECT * FROM messages_message_lookup;"
-## SIMULATED ERRORS
-mysqlslap --port=15306 --host=127.0.0.1 --iterations=10000 --create-schema=test_keyspace:80-@primary --query="SELECT name FROM messages;"
-mysqlslap --port=15306 --host=127.0.0.1 --iterations=10000 --create-schema=lookup_keyspace:-@replica --query="SELECT name FROM messages_message_lookup;"
-```
-
-Once the cluster is up and running you should be able to access:
-
-- grafana (default credentials > admin/admin) http://localhost:3030/
-- prometheus http://localhost:9000/
-- vitess control panel http://localhost:15000/
diff --git a/web/vtadmin/.prettiercc b/web/vtadmin/.prettiercc
index bed1b4f32cd..6fdc0af39dd 100644
--- a/web/vtadmin/.prettiercc
+++ b/web/vtadmin/.prettiercc
@@ -7,7 +7,6 @@
"bracketSpacing": true,
"endOfLine": "auto",
"jsxSingleQuote": false,
- "jsxBracketSameLine": false,
"quoteProps": "as-needed",
"htmlWhitespaceSensitivity": "css",
"printWidth": 120
diff --git a/web/vtadmin/package-lock.json b/web/vtadmin/package-lock.json
index 928217e9d4a..8ad7c67a5b4 100644
--- a/web/vtadmin/package-lock.json
+++ b/web/vtadmin/package-lock.json
@@ -21,6 +21,7 @@
"highcharts-react-official": "^3.1.0",
"history": "^5.3.0",
"lodash-es": "^4.17.21",
+ "path-to-regexp": "^8.1.0",
"postcss-flexbugs-fixes": "^5.0.2",
"postcss-preset-env": "^8.0.1",
"query-string": "^7.1.3",
@@ -51,7 +52,7 @@
"eslint-config-react-app": "^7.0.1",
"i": "^0.3.7",
"jsdom": "^21.1.1",
- "msw": "^0.36.8",
+ "msw": "^2.5.2",
"npm": "^10.8.0",
"postcss": "^8.4.31",
"prettier": "^2.2.1",
@@ -2277,6 +2278,37 @@
"resolved": "https://registry.npmjs.org/@bugsnag/safe-json-stringify/-/safe-json-stringify-6.0.0.tgz",
"integrity": "sha512-htzFO1Zc57S8kgdRK9mLcPVTW1BY2ijfH7Dk2CeZmspTWKdKqSo1iwmqrq2WtRjFlo8aRZYgLX0wFrDXF/9DLA=="
},
+ "node_modules/@bundled-es-modules/cookie": {
+ "version": "2.0.1",
+ "resolved": "https://registry.npmjs.org/@bundled-es-modules/cookie/-/cookie-2.0.1.tgz",
+ "integrity": "sha512-8o+5fRPLNbjbdGRRmJj3h6Hh1AQJf2dk3qQ/5ZFb+PXkRNiSoMGGUKlsgLfrxneb72axVJyIYji64E2+nNfYyw==",
+ "dev": true,
+ "license": "ISC",
+ "dependencies": {
+ "cookie": "^0.7.2"
+ }
+ },
+ "node_modules/@bundled-es-modules/statuses": {
+ "version": "1.0.1",
+ "resolved": "https://registry.npmjs.org/@bundled-es-modules/statuses/-/statuses-1.0.1.tgz",
+ "integrity": "sha512-yn7BklA5acgcBr+7w064fGV+SGIFySjCKpqjcWgBAIfrAkY+4GQTJJHQMeT3V/sgz23VTEVV8TtOmkvJAhFVfg==",
+ "dev": true,
+ "license": "ISC",
+ "dependencies": {
+ "statuses": "^2.0.1"
+ }
+ },
+ "node_modules/@bundled-es-modules/tough-cookie": {
+ "version": "0.1.6",
+ "resolved": "https://registry.npmjs.org/@bundled-es-modules/tough-cookie/-/tough-cookie-0.1.6.tgz",
+ "integrity": "sha512-dvMHbL464C0zI+Yqxbz6kZ5TOEp7GLW+pry/RWndAR8MJQAXZ2rPmIs8tziTZjeIyhSNZgZbCePtfSbdWqStJw==",
+ "dev": true,
+ "license": "ISC",
+ "dependencies": {
+ "@types/tough-cookie": "^4.0.5",
+ "tough-cookie": "^4.1.4"
+ }
+ },
"node_modules/@csstools/cascade-layer-name-parser": {
"version": "1.0.9",
"resolved": "https://registry.npmjs.org/@csstools/cascade-layer-name-parser/-/cascade-layer-name-parser-1.0.9.tgz",
@@ -3474,6 +3506,74 @@
"integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==",
"dev": true
},
+ "node_modules/@inquirer/core": {
+ "version": "10.0.1",
+ "resolved": "https://registry.npmjs.org/@inquirer/core/-/core-10.0.1.tgz",
+ "integrity": "sha512-KKTgjViBQUi3AAssqjUFMnMO3CM3qwCHvePV9EW+zTKGKafFGFF01sc1yOIYjLJ7QU52G/FbzKc+c01WLzXmVQ==",
+ "dev": true,
+ "license": "MIT",
+ "dependencies": {
+ "@inquirer/figures": "^1.0.7",
+ "@inquirer/type": "^3.0.0",
+ "ansi-escapes": "^4.3.2",
+ "cli-width": "^4.1.0",
+ "mute-stream": "^2.0.0",
+ "signal-exit": "^4.1.0",
+ "strip-ansi": "^6.0.1",
+ "wrap-ansi": "^6.2.0",
+ "yoctocolors-cjs": "^2.1.2"
+ },
+ "engines": {
+ "node": ">=18"
+ }
+ },
+ "node_modules/@inquirer/core/node_modules/@inquirer/type": {
+ "version": "3.0.0",
+ "resolved": "https://registry.npmjs.org/@inquirer/type/-/type-3.0.0.tgz",
+ "integrity": "sha512-YYykfbw/lefC7yKj7nanzQXILM7r3suIvyFlCcMskc99axmsSewXWkAfXKwMbgxL76iAFVmRwmYdwNZNc8gjog==",
+ "dev": true,
+ "license": "MIT",
+ "engines": {
+ "node": ">=18"
+ },
+ "peerDependencies": {
+ "@types/node": ">=18"
+ }
+ },
+ "node_modules/@inquirer/core/node_modules/@types/node": {
+ "version": "22.9.0",
+ "resolved": "https://registry.npmjs.org/@types/node/-/node-22.9.0.tgz",
+ "integrity": "sha512-vuyHg81vvWA1Z1ELfvLko2c8f34gyA0zaic0+Rllc5lbCnbSyuvb2Oxpm6TAUAC/2xZN3QGqxBNggD1nNR2AfQ==",
+ "dev": true,
+ "license": "MIT",
+ "peer": true,
+ "dependencies": {
+ "undici-types": "~6.19.8"
+ }
+ },
+ "node_modules/@inquirer/core/node_modules/signal-exit": {
+ "version": "4.1.0",
+ "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-4.1.0.tgz",
+ "integrity": "sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw==",
+ "dev": true,
+ "license": "ISC",
+ "engines": {
+ "node": ">=14"
+ },
+ "funding": {
+ "url": "https://github.com/sponsors/isaacs"
+ }
+ },
+ "node_modules/@inquirer/figures": {
+ "version": "1.0.7",
+ "resolved": "https://registry.npmjs.org/@inquirer/figures/-/figures-1.0.7.tgz",
+ "integrity": "sha512-m+Trk77mp54Zma6xLkLuY+mvanPxlE4A7yNKs2HBiyZ4UkVs28Mv5c/pgWrHeInx+USHeX/WEPzjrWrcJiQgjw==",
+ "dev": true,
+ "license": "MIT",
+ "engines": {
+ "node": ">=18"
+ }
+ },
"node_modules/@isaacs/cliui": {
"version": "8.0.2",
"resolved": "https://registry.npmjs.org/@isaacs/cliui/-/cliui-8.0.2.tgz",
@@ -3662,28 +3762,22 @@
"node": ">=v12.0.0"
}
},
- "node_modules/@mswjs/cookies": {
- "version": "0.1.7",
- "resolved": "https://registry.npmjs.org/@mswjs/cookies/-/cookies-0.1.7.tgz",
- "integrity": "sha512-bDg1ReMBx+PYDB4Pk7y1Q07Zz1iKIEUWQpkEXiA2lEWg9gvOZ8UBmGXilCEUvyYoRFlmr/9iXTRR69TrgSwX/Q==",
- "dev": true,
- "dependencies": {
- "@types/set-cookie-parser": "^2.4.0",
- "set-cookie-parser": "^2.4.6"
- }
- },
"node_modules/@mswjs/interceptors": {
- "version": "0.12.7",
- "resolved": "https://registry.npmjs.org/@mswjs/interceptors/-/interceptors-0.12.7.tgz",
- "integrity": "sha512-eGjZ3JRAt0Fzi5FgXiV/P3bJGj0NqsN7vBS0J0FO2AQRQ0jCKQS4lEFm4wvlSgKQNfeuc/Vz6d81VtU3Gkx/zg==",
+ "version": "0.36.9",
+ "resolved": "https://registry.npmjs.org/@mswjs/interceptors/-/interceptors-0.36.9.tgz",
+ "integrity": "sha512-mMRDUBwSNeCgjSMEWfjoh4Rm9fbyZ7xQ9SBq8eGHiiyRn1ieTip3pNEt0wxWVPPxR4i1Rv9bTkeEbkX7M4c15A==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "@open-draft/until": "^1.0.3",
- "@xmldom/xmldom": "^0.7.2",
- "debug": "^4.3.2",
- "headers-utils": "^3.0.2",
- "outvariant": "^1.2.0",
- "strict-event-emitter": "^0.2.0"
+ "@open-draft/deferred-promise": "^2.2.0",
+ "@open-draft/logger": "^0.3.0",
+ "@open-draft/until": "^2.0.0",
+ "is-node-process": "^1.2.0",
+ "outvariant": "^1.4.3",
+ "strict-event-emitter": "^0.5.1"
+ },
+ "engines": {
+ "node": ">=18"
}
},
"node_modules/@nicolo-ribaudo/eslint-scope-5-internals": {
@@ -3752,11 +3846,30 @@
"node": ">= 8"
}
},
+ "node_modules/@open-draft/deferred-promise": {
+ "version": "2.2.0",
+ "resolved": "https://registry.npmjs.org/@open-draft/deferred-promise/-/deferred-promise-2.2.0.tgz",
+ "integrity": "sha512-CecwLWx3rhxVQF6V4bAgPS5t+So2sTbPgAzafKkVizyi7tlwpcFpdFqq+wqF2OwNBmqFuu6tOyouTuxgpMfzmA==",
+ "dev": true,
+ "license": "MIT"
+ },
+ "node_modules/@open-draft/logger": {
+ "version": "0.3.0",
+ "resolved": "https://registry.npmjs.org/@open-draft/logger/-/logger-0.3.0.tgz",
+ "integrity": "sha512-X2g45fzhxH238HKO4xbSr7+wBS8Fvw6ixhTDuvLd5mqh6bJJCFAPwU9mPDxbcrRtfxv4u5IHCEH77BmxvXmmxQ==",
+ "dev": true,
+ "license": "MIT",
+ "dependencies": {
+ "is-node-process": "^1.2.0",
+ "outvariant": "^1.4.0"
+ }
+ },
"node_modules/@open-draft/until": {
- "version": "1.0.3",
- "resolved": "https://registry.npmjs.org/@open-draft/until/-/until-1.0.3.tgz",
- "integrity": "sha512-Aq58f5HiWdyDlFffbbSjAlv596h/cOnt2DO1w3DOC7OJ5EHs0hd/nycJfiu9RJbT6Yk6F1knnRRXNSpxoIVZ9Q==",
- "dev": true
+ "version": "2.1.0",
+ "resolved": "https://registry.npmjs.org/@open-draft/until/-/until-2.1.0.tgz",
+ "integrity": "sha512-U69T3ItWHvLwGg5eJ0n3I62nWuE6ilHlmz7zM0npLBRvPRd7e6NYmg54vvRtP5mZG7kZqZCFVdsTWo7BPtBujg==",
+ "dev": true,
+ "license": "MIT"
},
"node_modules/@pkgjs/parseargs": {
"version": "0.11.0",
@@ -3859,7 +3972,7 @@
"node": ">=14.0.0"
},
"peerDependencies": {
- "rollup": "^2.68.0||^3.0.0"
+ "rollup": "^3.29.5"
},
"peerDependenciesMeta": {
"rollup": {
@@ -3881,7 +3994,7 @@
"node": ">=14.0.0"
},
"peerDependencies": {
- "rollup": "^1.20.0||^2.0.0||^3.0.0||^4.0.0"
+ "rollup": "^3.29.5"
},
"peerDependenciesMeta": {
"rollup": {
@@ -4380,10 +4493,11 @@
}
},
"node_modules/@types/cookie": {
- "version": "0.4.1",
- "resolved": "https://registry.npmjs.org/@types/cookie/-/cookie-0.4.1.tgz",
- "integrity": "sha512-XW/Aa8APYr6jSVVA1y/DEIZX0/GMKLEVekNG727R8cs56ahETkRAy/3DR7+fJyh7oUgGwNQaRfXCun0+KbWY7Q==",
- "dev": true
+ "version": "0.6.0",
+ "resolved": "https://registry.npmjs.org/@types/cookie/-/cookie-0.6.0.tgz",
+ "integrity": "sha512-4Kh9a6B2bQciAhf7FSuMRRkUWecJgJu9nPnx3yzpsfXX/c50REIqpHY4C82bXP90qrLtXtkDxTZosYO3UpOwlA==",
+ "dev": true,
+ "license": "MIT"
},
"node_modules/@types/d3": {
"version": "7.4.3",
@@ -4634,16 +4748,6 @@
"resolved": "https://registry.npmjs.org/@types/history/-/history-4.7.11.tgz",
"integrity": "sha512-qjDJRrmvBMiTx+jyLxvLfJU7UznFuokDv4f3WRuriHKERccVpFU+8XMQUAbDzoiJCsmexxRExQeMwwCdamSKDA=="
},
- "node_modules/@types/inquirer": {
- "version": "8.2.10",
- "resolved": "https://registry.npmjs.org/@types/inquirer/-/inquirer-8.2.10.tgz",
- "integrity": "sha512-IdD5NmHyVjWM8SHWo/kPBgtzXatwPkfwzyP3fN1jF2g9BWt5WO+8hL2F4o2GKIYsU40PpqeevuUWvkS/roXJkA==",
- "dev": true,
- "dependencies": {
- "@types/through": "*",
- "rxjs": "^7.2.0"
- }
- },
"node_modules/@types/istanbul-lib-coverage": {
"version": "2.0.6",
"resolved": "https://registry.npmjs.org/@types/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.6.tgz",
@@ -4703,12 +4807,6 @@
"resolved": "https://registry.npmjs.org/react-is/-/react-is-18.3.1.tgz",
"integrity": "sha512-/LLMVyas0ljjAtoYiPqYiL8VWXzUUdThrmU5+n20DZv+a+ClRoevUzw5JxU+Ieh5/c87ytoTBV9G1FiKfNJdmg=="
},
- "node_modules/@types/js-levenshtein": {
- "version": "1.1.3",
- "resolved": "https://registry.npmjs.org/@types/js-levenshtein/-/js-levenshtein-1.1.3.tgz",
- "integrity": "sha512-jd+Q+sD20Qfu9e2aEXogiO3vpOC1PYJOUdyN9gvs4Qrvkg4wF43L5OhqrPeokdv8TL0/mXoYfpkcoGZMNN2pkQ==",
- "dev": true
- },
"node_modules/@types/json-schema": {
"version": "7.0.15",
"resolved": "https://registry.npmjs.org/@types/json-schema/-/json-schema-7.0.15.tgz",
@@ -4843,20 +4941,18 @@
"integrity": "sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ==",
"dev": true
},
- "node_modules/@types/set-cookie-parser": {
- "version": "2.4.7",
- "resolved": "https://registry.npmjs.org/@types/set-cookie-parser/-/set-cookie-parser-2.4.7.tgz",
- "integrity": "sha512-+ge/loa0oTozxip6zmhRIk8Z/boU51wl9Q6QdLZcokIGMzY5lFXYy/x7Htj2HTC6/KZP1hUbZ1ekx8DYXICvWg==",
- "dev": true,
- "dependencies": {
- "@types/node": "*"
- }
- },
"node_modules/@types/stack-utils": {
"version": "2.0.3",
"resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz",
"integrity": "sha512-9aEbYZ3TbYMznPdcdr3SmIrLXwC/AKZXQeCf9Pgao5CKb8CyHuEX5jzWPTkvregvhRJHcpRO6BFoGW9ycaOkYw=="
},
+ "node_modules/@types/statuses": {
+ "version": "2.0.5",
+ "resolved": "https://registry.npmjs.org/@types/statuses/-/statuses-2.0.5.tgz",
+ "integrity": "sha512-jmIUGWrAiwu3dZpxntxieC+1n/5c3mjrImkmOSQ2NC5uP6cYO4aAZDdSmRcI5C1oiTmqlZGHC+/NmJrKogbP5A==",
+ "dev": true,
+ "license": "MIT"
+ },
"node_modules/@types/testing-library__jest-dom": {
"version": "5.14.9",
"resolved": "https://registry.npmjs.org/@types/testing-library__jest-dom/-/testing-library__jest-dom-5.14.9.tgz",
@@ -4866,14 +4962,12 @@
"@types/jest": "*"
}
},
- "node_modules/@types/through": {
- "version": "0.0.33",
- "resolved": "https://registry.npmjs.org/@types/through/-/through-0.0.33.tgz",
- "integrity": "sha512-HsJ+z3QuETzP3cswwtzt2vEIiHBk/dCcHGhbmG5X3ecnwFD/lPrMpliGXxSCg03L9AhrdwA4Oz/qfspkDW+xGQ==",
+ "node_modules/@types/tough-cookie": {
+ "version": "4.0.5",
+ "resolved": "https://registry.npmjs.org/@types/tough-cookie/-/tough-cookie-4.0.5.tgz",
+ "integrity": "sha512-/Ad8+nIOV7Rl++6f1BdKxFSMgmoqEoYbHRpPcx3JEfv8VRsQe9Z4mCXeJBzxs7mbHY/XOZZuXlRNfhpVPbs6ZA==",
"dev": true,
- "dependencies": {
- "@types/node": "*"
- }
+ "license": "MIT"
},
"node_modules/@types/yargs": {
"version": "17.0.32",
@@ -5238,7 +5332,7 @@
"node": "^14.18.0 || >=16.0.0"
},
"peerDependencies": {
- "vite": "^4.1.0-beta.0"
+ "vite": "^4.5.4"
}
},
"node_modules/@vitest/expect": {
@@ -5311,15 +5405,6 @@
"pretty-format": "^27.5.1"
}
},
- "node_modules/@xmldom/xmldom": {
- "version": "0.7.13",
- "resolved": "https://registry.npmjs.org/@xmldom/xmldom/-/xmldom-0.7.13.tgz",
- "integrity": "sha512-lm2GW5PkosIzccsaZIz7tp8cPADSIlIHWDFTR1N0SzfinhhYgeIQjFMz4rYzanCScr3DqQLeomUDArp6MWKm+g==",
- "dev": true,
- "engines": {
- "node": ">=10.0.0"
- }
- },
"node_modules/@zeit/schemas": {
"version": "2.36.0",
"resolved": "https://registry.npmjs.org/@zeit/schemas/-/schemas-2.36.0.tgz",
@@ -5428,6 +5513,7 @@
"resolved": "https://registry.npmjs.org/ansi-escapes/-/ansi-escapes-4.3.2.tgz",
"integrity": "sha512-gKXj5ALrKWQLsYG9jlTRmR/xKluxHV+Z9QEwNIgCfM1/uwPMCuzVVnh5mwTd+OuBZcwSIMbqssNWRm1lE51QaQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"type-fest": "^0.21.3"
},
@@ -5443,6 +5529,7 @@
"resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.21.3.tgz",
"integrity": "sha512-t0rzBq87m3fVcduHDUFhKmyyX+9eo6WQjZvf51Ea/M0Q7+T374Jp1aUiyUl0GKxp8M/OETVHSDvmkyPgvX+X2w==",
"dev": true,
+ "license": "(MIT OR CC0-1.0)",
"engines": {
"node": ">=10"
},
@@ -5897,26 +5984,6 @@
"resolved": "https://registry.npmjs.org/balanced-match/-/balanced-match-1.0.2.tgz",
"integrity": "sha512-3oSeUO0TMV67hN1AmbXsK4yaqU7tjiHlbxRDZOpH0KW9+CeX4bRAaX0Anxt0tx2MrpRpWwQaPwIlISEJhYU5Pw=="
},
- "node_modules/base64-js": {
- "version": "1.5.1",
- "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.5.1.tgz",
- "integrity": "sha512-AKpaYlHn8t4SVbOHCy+b5+KKgvR4vrsD8vbvrbiQJps7fKDTkjkDry6ji0rUJjC0kzbNePLwzxq8iypo41qeWA==",
- "dev": true,
- "funding": [
- {
- "type": "github",
- "url": "https://github.com/sponsors/feross"
- },
- {
- "type": "patreon",
- "url": "https://www.patreon.com/feross"
- },
- {
- "type": "consulting",
- "url": "https://feross.org/support"
- }
- ]
- },
"node_modules/big-integer": {
"version": "1.6.52",
"resolved": "https://registry.npmjs.org/big-integer/-/big-integer-1.6.52.tgz",
@@ -5936,17 +6003,6 @@
"url": "https://github.com/sponsors/sindresorhus"
}
},
- "node_modules/bl": {
- "version": "4.1.0",
- "resolved": "https://registry.npmjs.org/bl/-/bl-4.1.0.tgz",
- "integrity": "sha512-1W07cM9gS6DcLperZfFSj+bWLtaPGSOHWhPiGzXmvVJbRLdG82sH/Kn8EtW1VqWVA54AKf2h5k5BbnIbwF3h6w==",
- "dev": true,
- "dependencies": {
- "buffer": "^5.5.0",
- "inherits": "^2.0.4",
- "readable-stream": "^3.4.0"
- }
- },
"node_modules/bluebird": {
"version": "3.7.2",
"resolved": "https://registry.npmjs.org/bluebird/-/bluebird-3.7.2.tgz",
@@ -6144,30 +6200,6 @@
"node": "^6 || ^7 || ^8 || ^9 || ^10 || ^11 || ^12 || >=13.7"
}
},
- "node_modules/buffer": {
- "version": "5.7.1",
- "resolved": "https://registry.npmjs.org/buffer/-/buffer-5.7.1.tgz",
- "integrity": "sha512-EHcyIPBQ4BSGlvjB16k5KgAJ27CIsHY/2JBmCRReo48y9rQ3MaUzWX3KVlBa4U7MyX02HdVj0K7C3WaB3ju7FQ==",
- "dev": true,
- "funding": [
- {
- "type": "github",
- "url": "https://github.com/sponsors/feross"
- },
- {
- "type": "patreon",
- "url": "https://www.patreon.com/feross"
- },
- {
- "type": "consulting",
- "url": "https://feross.org/support"
- }
- ],
- "dependencies": {
- "base64-js": "^1.3.1",
- "ieee754": "^1.1.13"
- }
- },
"node_modules/byline": {
"version": "5.0.0",
"resolved": "https://registry.npmjs.org/byline/-/byline-5.0.0.tgz",
@@ -6348,12 +6380,6 @@
"url": "https://github.com/chalk/chalk-template?sponsor=1"
}
},
- "node_modules/chardet": {
- "version": "0.7.0",
- "resolved": "https://registry.npmjs.org/chardet/-/chardet-0.7.0.tgz",
- "integrity": "sha512-mT8iDcrh03qDGRRmoA2hmBJnxpllMR+0/0qlzjqZES6NdiWDcZkCNAk4rPFZ9Q85r27unkiNNg8ZOiwZXBHwcA==",
- "dev": true
- },
"node_modules/check-error": {
"version": "1.0.3",
"resolved": "https://registry.npmjs.org/check-error/-/check-error-1.0.3.tgz",
@@ -6436,30 +6462,6 @@
"url": "https://github.com/sponsors/sindresorhus"
}
},
- "node_modules/cli-cursor": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/cli-cursor/-/cli-cursor-3.1.0.tgz",
- "integrity": "sha512-I/zHAwsKf9FqGoXM4WWRACob9+SNukZTd94DWF57E4toouRulbCxcUh6RKUEOQlYTHJnzkPMySvPNaaSLNfLZw==",
- "dev": true,
- "dependencies": {
- "restore-cursor": "^3.1.0"
- },
- "engines": {
- "node": ">=8"
- }
- },
- "node_modules/cli-spinners": {
- "version": "2.9.2",
- "resolved": "https://registry.npmjs.org/cli-spinners/-/cli-spinners-2.9.2.tgz",
- "integrity": "sha512-ywqV+5MmyL4E7ybXgKys4DugZbX0FC6LnwrhjuykIjnK9k8OQacQ7axGKnjDXWNhns0xot3bZI5h55H8yo9cJg==",
- "dev": true,
- "engines": {
- "node": ">=6"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
"node_modules/cli-truncate": {
"version": "3.1.0",
"resolved": "https://registry.npmjs.org/cli-truncate/-/cli-truncate-3.1.0.tgz",
@@ -6561,12 +6563,13 @@
}
},
"node_modules/cli-width": {
- "version": "3.0.0",
- "resolved": "https://registry.npmjs.org/cli-width/-/cli-width-3.0.0.tgz",
- "integrity": "sha512-FxqpkPPwu1HjuN93Omfm4h8uIanXofW0RxVEW3k5RKx+mJJYSthzNhp32Kzxxy3YAEZ/Dc/EWN1vZRY0+kOhbw==",
+ "version": "4.1.0",
+ "resolved": "https://registry.npmjs.org/cli-width/-/cli-width-4.1.0.tgz",
+ "integrity": "sha512-ouuZd4/dm2Sw5Gmqy6bGyNNNe1qt9RpmxveLSO7KcgsTnU7RXfsw+/bukWGo1abgBiMAic068rclZsO4IWmmxQ==",
"dev": true,
+ "license": "ISC",
"engines": {
- "node": ">= 10"
+ "node": ">= 12"
}
},
"node_modules/client-only": {
@@ -6622,15 +6625,6 @@
"url": "https://github.com/chalk/wrap-ansi?sponsor=1"
}
},
- "node_modules/clone": {
- "version": "1.0.4",
- "resolved": "https://registry.npmjs.org/clone/-/clone-1.0.4.tgz",
- "integrity": "sha512-JQHZ2QMW6l3aH/j6xCqQThY/9OH4D/9ls34cgkUBiEeocRTU04tHfKPBsUK1PqZCUQM7GiA0IIXJSuXHI64Kbg==",
- "dev": true,
- "engines": {
- "node": ">=0.8"
- }
- },
"node_modules/clsx": {
"version": "1.2.1",
"resolved": "https://registry.npmjs.org/clsx/-/clsx-1.2.1.tgz",
@@ -6770,10 +6764,11 @@
"dev": true
},
"node_modules/cookie": {
- "version": "0.4.2",
- "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.4.2.tgz",
- "integrity": "sha512-aSWTXFzaKWkvHO1Ny/s+ePFpvKsPnjc551iI41v3ny/ow6tBG5Vd+FuqGNhh1LxOmVzOlGUriIlOaokOvhaStA==",
+ "version": "0.7.2",
+ "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.7.2.tgz",
+ "integrity": "sha512-yki5XnKuf750l50uGTllt6kKILY4nQ1eNIQatoXEByZ5dWgnKqbnqmTrBE5B4N7lrMJKQ2ytWMiTO2o0v6Ew/w==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">= 0.6"
}
@@ -7569,18 +7564,6 @@
"integrity": "sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==",
"dev": true
},
- "node_modules/defaults": {
- "version": "1.0.4",
- "resolved": "https://registry.npmjs.org/defaults/-/defaults-1.0.4.tgz",
- "integrity": "sha512-eFuaLoy/Rxalv2kr+lqMlUnrDWV+3j4pljOIJgLIhI058IQfWJ7vXhyEIHu+HtC738klGALYxOKDO0bQP3tg8A==",
- "dev": true,
- "dependencies": {
- "clone": "^1.0.2"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
"node_modules/define-data-property": {
"version": "1.1.4",
"resolved": "https://registry.npmjs.org/define-data-property/-/define-data-property-1.1.4.tgz",
@@ -8548,15 +8531,6 @@
"node": ">=0.10.0"
}
},
- "node_modules/events": {
- "version": "3.3.0",
- "resolved": "https://registry.npmjs.org/events/-/events-3.3.0.tgz",
- "integrity": "sha512-mQw+2fkQbALzQ7V0MY0IqdnXNOeTtP4r0lN9z7AAawCXgqea7bDii20AYrIBrFd/Hx0M2Ocz6S111CaFkUcb0Q==",
- "dev": true,
- "engines": {
- "node": ">=0.8.x"
- }
- },
"node_modules/execa": {
"version": "5.1.1",
"resolved": "https://registry.npmjs.org/execa/-/execa-5.1.1.tgz",
@@ -8595,20 +8569,6 @@
"node": "^14.15.0 || ^16.10.0 || >=18.0.0"
}
},
- "node_modules/external-editor": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/external-editor/-/external-editor-3.1.0.tgz",
- "integrity": "sha512-hMQ4CX1p1izmuLYyZqLMO/qGNw10wSv9QDCPfzXfyFrOaCSSoRfqE1Kf1s5an66J5JZC62NewG+mK49jOCtQew==",
- "dev": true,
- "dependencies": {
- "chardet": "^0.7.0",
- "iconv-lite": "^0.4.24",
- "tmp": "^0.0.33"
- },
- "engines": {
- "node": ">=4"
- }
- },
"node_modules/fast-deep-equal": {
"version": "3.1.3",
"resolved": "https://registry.npmjs.org/fast-deep-equal/-/fast-deep-equal-3.1.3.tgz",
@@ -8682,30 +8642,6 @@
"reusify": "^1.0.4"
}
},
- "node_modules/figures": {
- "version": "3.2.0",
- "resolved": "https://registry.npmjs.org/figures/-/figures-3.2.0.tgz",
- "integrity": "sha512-yaduQFRKLXYOGgEn6AZau90j3ggSOyiqXU0F9JZfeXYhNa+Jk4X+s45A2zg5jns87GAFa34BBm2kXw4XpNcbdg==",
- "dev": true,
- "dependencies": {
- "escape-string-regexp": "^1.0.5"
- },
- "engines": {
- "node": ">=8"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
- "node_modules/figures/node_modules/escape-string-regexp": {
- "version": "1.0.5",
- "resolved": "https://registry.npmjs.org/escape-string-regexp/-/escape-string-regexp-1.0.5.tgz",
- "integrity": "sha512-vbRorB5FUQWvla16U8R/qgaFIya2qGzwDrNmCZuYKrbdSUMG6I1ZCGQRefkRVhuOkIGVne7BQ35DSfo1qvJqFg==",
- "dev": true,
- "engines": {
- "node": ">=0.8.0"
- }
- },
"node_modules/file-entry-cache": {
"version": "6.0.1",
"resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-6.0.1.tgz",
@@ -9140,12 +9076,13 @@
"dev": true
},
"node_modules/graphql": {
- "version": "15.8.0",
- "resolved": "https://registry.npmjs.org/graphql/-/graphql-15.8.0.tgz",
- "integrity": "sha512-5gghUc24tP9HRznNpV2+FIoq3xKkj5dTQqf4v0CpdPbFVwFkWoxOM+o+2OC9ZSvjEMTjfmG9QT+gcvggTwW1zw==",
+ "version": "16.9.0",
+ "resolved": "https://registry.npmjs.org/graphql/-/graphql-16.9.0.tgz",
+ "integrity": "sha512-GGTKBX4SD7Wdb8mqeDLni2oaRGYQWjWHGKPQ24ZMnUtKfcsVoiv4uX8+LJr1K6U5VW2Lu1BwJnj7uiori0YtRw==",
"dev": true,
+ "license": "MIT",
"engines": {
- "node": ">= 10.x"
+ "node": "^12.22.0 || ^14.16.0 || ^16.0.0 || >=17.0.0"
}
},
"node_modules/hard-rejection": {
@@ -9237,11 +9174,12 @@
"node": ">= 0.4"
}
},
- "node_modules/headers-utils": {
- "version": "3.0.2",
- "resolved": "https://registry.npmjs.org/headers-utils/-/headers-utils-3.0.2.tgz",
- "integrity": "sha512-xAxZkM1dRyGV2Ou5bzMxBPNLoRCjcX+ya7KSWybQD2KwLphxsapUVK6x/02o7f4VU6GPSXch9vNY2+gkU8tYWQ==",
- "dev": true
+ "node_modules/headers-polyfill": {
+ "version": "4.0.3",
+ "resolved": "https://registry.npmjs.org/headers-polyfill/-/headers-polyfill-4.0.3.tgz",
+ "integrity": "sha512-IScLbePpkvO846sIwOtOTDjutRMWdXdJmXdMvk6gCBHxFO8d+QKOQedyZSxFTTFYRSmlgSTDtXqqq4pcenBXLQ==",
+ "dev": true,
+ "license": "MIT"
},
"node_modules/highcharts": {
"version": "10.3.3",
@@ -9378,38 +9316,6 @@
"node": ">=0.4"
}
},
- "node_modules/iconv-lite": {
- "version": "0.4.24",
- "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.24.tgz",
- "integrity": "sha512-v3MXnZAcvnywkTUEZomIActle7RXXeedOR31wwl7VlyoXO4Qi9arvSenNQWne1TcRwhCL1HwLI21bEqdpj8/rA==",
- "dev": true,
- "dependencies": {
- "safer-buffer": ">= 2.1.2 < 3"
- },
- "engines": {
- "node": ">=0.10.0"
- }
- },
- "node_modules/ieee754": {
- "version": "1.2.1",
- "resolved": "https://registry.npmjs.org/ieee754/-/ieee754-1.2.1.tgz",
- "integrity": "sha512-dcyqhDvX1C46lXZcVqCpK+FtMRQVdIMN6/Df5js2zouUsqG7I6sFxitIC+7KYK29KdXOLHdu9zL4sFnoVQnqaA==",
- "dev": true,
- "funding": [
- {
- "type": "github",
- "url": "https://github.com/sponsors/feross"
- },
- {
- "type": "patreon",
- "url": "https://www.patreon.com/feross"
- },
- {
- "type": "consulting",
- "url": "https://feross.org/support"
- }
- ]
- },
"node_modules/ignore": {
"version": "5.3.1",
"resolved": "https://registry.npmjs.org/ignore/-/ignore-5.3.1.tgz",
@@ -9487,32 +9393,6 @@
"integrity": "sha512-JV/yugV2uzW5iMRSiZAyDtQd+nxtUnjeLt0acNdw98kKLrvuRVyB80tsREOE7yvGVgalhZ6RNXCmEHkUKBKxew==",
"dev": true
},
- "node_modules/inquirer": {
- "version": "8.2.6",
- "resolved": "https://registry.npmjs.org/inquirer/-/inquirer-8.2.6.tgz",
- "integrity": "sha512-M1WuAmb7pn9zdFRtQYk26ZBoY043Sse0wVDdk4Bppr+JOXyQYybdtvK+l9wUibhtjdjvtoiNy8tk+EgsYIUqKg==",
- "dev": true,
- "dependencies": {
- "ansi-escapes": "^4.2.1",
- "chalk": "^4.1.1",
- "cli-cursor": "^3.1.0",
- "cli-width": "^3.0.0",
- "external-editor": "^3.0.3",
- "figures": "^3.0.0",
- "lodash": "^4.17.21",
- "mute-stream": "0.0.8",
- "ora": "^5.4.1",
- "run-async": "^2.4.0",
- "rxjs": "^7.5.5",
- "string-width": "^4.1.0",
- "strip-ansi": "^6.0.0",
- "through": "^2.3.6",
- "wrap-ansi": "^6.0.1"
- },
- "engines": {
- "node": ">=12.0.0"
- }
- },
"node_modules/internal-slot": {
"version": "1.0.7",
"resolved": "https://registry.npmjs.org/internal-slot/-/internal-slot-1.0.7.tgz",
@@ -9735,15 +9615,6 @@
"node": ">=0.10.0"
}
},
- "node_modules/is-interactive": {
- "version": "1.0.0",
- "resolved": "https://registry.npmjs.org/is-interactive/-/is-interactive-1.0.0.tgz",
- "integrity": "sha512-2HvIEKRoqS62guEC+qBjpvRubdX910WCMuJTZ+I9yvqKU2/12eSL549HMwtabb4oupdj2sMP50k+XJfB/8JE6w==",
- "dev": true,
- "engines": {
- "node": ">=8"
- }
- },
"node_modules/is-map": {
"version": "2.0.3",
"resolved": "https://registry.npmjs.org/is-map/-/is-map-2.0.3.tgz",
@@ -9772,7 +9643,8 @@
"version": "1.2.0",
"resolved": "https://registry.npmjs.org/is-node-process/-/is-node-process-1.2.0.tgz",
"integrity": "sha512-Vg4o6/fqPxIjtxgUH5QLJhwZ7gW5diGCVlXpuUfELC62CuxM1iHcRe51f2W1FDy04Ai4KJkagKjx3XaqyfRKXw==",
- "dev": true
+ "dev": true,
+ "license": "MIT"
},
"node_modules/is-number": {
"version": "7.0.0",
@@ -9952,18 +9824,6 @@
"url": "https://github.com/sponsors/ljharb"
}
},
- "node_modules/is-unicode-supported": {
- "version": "0.1.0",
- "resolved": "https://registry.npmjs.org/is-unicode-supported/-/is-unicode-supported-0.1.0.tgz",
- "integrity": "sha512-knxG2q4UC3u8stRGyAVJCOdxFmv5DZiRcdlIaAQXAbSfJya+OhopNotLQrstBhququ4ZpuKbDc/8S6mgXgPFPw==",
- "dev": true,
- "engines": {
- "node": ">=10"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
"node_modules/is-weakmap": {
"version": "2.0.2",
"resolved": "https://registry.npmjs.org/is-weakmap/-/is-weakmap-2.0.2.tgz",
@@ -10231,15 +10091,6 @@
"jiti": "bin/jiti.js"
}
},
- "node_modules/js-levenshtein": {
- "version": "1.1.6",
- "resolved": "https://registry.npmjs.org/js-levenshtein/-/js-levenshtein-1.1.6.tgz",
- "integrity": "sha512-X2BB11YZtrRqY4EnQcLX5Rh373zbK4alC1FW7D7MBhL2gtcC17cTnr6DmfHZeS0s2rTHjUTMMHfG7gO8SSdw+g==",
- "dev": true,
- "engines": {
- "node": ">=0.10.0"
- }
- },
"node_modules/js-sha3": {
"version": "0.8.0",
"resolved": "https://registry.npmjs.org/js-sha3/-/js-sha3-0.8.0.tgz",
@@ -10561,22 +10412,6 @@
"integrity": "sha512-jttmRe7bRse52OsWIMDLaXxWqRAmtIUccAQ3garviCqJjafXOfNMO0yMfNpdD6zbGaTU0P5Nz7e7gAT6cKmJRw==",
"dev": true
},
- "node_modules/log-symbols": {
- "version": "4.1.0",
- "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-4.1.0.tgz",
- "integrity": "sha512-8XPvpAA8uyhfteu8pIvQxpJZ7SYYdpUivZpGy6sFsBuKRY/7rQGavedeB8aK+Zkyq6upMFVL/9AW6vOYzfRyLg==",
- "dev": true,
- "dependencies": {
- "chalk": "^4.1.0",
- "is-unicode-supported": "^0.1.0"
- },
- "engines": {
- "node": ">=10"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
"node_modules/long": {
"version": "5.2.3",
"resolved": "https://registry.npmjs.org/long/-/long-5.2.3.tgz",
@@ -10777,11 +10612,11 @@
}
},
"node_modules/micromatch": {
- "version": "4.0.5",
- "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.5.tgz",
- "integrity": "sha512-DMy+ERcEW2q8Z2Po+WNXuw3c5YaUSFjAO5GsJqfEl7UjvtIuFKO6ZrKvcItdy98dwFI2N1tg3zNIdKaQT+aNdA==",
+ "version": "4.0.8",
+ "resolved": "https://registry.npmjs.org/micromatch/-/micromatch-4.0.8.tgz",
+ "integrity": "sha512-PXwfBhYu0hBCPw8Dn0E+WDYb7af3dSLVWKi3HGv84IdF4TyFoC0ysxFd0Goxw7nSv4T/PzEJQxsYsEiFCKo2BA==",
"dependencies": {
- "braces": "^3.0.2",
+ "braces": "^3.0.3",
"picomatch": "^2.3.1"
},
"engines": {
@@ -10906,62 +10741,107 @@
"dev": true
},
"node_modules/msw": {
- "version": "0.36.8",
- "resolved": "https://registry.npmjs.org/msw/-/msw-0.36.8.tgz",
- "integrity": "sha512-K7lOQoYqhGhTSChsmHMQbf/SDCsxh/m0uhN6Ipt206lGoe81fpTmaGD0KLh4jUxCONMOUnwCSj0jtX2CM4pEdw==",
+ "version": "2.6.0",
+ "resolved": "https://registry.npmjs.org/msw/-/msw-2.6.0.tgz",
+ "integrity": "sha512-n3tx2w0MZ3H4pxY0ozrQ4sNPzK/dGtlr2cIIyuEsgq2Bhy4wvcW6ZH2w/gXM9+MEUY6HC1fWhqtcXDxVZr5Jxw==",
"dev": true,
"hasInstallScript": true,
+ "license": "MIT",
"dependencies": {
- "@mswjs/cookies": "^0.1.7",
- "@mswjs/interceptors": "^0.12.7",
- "@open-draft/until": "^1.0.3",
- "@types/cookie": "^0.4.1",
- "@types/inquirer": "^8.1.3",
- "@types/js-levenshtein": "^1.1.0",
- "chalk": "4.1.1",
- "chokidar": "^3.4.2",
- "cookie": "^0.4.1",
- "graphql": "^15.5.1",
- "headers-utils": "^3.0.2",
- "inquirer": "^8.2.0",
- "is-node-process": "^1.0.1",
- "js-levenshtein": "^1.1.6",
- "node-fetch": "^2.6.7",
- "path-to-regexp": "^6.2.0",
- "statuses": "^2.0.0",
- "strict-event-emitter": "^0.2.0",
- "type-fest": "^1.2.2",
- "yargs": "^17.3.0"
+ "@bundled-es-modules/cookie": "^2.0.0",
+ "@bundled-es-modules/statuses": "^1.0.1",
+ "@bundled-es-modules/tough-cookie": "^0.1.6",
+ "@inquirer/confirm": "^5.0.0",
+ "@mswjs/interceptors": "^0.36.5",
+ "@open-draft/deferred-promise": "^2.2.0",
+ "@open-draft/until": "^2.1.0",
+ "@types/cookie": "^0.6.0",
+ "@types/statuses": "^2.0.4",
+ "chalk": "^4.1.2",
+ "graphql": "^16.8.1",
+ "headers-polyfill": "^4.0.2",
+ "is-node-process": "^1.2.0",
+ "outvariant": "^1.4.3",
+ "path-to-regexp": "^6.3.0",
+ "strict-event-emitter": "^0.5.1",
+ "type-fest": "^4.26.1",
+ "yargs": "^17.7.2"
},
"bin": {
"msw": "cli/index.js"
},
+ "engines": {
+ "node": ">=18"
+ },
"funding": {
- "type": "opencollective",
- "url": "https://opencollective.com/mswjs"
+ "url": "https://github.com/sponsors/mswjs"
+ },
+ "peerDependencies": {
+ "typescript": ">= 4.8.x"
+ },
+ "peerDependenciesMeta": {
+ "typescript": {
+ "optional": true
+ }
}
},
- "node_modules/msw/node_modules/chalk": {
- "version": "4.1.1",
- "resolved": "https://registry.npmjs.org/chalk/-/chalk-4.1.1.tgz",
- "integrity": "sha512-diHzdDKxcU+bAsUboHLPEDQiw0qEe0qd7SYUn3HgcFlWgbDcfLGswOHYeGrHKzG9z6UYf01d9VFMfZxPM1xZSg==",
+ "node_modules/msw/node_modules/@inquirer/confirm": {
+ "version": "5.0.1",
+ "resolved": "https://registry.npmjs.org/@inquirer/confirm/-/confirm-5.0.1.tgz",
+ "integrity": "sha512-6ycMm7k7NUApiMGfVc32yIPp28iPKxhGRMqoNDiUjq2RyTAkbs5Fx0TdzBqhabcKvniDdAAvHCmsRjnNfTsogw==",
"dev": true,
+ "license": "MIT",
"dependencies": {
- "ansi-styles": "^4.1.0",
- "supports-color": "^7.1.0"
+ "@inquirer/core": "^10.0.1",
+ "@inquirer/type": "^3.0.0"
},
"engines": {
- "node": ">=10"
+ "node": ">=18"
},
- "funding": {
- "url": "https://github.com/chalk/chalk?sponsor=1"
+ "peerDependencies": {
+ "@types/node": ">=18"
+ }
+ },
+ "node_modules/msw/node_modules/@inquirer/type": {
+ "version": "3.0.0",
+ "resolved": "https://registry.npmjs.org/@inquirer/type/-/type-3.0.0.tgz",
+ "integrity": "sha512-YYykfbw/lefC7yKj7nanzQXILM7r3suIvyFlCcMskc99axmsSewXWkAfXKwMbgxL76iAFVmRwmYdwNZNc8gjog==",
+ "dev": true,
+ "license": "MIT",
+ "engines": {
+ "node": ">=18"
+ },
+ "peerDependencies": {
+ "@types/node": ">=18"
}
},
+ "node_modules/msw/node_modules/@types/node": {
+ "version": "22.9.0",
+ "resolved": "https://registry.npmjs.org/@types/node/-/node-22.9.0.tgz",
+ "integrity": "sha512-vuyHg81vvWA1Z1ELfvLko2c8f34gyA0zaic0+Rllc5lbCnbSyuvb2Oxpm6TAUAC/2xZN3QGqxBNggD1nNR2AfQ==",
+ "dev": true,
+ "license": "MIT",
+ "peer": true,
+ "dependencies": {
+ "undici-types": "~6.19.8"
+ }
+ },
+ "node_modules/msw/node_modules/path-to-regexp": {
+ "version": "6.3.0",
+ "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-6.3.0.tgz",
+ "integrity": "sha512-Yhpw4T9C6hPpgPeA28us07OJeqZ5EzQTkbfwuhsUg0c237RomFoETJgmp2sa3F/41gfLE6G5cqcYwznmeEeOlQ==",
+ "dev": true,
+ "license": "MIT"
+ },
"node_modules/mute-stream": {
- "version": "0.0.8",
- "resolved": "https://registry.npmjs.org/mute-stream/-/mute-stream-0.0.8.tgz",
- "integrity": "sha512-nnbWWOkoWyUsTjKrhgD0dcz22mdkSnpYqbEjIm2nhwhuxlSkpywJmBo8h0ZqJdkp73mb90SssHkN4rsRaBAfAA==",
- "dev": true
+ "version": "2.0.0",
+ "resolved": "https://registry.npmjs.org/mute-stream/-/mute-stream-2.0.0.tgz",
+ "integrity": "sha512-WWdIxpyjEn+FhQJQQv9aQAYlHoNVdzIzUySNV1gHUPDSdZJ3yZn7pAAbQcV7B56Mvu881q9FZV+0Vx2xC44VWA==",
+ "dev": true,
+ "license": "ISC",
+ "engines": {
+ "node": "^18.17.0 || >=20.5.0"
+ }
},
"node_modules/mz": {
"version": "2.7.0",
@@ -10983,15 +10863,16 @@
}
},
"node_modules/nanoid": {
- "version": "3.3.7",
- "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.7.tgz",
- "integrity": "sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g==",
+ "version": "3.3.8",
+ "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.8.tgz",
+ "integrity": "sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w==",
"funding": [
{
"type": "github",
"url": "https://github.com/sponsors/ai"
}
],
+ "license": "MIT",
"bin": {
"nanoid": "bin/nanoid.cjs"
},
@@ -13947,43 +13828,12 @@
"node": ">= 0.8.0"
}
},
- "node_modules/ora": {
- "version": "5.4.1",
- "resolved": "https://registry.npmjs.org/ora/-/ora-5.4.1.tgz",
- "integrity": "sha512-5b6Y85tPxZZ7QytO+BQzysW31HJku27cRIlkbAXaNx+BdcVi+LlRFmVXzeF6a7JCwJpyw5c4b+YSVImQIrBpuQ==",
- "dev": true,
- "dependencies": {
- "bl": "^4.1.0",
- "chalk": "^4.1.0",
- "cli-cursor": "^3.1.0",
- "cli-spinners": "^2.5.0",
- "is-interactive": "^1.0.0",
- "is-unicode-supported": "^0.1.0",
- "log-symbols": "^4.1.0",
- "strip-ansi": "^6.0.0",
- "wcwidth": "^1.0.1"
- },
- "engines": {
- "node": ">=10"
- },
- "funding": {
- "url": "https://github.com/sponsors/sindresorhus"
- }
- },
- "node_modules/os-tmpdir": {
- "version": "1.0.2",
- "resolved": "https://registry.npmjs.org/os-tmpdir/-/os-tmpdir-1.0.2.tgz",
- "integrity": "sha512-D2FR03Vir7FIu45XBY20mTb+/ZSWB00sjU9jdQXt83gDrI4Ztz5Fs7/yy74g2N5SVQY4xY1qDr4rNddwYRVX0g==",
- "dev": true,
- "engines": {
- "node": ">=0.10.0"
- }
- },
"node_modules/outvariant": {
- "version": "1.4.2",
- "resolved": "https://registry.npmjs.org/outvariant/-/outvariant-1.4.2.tgz",
- "integrity": "sha512-Ou3dJ6bA/UJ5GVHxah4LnqDwZRwAmWxrG3wtrHrbGnP4RnLCtA64A4F+ae7Y8ww660JaddSoArUR5HjipWSHAQ==",
- "dev": true
+ "version": "1.4.3",
+ "resolved": "https://registry.npmjs.org/outvariant/-/outvariant-1.4.3.tgz",
+ "integrity": "sha512-+Sl2UErvtsoajRDKCE5/dBz4DIvHXQQnAxtQTF04OJxY0+DyZXSo5P5Bb7XYWOh81syohlYL24hbDwxedPUJCA==",
+ "dev": true,
+ "license": "MIT"
},
"node_modules/p-limit": {
"version": "3.1.0",
@@ -14130,10 +13980,12 @@
}
},
"node_modules/path-to-regexp": {
- "version": "6.2.2",
- "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-6.2.2.tgz",
- "integrity": "sha512-GQX3SSMokngb36+whdpRXE+3f9V8UzyAorlYvOGx87ufGHehNTn5lCxrKtLyZ4Yl/wEKnNnr98ZzOwwDZV5ogw==",
- "dev": true
+ "version": "8.1.0",
+ "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-8.1.0.tgz",
+ "integrity": "sha512-Bqn3vc8CMHty6zuD+tG23s6v2kwxslHEhTj4eYaVKGIEB+YX/2wd0/rgXLFD9G9id9KCtbVy/3ZgmvZjpa0UdQ==",
+ "engines": {
+ "node": ">=16"
+ }
},
"node_modules/path-type": {
"version": "4.0.0",
@@ -15479,7 +15331,7 @@
"history": "^4.9.0",
"hoist-non-react-statics": "^3.1.0",
"loose-envify": "^1.3.1",
- "path-to-regexp": "^1.7.0",
+ "path-to-regexp": "^1.9.0",
"prop-types": "^15.6.2",
"react-is": "^16.6.0",
"tiny-invariant": "^1.0.2",
@@ -15538,9 +15390,9 @@
"integrity": "sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ=="
},
"node_modules/react-router/node_modules/path-to-regexp": {
- "version": "1.8.0",
- "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-1.8.0.tgz",
- "integrity": "sha512-n43JRhlUKUAlibEJhPeir1ncUID16QnEjNpwzNdO3Lm4ywrBpBZ5oLD0I6br9evr1Y9JTqwRtAh7JLoOzAQdVA==",
+ "version": "1.9.0",
+ "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-1.9.0.tgz",
+ "integrity": "sha512-xIp7/apCFJuUHdDLWe8O1HIkb0kQrOMb/0u6FXQjemHn/ii5LrIzU6bdECnsiTF/GjZkMEKg1xdiZwNqDYlZ6g==",
"dependencies": {
"isarray": "0.0.1"
}
@@ -15709,20 +15561,6 @@
"node": ">=8"
}
},
- "node_modules/readable-stream": {
- "version": "3.6.2",
- "resolved": "https://registry.npmjs.org/readable-stream/-/readable-stream-3.6.2.tgz",
- "integrity": "sha512-9u/sniCrY3D5WdsERHzHE4G2YCXqoG5FTHUiCC4SIbr6XcLZBY05ya9EKjYek9O5xOAwjGq+1JdGBAS7Q9ScoA==",
- "dev": true,
- "dependencies": {
- "inherits": "^2.0.3",
- "string_decoder": "^1.1.1",
- "util-deprecate": "^1.0.1"
- },
- "engines": {
- "node": ">= 6"
- }
- },
"node_modules/readdirp": {
"version": "3.6.0",
"resolved": "https://registry.npmjs.org/readdirp/-/readdirp-3.6.0.tgz",
@@ -15947,19 +15785,6 @@
"resolved": "https://registry.npmjs.org/resolve-pathname/-/resolve-pathname-3.0.0.tgz",
"integrity": "sha512-C7rARubxI8bXFNB/hqcp/4iUeIXJhJZvFPFPiSPRnhU5UPxzMFIl+2E6yY6c4k9giDJAhtV+enfA+G89N6Csng=="
},
- "node_modules/restore-cursor": {
- "version": "3.1.0",
- "resolved": "https://registry.npmjs.org/restore-cursor/-/restore-cursor-3.1.0.tgz",
- "integrity": "sha512-l+sSefzHpj5qimhFSE5a8nufZYAM3sBSVMAPtYkmC+4EH2anSGaEMXSD0izRQbu9nfyQ9y5JrVmp7E8oZrUjvA==",
- "dev": true,
- "dependencies": {
- "onetime": "^5.1.0",
- "signal-exit": "^3.0.2"
- },
- "engines": {
- "node": ">=8"
- }
- },
"node_modules/reusify": {
"version": "1.0.4",
"resolved": "https://registry.npmjs.org/reusify/-/reusify-1.0.4.tgz",
@@ -16009,10 +15834,11 @@
"integrity": "sha512-IXgzBWvWQwE6PrDI05OvmXUIruQTcoMDzRsOd5CDvHCVLcLHMTSYvOK5Cm46kWqlV3yAbuSpBZdJ5oP5OUoStg=="
},
"node_modules/rollup": {
- "version": "3.29.4",
- "resolved": "https://registry.npmjs.org/rollup/-/rollup-3.29.4.tgz",
- "integrity": "sha512-oWzmBZwvYrU0iJHtDmhsm662rC15FRXmcjCk1xD771dFDx5jJ02ufAQQTn0etB2emNk4J9EZg/yWKpsn9BWGRw==",
+ "version": "3.29.5",
+ "resolved": "https://registry.npmjs.org/rollup/-/rollup-3.29.5.tgz",
+ "integrity": "sha512-GVsDdsbJzzy4S/v3dqWPJ7EfvZJfCHiDqe80IyrF59LYuP+e6U1LJoUqeuqRbwAWoMNoXivMNeNAOf5E22VA1w==",
"dev": true,
+ "license": "MIT",
"bin": {
"rollup": "dist/bin/rollup"
},
@@ -16030,15 +15856,6 @@
"integrity": "sha512-APM0Gt1KoXBz0iIkkdB/kfvGOwC4UuJFeG/c+yV7wSc7q96cG/kJ0HiYCnzivD9SB53cLV1MlHFNfOuPaadYSw==",
"dev": true
},
- "node_modules/run-async": {
- "version": "2.4.1",
- "resolved": "https://registry.npmjs.org/run-async/-/run-async-2.4.1.tgz",
- "integrity": "sha512-tvVnVv01b8c1RrA6Ep7JkStj85Guv/YrMcwqYQnwjsAS2cTmmPGBBjAjpCW7RrSodNSoE2/qg9O4bceNvUuDgQ==",
- "dev": true,
- "engines": {
- "node": ">=0.12.0"
- }
- },
"node_modules/run-parallel": {
"version": "1.2.0",
"resolved": "https://registry.npmjs.org/run-parallel/-/run-parallel-1.2.0.tgz",
@@ -16067,15 +15884,6 @@
"resolved": "https://registry.npmjs.org/rw/-/rw-1.3.3.tgz",
"integrity": "sha512-PdhdWy89SiZogBLaw42zdeqtRJ//zFd2PgQavcICDUgJT5oW10QCRKbJ6bg4r0/UY2M6BWd5tkxuGFRvCkgfHQ=="
},
- "node_modules/rxjs": {
- "version": "7.8.1",
- "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-7.8.1.tgz",
- "integrity": "sha512-AA3TVj+0A2iuIoQkWEK/tqFjBq2j+6PO6Y0zJcvzLAFhEFIO3HL0vls9hWLncZbAAbK0mar7oZ4V079I/qPMxg==",
- "dev": true,
- "dependencies": {
- "tslib": "^2.1.0"
- }
- },
"node_modules/safe-array-concat": {
"version": "1.1.2",
"resolved": "https://registry.npmjs.org/safe-array-concat/-/safe-array-concat-1.1.2.tgz",
@@ -16205,7 +16013,7 @@
"mime-types": "2.1.18",
"minimatch": "3.1.2",
"path-is-inside": "1.0.2",
- "path-to-regexp": "2.2.1",
+ "path-to-regexp": "^3.3.0",
"range-parser": "1.2.0"
}
},
@@ -16231,9 +16039,9 @@
}
},
"node_modules/serve-handler/node_modules/path-to-regexp": {
- "version": "2.2.1",
- "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-2.2.1.tgz",
- "integrity": "sha512-gu9bD6Ta5bwGrrU8muHzVOBFFREpp2iRkVfhBJahwJ6p6Xw20SjT0MxLnwkjOibQmGSYhiUnf2FLe7k+jcFmGQ==",
+ "version": "3.3.0",
+ "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-3.3.0.tgz",
+ "integrity": "sha512-qyCH421YQPS2WFDxDjftfc1ZR5WKQzVzqsp4n9M2kQhVOo/ByahFoUNJfl58kOcEGfQ//7weFTDhm+ss8Ecxgw==",
"dev": true
},
"node_modules/serve/node_modules/ajv": {
@@ -16270,12 +16078,6 @@
"integrity": "sha512-NM8/P9n3XjXhIZn1lLhkFaACTOURQXjWhV4BA/RnOv8xvgqtqpAX9IO4mRQxSx1Rlo4tqzeqb0sOlruaOy3dug==",
"dev": true
},
- "node_modules/set-cookie-parser": {
- "version": "2.6.0",
- "resolved": "https://registry.npmjs.org/set-cookie-parser/-/set-cookie-parser-2.6.0.tgz",
- "integrity": "sha512-RVnVQxTXuerk653XfuliOxBP81Sf0+qfQE73LIYKcyMYHG94AuH0kgrQpRDuTZnSmjpysHmzxJXKNfa6PjFhyQ==",
- "dev": true
- },
"node_modules/set-function-length": {
"version": "1.2.2",
"resolved": "https://registry.npmjs.org/set-function-length/-/set-function-length-1.2.2.tgz",
@@ -16484,6 +16286,7 @@
"resolved": "https://registry.npmjs.org/statuses/-/statuses-2.0.1.tgz",
"integrity": "sha512-RwNA9Z/7PrK06rYLIzFMlaF+l73iwpzsqRIFgbMLbTcLD6cOao82TaWefPXQvB2fOC4AjuYSEndS7N/mTCbkdQ==",
"dev": true,
+ "license": "MIT",
"engines": {
"node": ">= 0.8"
}
@@ -16495,13 +16298,11 @@
"dev": true
},
"node_modules/strict-event-emitter": {
- "version": "0.2.8",
- "resolved": "https://registry.npmjs.org/strict-event-emitter/-/strict-event-emitter-0.2.8.tgz",
- "integrity": "sha512-KDf/ujU8Zud3YaLtMCcTI4xkZlZVIYxTLr+XIULexP+77EEVWixeXroLUXQXiVtH4XH2W7jr/3PT1v3zBuvc3A==",
+ "version": "0.5.1",
+ "resolved": "https://registry.npmjs.org/strict-event-emitter/-/strict-event-emitter-0.5.1.tgz",
+ "integrity": "sha512-vMgjE/GGEPEFnhFub6pa4FmJBRBVOLpIII2hvCZ8Kzb7K0hlHo7mQv6xYrBvCL2LtAIBwFUK8wvuJgTVSQ5MFQ==",
"dev": true,
- "dependencies": {
- "events": "^3.3.0"
- }
+ "license": "MIT"
},
"node_modules/strict-uri-encode": {
"version": "2.0.0",
@@ -16511,35 +16312,6 @@
"node": ">=4"
}
},
- "node_modules/string_decoder": {
- "version": "1.3.0",
- "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.3.0.tgz",
- "integrity": "sha512-hkRX8U1WjJFd8LsDJ2yQ/wWWxaopEsABU1XfkM8A+j0+85JAGppt16cr1Whg6KIbb4okU6Mql6BOj+uup/wKeA==",
- "dev": true,
- "dependencies": {
- "safe-buffer": "~5.2.0"
- }
- },
- "node_modules/string_decoder/node_modules/safe-buffer": {
- "version": "5.2.1",
- "resolved": "https://registry.npmjs.org/safe-buffer/-/safe-buffer-5.2.1.tgz",
- "integrity": "sha512-rp3So07KcdmmKbGvgaNxQSJr7bGVSVk5S9Eq1F+ppbRo70+YeaDxkw5Dd8NPN+GD6bjnYm2VuPuCXmpuYvmCXQ==",
- "dev": true,
- "funding": [
- {
- "type": "github",
- "url": "https://github.com/sponsors/feross"
- },
- {
- "type": "patreon",
- "url": "https://www.patreon.com/feross"
- },
- {
- "type": "consulting",
- "url": "https://feross.org/support"
- }
- ]
- },
"node_modules/string-natural-compare": {
"version": "3.0.1",
"resolved": "https://registry.npmjs.org/string-natural-compare/-/string-natural-compare-3.0.1.tgz",
@@ -17154,12 +16926,6 @@
"node": ">=0.8"
}
},
- "node_modules/through": {
- "version": "2.3.8",
- "resolved": "https://registry.npmjs.org/through/-/through-2.3.8.tgz",
- "integrity": "sha512-w89qg7PI8wAdvX60bMDP+bFoD5Dvhm9oLheFp5O4a2QF0cSBGsBX4qZmadPMvVqlLJBBci+WqGGOAPvcDeNSVg==",
- "dev": true
- },
"node_modules/tiny-invariant": {
"version": "1.3.3",
"resolved": "https://registry.npmjs.org/tiny-invariant/-/tiny-invariant-1.3.3.tgz",
@@ -17194,18 +16960,6 @@
"node": ">=14.0.0"
}
},
- "node_modules/tmp": {
- "version": "0.0.33",
- "resolved": "https://registry.npmjs.org/tmp/-/tmp-0.0.33.tgz",
- "integrity": "sha512-jRCJlojKnZ3addtTOjdIqoRuPEKBvNXcGYqzO6zWZX8KfKEpnGY5jfggJQ3EjKuu8D4bJRr0y+cYJFmYbImXGw==",
- "dev": true,
- "dependencies": {
- "os-tmpdir": "~1.0.2"
- },
- "engines": {
- "node": ">=0.6.0"
- }
- },
"node_modules/to-fast-properties": {
"version": "2.0.0",
"resolved": "https://registry.npmjs.org/to-fast-properties/-/to-fast-properties-2.0.0.tgz",
@@ -17359,12 +17113,13 @@
}
},
"node_modules/type-fest": {
- "version": "1.4.0",
- "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-1.4.0.tgz",
- "integrity": "sha512-yGSza74xk0UG8k+pLh5oeoYirvIiWo5t0/o3zHHAO2tRDiZcxWP7fywNlXhqb6/r6sWvwi+RsyQMWhVLe4BVuA==",
+ "version": "4.26.1",
+ "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-4.26.1.tgz",
+ "integrity": "sha512-yOGpmOAL7CkKe/91I5O3gPICmJNLJ1G4zFYVAsRHg7M64biSnPtRj0WNQt++bRkjYOqjWXrhnUw1utzmVErAdg==",
"dev": true,
+ "license": "(MIT OR CC0-1.0)",
"engines": {
- "node": ">=10"
+ "node": ">=16"
},
"funding": {
"url": "https://github.com/sponsors/sindresorhus"
@@ -17501,6 +17256,14 @@
"integrity": "sha512-+A5Sja4HP1M08MaXya7p5LvjuM7K6q/2EaC0+iovj/wOcMsTzMvDFbasi/oSapiwOlt252IqsKqPjCl7huKS0A==",
"dev": true
},
+ "node_modules/undici-types": {
+ "version": "6.19.8",
+ "resolved": "https://registry.npmjs.org/undici-types/-/undici-types-6.19.8.tgz",
+ "integrity": "sha512-ve2KP6f/JnbPBFyobGHuerC9g1FYGn/F8n1LWTwNxCEzd6IfqTwUQcNXgEtmmQ6DlRrC1hrSrBnCZPokRrDHjw==",
+ "dev": true,
+ "license": "MIT",
+ "peer": true
+ },
"node_modules/unicode-canonical-property-names-ecmascript": {
"version": "2.0.0",
"resolved": "https://registry.npmjs.org/unicode-canonical-property-names-ecmascript/-/unicode-canonical-property-names-ecmascript-2.0.0.tgz",
@@ -17662,14 +17425,15 @@
}
},
"node_modules/vite": {
- "version": "4.5.3",
- "resolved": "https://registry.npmjs.org/vite/-/vite-4.5.3.tgz",
- "integrity": "sha512-kQL23kMeX92v3ph7IauVkXkikdDRsYMGTVl5KY2E9OY4ONLvkHf04MDTbnfo6NKxZiDLWzVpP5oTa8hQD8U3dg==",
+ "version": "4.5.5",
+ "resolved": "https://registry.npmjs.org/vite/-/vite-4.5.5.tgz",
+ "integrity": "sha512-ifW3Lb2sMdX+WU91s3R0FyQlAyLxOzCSCP37ujw0+r5POeHPwe6udWVIElKQq8gk3t7b8rkmvqC6IHBpCff4GQ==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"esbuild": "^0.18.10",
"postcss": "^8.4.27",
- "rollup": "^3.27.1"
+ "rollup": "^3.29.5"
},
"bin": {
"vite": "bin/vite.js"
@@ -17727,7 +17491,7 @@
"mlly": "^1.1.0",
"pathe": "^1.1.0",
"picocolors": "^1.0.0",
- "vite": "^3.0.0 || ^4.0.0"
+ "vite": "^4.5.4"
},
"bin": {
"vite-node": "vite-node.mjs"
@@ -17747,11 +17511,11 @@
"dependencies": {
"@rollup/pluginutils": "^4.2.1",
"@types/eslint": "^8.4.5",
- "rollup": "^2.77.2"
+ "rollup": "^3.29.5"
},
"peerDependencies": {
"eslint": ">=7",
- "vite": ">=2"
+ "vite": ">=4.5.4"
}
},
"node_modules/vite-plugin-eslint/node_modules/@rollup/pluginutils": {
@@ -17767,21 +17531,6 @@
"node": ">= 8.0.0"
}
},
- "node_modules/vite-plugin-eslint/node_modules/rollup": {
- "version": "2.79.1",
- "resolved": "https://registry.npmjs.org/rollup/-/rollup-2.79.1.tgz",
- "integrity": "sha512-uKxbd0IhMZOhjAiD5oAFp7BqvkA4Dv47qpOCtaNvng4HBwdbWtdOh8f5nZNuk2rp51PMGk3bzfWu5oayNEuYnw==",
- "dev": true,
- "bin": {
- "rollup": "dist/bin/rollup"
- },
- "engines": {
- "node": ">=10.0.0"
- },
- "optionalDependencies": {
- "fsevents": "~2.3.2"
- }
- },
"node_modules/vite-plugin-svgr": {
"version": "2.4.0",
"resolved": "https://registry.npmjs.org/vite-plugin-svgr/-/vite-plugin-svgr-2.4.0.tgz",
@@ -17792,7 +17541,7 @@
"@svgr/core": "^6.5.1"
},
"peerDependencies": {
- "vite": "^2.6.0 || 3 || 4"
+ "vite": "^4.5.4"
}
},
"node_modules/vitest": {
@@ -17822,7 +17571,7 @@
"tinybench": "^2.3.1",
"tinypool": "^0.4.0",
"tinyspy": "^1.0.2",
- "vite": "^3.0.0 || ^4.0.0",
+ "vite": "^4.5.4",
"vite-node": "0.29.8",
"why-is-node-running": "^2.2.2"
},
@@ -17884,15 +17633,6 @@
"node": ">=14"
}
},
- "node_modules/wcwidth": {
- "version": "1.0.1",
- "resolved": "https://registry.npmjs.org/wcwidth/-/wcwidth-1.0.1.tgz",
- "integrity": "sha512-XHPEwS0q6TaxcvG85+8EYkbiCux2XtWG2mkc47Ng2A77BQu9+DqIOJldST4HgPkuea7dvKSj5VgX3P1d4rW8Tg==",
- "dev": true,
- "dependencies": {
- "defaults": "^1.0.3"
- }
- },
"node_modules/web-vitals": {
"version": "3.5.2",
"resolved": "https://registry.npmjs.org/web-vitals/-/web-vitals-3.5.2.tgz",
@@ -18136,6 +17876,7 @@
"resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-6.2.0.tgz",
"integrity": "sha512-r6lPcBGxZXlIcymEu7InxDMhdW0KDxpLgoFLcguasxCaJ/SOIZwINatK9KY/tf+ZrlywOKU0UDj3ATXUBfxJXA==",
"dev": true,
+ "license": "MIT",
"dependencies": {
"ansi-styles": "^4.0.0",
"string-width": "^4.1.0",
@@ -18295,6 +18036,19 @@
"url": "https://github.com/sponsors/sindresorhus"
}
},
+ "node_modules/yoctocolors-cjs": {
+ "version": "2.1.2",
+ "resolved": "https://registry.npmjs.org/yoctocolors-cjs/-/yoctocolors-cjs-2.1.2.tgz",
+ "integrity": "sha512-cYVsTjKl8b+FrnidjibDWskAv7UKOfcwaVZdp/it9n1s9fU3IkgDbhdIRKCW4JDsAlECJY0ytoVPT3sK6kideA==",
+ "dev": true,
+ "license": "MIT",
+ "engines": {
+ "node": ">=18"
+ },
+ "funding": {
+ "url": "https://github.com/sponsors/sindresorhus"
+ }
+ },
"node_modules/zustand": {
"version": "3.7.2",
"resolved": "https://registry.npmjs.org/zustand/-/zustand-3.7.2.tgz",
diff --git a/web/vtadmin/package.json b/web/vtadmin/package.json
index 52029ddd9e4..5d4d5dc787a 100644
--- a/web/vtadmin/package.json
+++ b/web/vtadmin/package.json
@@ -20,6 +20,7 @@
"highcharts-react-official": "^3.1.0",
"history": "^5.3.0",
"lodash-es": "^4.17.21",
+ "path-to-regexp": "^8.1.0",
"postcss-flexbugs-fixes": "^5.0.2",
"postcss-preset-env": "^8.0.1",
"query-string": "^7.1.3",
@@ -88,7 +89,7 @@
"eslint-config-react-app": "^7.0.1",
"i": "^0.3.7",
"jsdom": "^21.1.1",
- "msw": "^0.36.8",
+ "msw": "^2.5.2",
"npm": "^10.8.0",
"postcss": "^8.4.31",
"prettier": "^2.2.1",
diff --git a/web/vtadmin/src/api/http.test.ts b/web/vtadmin/src/api/http.test.ts
index 140b1a74c70..8da5d91c55d 100644
--- a/web/vtadmin/src/api/http.test.ts
+++ b/web/vtadmin/src/api/http.test.ts
@@ -13,7 +13,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-import { rest } from 'msw';
+import { http, HttpResponse } from 'msw';
import * as api from './http';
import {
@@ -32,7 +32,7 @@ vi.mock('../errors/errorHandler');
// for all requests made against the given `endpoint`.
const mockServerJson = (endpoint: string, json: object, status: number = 200) => {
const apiAddr = import.meta.env.VITE_VTADMIN_API_ADDRESS;
- global.server.use(rest.get(`${apiAddr}${endpoint}`, (req, res, ctx) => res(ctx.status(status), ctx.json(json))));
+ global.server.use(http.get(`${apiAddr}${endpoint}`, (info) => HttpResponse.json(json, { status: status })));
};
describe('api/http', () => {
@@ -80,8 +80,10 @@ describe('api/http', () => {
errorHandler.notify.mockReset();
const endpoint = `/api/tablets`;
global.server.use(
- rest.get(`${import.meta.env.VITE_VTADMIN_API_ADDRESS}${endpoint}`, (req, res, ctx) =>
- res(ctx.status(504), ctx.body('504 Gateway Time-out '))
+ http.get(`${import.meta.env.VITE_VTADMIN_API_ADDRESS}${endpoint}`, (info) =>
+ HttpResponse.html('504 Gateway Time-out ', {
+ status: 504,
+ })
)
);
@@ -94,7 +96,7 @@ describe('api/http', () => {
/* eslint-disable jest/no-conditional-expect */
expect(e.name).toEqual(MALFORMED_HTTP_RESPONSE_ERROR);
expect(e.message).toContain(
- '[status 504] /api/tablets: invalid json response body at http://test-api.com/api/tablets'
+ `[status 504] /api/tablets: Unexpected token '<', " {
+ const { result } = await vtfetch(`/api/transaction/${clusterID}/${dtid}/info`);
+
+ const err = vtctldata.GetTransactionInfoResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.GetTransactionInfoResponse.create(result);
+};
+
+export interface FetchTransactionsParams {
+ clusterID: string;
+ keyspace: string;
+ abandonAge?: string;
+}
+
+export const fetchTransactions = async ({ clusterID, keyspace, abandonAge = '' }: FetchTransactionsParams) => {
+ const req = new URLSearchParams();
+ req.append('abandon_age', abandonAge);
+
+ const { result } = await vtfetch(`/api/transactions/${clusterID}/${keyspace}?${req}`);
+
+ const err = vtctldata.GetUnresolvedTransactionsResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.GetUnresolvedTransactionsResponse.create(result);
+};
+
+export interface ConcludeTransactionParams {
+ clusterID: string;
+ dtid: string;
+}
+
+export const concludeTransaction = async ({ clusterID, dtid }: ConcludeTransactionParams) => {
+ const { result } = await vtfetch(`/api/transaction/${clusterID}/${dtid}/conclude`);
+ const err = vtctldata.ConcludeTransactionResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.ConcludeTransactionResponse.create(result);
+};
+
export const fetchWorkflows = async () => {
const { result } = await vtfetch(`/api/workflows`);
@@ -437,6 +484,142 @@ export const fetchWorkflow = async (params: { clusterID: string; keyspace: strin
return pb.Workflow.create(result);
};
+export const fetchWorkflowStatus = async (params: { clusterID: string; keyspace: string; name: string }) => {
+ const { result } = await vtfetch(`/api/workflow/${params.clusterID}/${params.keyspace}/${params.name}/status`);
+
+ const err = vtctldata.WorkflowStatusResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowStatusResponse.create(result);
+};
+
+export interface CreateMaterializeParams {
+ clusterID: string;
+ tableSettings: string;
+ request: vtctldata.IMaterializeCreateRequest;
+}
+
+export const createMaterialize = async ({ clusterID, tableSettings, request }: CreateMaterializeParams) => {
+ const body = {
+ table_settings: tableSettings,
+ request: request,
+ };
+
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/materialize`, {
+ body: JSON.stringify(body),
+ method: 'post',
+ });
+
+ const err = vtctldata.MaterializeCreateResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.MaterializeCreateResponse.create(result);
+};
+
+export interface CreateMoveTablesParams {
+ clusterID: string;
+ request: vtctldata.IMoveTablesCreateRequest;
+}
+
+export const createMoveTables = async ({ clusterID, request }: CreateMoveTablesParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/movetables`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+
+ const err = vtctldata.WorkflowStatusResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowStatusResponse.create(result);
+};
+
+export interface CreateReshardParams {
+ clusterID: string;
+ request: vtctldata.IReshardCreateRequest;
+}
+
+export const createReshard = async ({ clusterID, request }: CreateReshardParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/reshard`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+
+ const err = vtctldata.WorkflowStatusResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowStatusResponse.create(result);
+};
+
+export interface WorkflowActionParams {
+ clusterID: string;
+ keyspace: string;
+ name: string;
+}
+
+export const startWorkflow = async ({ clusterID, keyspace, name }: WorkflowActionParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/${keyspace}/${name}/start`);
+ const err = vtctldata.WorkflowUpdateResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowUpdateResponse.create(result);
+};
+
+export const stopWorkflow = async ({ clusterID, keyspace, name }: WorkflowActionParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/${keyspace}/${name}/stop`);
+ const err = vtctldata.WorkflowUpdateResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowUpdateResponse.create(result);
+};
+
+export interface MoveTablesCompleteParams {
+ clusterID: string;
+ request: vtctldata.IMoveTablesCompleteRequest;
+}
+
+export const completeMoveTables = async ({ clusterID, request }: MoveTablesCompleteParams) => {
+ const { result } = await vtfetch(`/api/movetables/${clusterID}/complete`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+ const err = vtctldata.MoveTablesCompleteResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.MoveTablesCompleteResponse.create(result);
+};
+
+export interface WorkflowSwitchTrafficParams {
+ clusterID: string;
+ request: vtctldata.IWorkflowSwitchTrafficRequest;
+}
+
+export const workflowSwitchTraffic = async ({ clusterID, request }: WorkflowSwitchTrafficParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/switchtraffic`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+ const err = vtctldata.WorkflowSwitchTrafficResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowSwitchTrafficResponse.create(result);
+};
+
+export interface WorkflowDeleteParams {
+ clusterID: string;
+ request: vtctldata.IWorkflowDeleteRequest;
+}
+
+export const workflowDelete = async ({ clusterID, request }: WorkflowDeleteParams) => {
+ const { result } = await vtfetch(`/api/workflow/${clusterID}/delete`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+ const err = vtctldata.WorkflowDeleteResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.WorkflowDeleteResponse.create(result);
+};
+
export const fetchVTExplain = async ({ cluster, keyspace, sql }: R) => {
// As an easy enhancement for later, we can also validate the request parameters on the front-end
// instead of defaulting to '', to save a round trip.
@@ -553,7 +736,9 @@ export const reloadSchema = async (params: ReloadSchemaParams) => {
req.append('wait_position', params.waitPosition);
}
- const { result } = await vtfetch(`/api/schemas/reload?${req}`, { method: 'put' });
+ const { result } = await vtfetch(`/api/schemas/reload?${req}`, {
+ method: 'put',
+ });
const err = pb.ReloadSchemasResponse.verify(result);
if (err) throw Error(err);
@@ -574,7 +759,9 @@ export const deleteShard = async (params: DeleteShardParams) => {
req.append('even_if_serving', String(params.evenIfServing));
req.append('recursive', String(params.recursive));
- const { result } = await vtfetch(`/api/shards/${params.clusterID}?${req}`, { method: 'delete' });
+ const { result } = await vtfetch(`/api/shards/${params.clusterID}?${req}`, {
+ method: 'delete',
+ });
const err = vtctldata.DeleteShardsResponse.verify(result);
if (err) throw Error(err);
@@ -701,7 +888,10 @@ export interface RebuildKeyspaceGraphParams {
export const rebuildKeyspaceGraph = async (params: RebuildKeyspaceGraphParams) => {
const { result } = await vtfetch(`/api/keyspace/${params.clusterID}/${params.keyspace}/rebuild_keyspace_graph`, {
method: 'put',
- body: JSON.stringify({ cells: params.cells, allow_partial: params.allowPartial }),
+ body: JSON.stringify({
+ cells: params.cells,
+ allow_partial: params.allowPartial,
+ }),
});
const err = pb.RebuildKeyspaceGraphRequest.verify(result);
if (err) throw Error(err);
@@ -720,7 +910,11 @@ export interface RemoveKeyspaceCellParams {
export const removeKeyspaceCell = async (params: RemoveKeyspaceCellParams) => {
const { result } = await vtfetch(`/api/keyspace/${params.clusterID}/${params.keyspace}/remove_keyspace_cell`, {
method: 'put',
- body: JSON.stringify({ cell: params.cell, force: params.force, recursive: params.recursive }),
+ body: JSON.stringify({
+ cell: params.cell,
+ force: params.force,
+ recursive: params.recursive,
+ }),
});
const err = pb.RemoveKeyspaceCellRequest.verify(result);
if (err) throw Error(err);
@@ -840,3 +1034,75 @@ export const validateVersionShard = async (params: ValidateVersionShardParams) =
return vtctldata.ValidateVersionShardResponse.create(result);
};
+
+export interface CreateVDiffParams {
+ clusterID: string;
+ request: vtctldata.IVDiffCreateRequest;
+}
+
+export const createVDiff = async ({ clusterID, request }: CreateVDiffParams) => {
+ const { result } = await vtfetch(`/api/vdiff/${clusterID}/`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+
+ const err = vtctldata.VDiffCreateResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.VDiffCreateResponse.create(result);
+};
+
+export interface ShowVDiffParams {
+ clusterID: string;
+ request: vtctldata.IVDiffShowRequest;
+}
+
+export const showVDiff = async ({ clusterID, request }: ShowVDiffParams) => {
+ const { result } = await vtfetch(`/api/vdiff/${clusterID}/show`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+
+ const err = vtadmin.VDiffShowResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtadmin.VDiffShowResponse.create(result);
+};
+
+export const fetchSchemaMigrations = async (request: vtadmin.IGetSchemaMigrationsRequest) => {
+ const { result } = await vtfetch(`/api/migrations/`, {
+ body: JSON.stringify(request),
+ method: 'post',
+ });
+
+ const err = vtadmin.GetSchemaMigrationsResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtadmin.GetSchemaMigrationsResponse.create(result);
+};
+
+export interface ApplySchemaParams {
+ clusterID: string;
+ keyspace: string;
+ callerID: string;
+ sql: string;
+ request: vtctldata.IApplySchemaRequest;
+}
+
+export const applySchema = async ({ clusterID, keyspace, callerID, sql, request }: ApplySchemaParams) => {
+ const body = {
+ sql,
+ caller_id: callerID,
+ request,
+ };
+
+ const { result } = await vtfetch(`/api/migration/${clusterID}/${keyspace}`, {
+ body: JSON.stringify(body),
+ method: 'post',
+ });
+
+ const err = vtctldata.ApplySchemaResponse.verify(result);
+ if (err) throw Error(err);
+
+ return vtctldata.ApplySchemaResponse.create(result);
+};
diff --git a/web/vtadmin/src/components/ActionPanel.test.tsx b/web/vtadmin/src/components/ActionPanel.test.tsx
index 6d0bc02b30f..6f1bed60603 100644
--- a/web/vtadmin/src/components/ActionPanel.test.tsx
+++ b/web/vtadmin/src/components/ActionPanel.test.tsx
@@ -16,7 +16,7 @@
import { render, screen, waitFor } from '@testing-library/react';
import userEvent from '@testing-library/user-event';
-import { rest } from 'msw';
+import { delay, http, HttpResponse } from 'msw';
import { QueryClient, QueryClientProvider, useMutation } from 'react-query';
import { describe, it, expect, vi } from 'vitest';
@@ -33,11 +33,7 @@ describe('ActionPanel', () => {
* provides such a function and should be `render`ed in the context QueryClientProvider.
*/
const Wrapper: React.FC = (props) => {
- const mutation = useMutation(() => fetch(new URL(props['url']), { method: 'post' }), {
- onError: (error) => {
- console.log('ERROR: ', error);
- },
- });
+ const mutation = useMutation(() => fetch(new URL(props['url']), { method: 'post' }));
return ;
};
@@ -46,8 +42,9 @@ describe('ActionPanel', () => {
const url = `${import.meta.env.VITE_VTADMIN_API_ADDRESS}/api/test`;
global.server.use(
- rest.post(url, (req, res, ctx) => {
- return res(ctx.json({ ok: true }));
+ http.post(url, async (info) => {
+ await delay();
+ return HttpResponse.json({ ok: true });
})
);
diff --git a/web/vtadmin/src/components/App.tsx b/web/vtadmin/src/components/App.tsx
index f7821f3a4ad..fd0f772ae19 100644
--- a/web/vtadmin/src/components/App.tsx
+++ b/web/vtadmin/src/components/App.tsx
@@ -40,6 +40,14 @@ import { isReadOnlyMode } from '../util/env';
import { CreateKeyspace } from './routes/createKeyspace/CreateKeyspace';
import { Topology } from './routes/topology/Topology';
import { ClusterTopology } from './routes/topology/ClusterTopology';
+import { CreateMoveTables } from './routes/createWorkflow/CreateMoveTables';
+import { Transactions } from './routes/Transactions';
+import { Transaction } from './routes/transaction/Transaction';
+import { CreateReshard } from './routes/createWorkflow/CreateReshard';
+import { CreateMaterialize } from './routes/createWorkflow/CreateMaterialize';
+import { TopologyTree } from './routes/topologyTree/TopologyTree';
+import { SchemaMigrations } from './routes/SchemaMigrations';
+import { CreateSchemaMigration } from './routes/createSchemaMigration/CreateSchemaMigration';
export const App = () => {
return (
@@ -105,10 +113,28 @@ export const App = () => {
-
+
+ {!isReadOnlyMode() && (
+
+
+
+ )}
+
+ {!isReadOnlyMode() && (
+
+
+
+ )}
+
+ {!isReadOnlyMode() && (
+
+
+
+ )}
+
@@ -117,10 +143,32 @@ export const App = () => {
+
+
+
+
+ {!isReadOnlyMode() && (
+
+
+
+ )}
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/web/vtadmin/src/components/NavRail.tsx b/web/vtadmin/src/components/NavRail.tsx
index e2897395760..b30cd165684 100644
--- a/web/vtadmin/src/components/NavRail.tsx
+++ b/web/vtadmin/src/components/NavRail.tsx
@@ -65,6 +65,12 @@ export const NavRail = () => {
+
+
+
+
+
+
diff --git a/web/vtadmin/src/components/TextInput.module.scss b/web/vtadmin/src/components/TextInput.module.scss
index cb087f68956..8611985108c 100644
--- a/web/vtadmin/src/components/TextInput.module.scss
+++ b/web/vtadmin/src/components/TextInput.module.scss
@@ -30,6 +30,7 @@ $iconPositionHorizontalLarge: 1.6rem;
&:disabled {
background: var(--backgroundSecondary);
border-color: var(--backgroundSecondaryHighlight);
+ color: var(--colorDisabled);
cursor: not-allowed;
}
diff --git a/web/vtadmin/src/components/dataTable/DataTable.tsx b/web/vtadmin/src/components/dataTable/DataTable.tsx
index ed14067bd34..9e863a6aa54 100644
--- a/web/vtadmin/src/components/dataTable/DataTable.tsx
+++ b/web/vtadmin/src/components/dataTable/DataTable.tsx
@@ -31,6 +31,10 @@ interface Props {
pageSize?: number;
renderRows: (rows: T[]) => JSX.Element[];
title?: string;
+ // Pass a unique `pageKey` for each DataTable, in case multiple
+ // DataTables access the same URL. This will be used to
+ // access page number from the URL.
+ pageKey?: string;
}
// Generally, page sizes of ~100 rows are fine in terms of performance,
@@ -43,12 +47,15 @@ export const DataTable = ({
pageSize = DEFAULT_PAGE_SIZE,
renderRows,
title,
+ pageKey = '',
}: Props) => {
const { pathname } = useLocation();
const urlQuery = useURLQuery();
+ const pageQueryKey = `${pageKey}page`;
+
const totalPages = Math.ceil(data.length / pageSize);
- const { page } = useURLPagination({ totalPages });
+ const { page } = useURLPagination({ totalPages, pageQueryKey });
const startIndex = (page - 1) * pageSize;
const endIndex = startIndex + pageSize;
@@ -59,7 +66,7 @@ export const DataTable = ({
const formatPageLink = (p: number) => ({
pathname,
- search: stringify({ ...urlQuery.query, page: p === 1 ? undefined : p }),
+ search: stringify({ ...urlQuery.query, [pageQueryKey]: p === 1 ? undefined : p }),
});
return (
diff --git a/web/vtadmin/src/components/dialog/ErrorDialog.tsx b/web/vtadmin/src/components/dialog/ErrorDialog.tsx
new file mode 100644
index 00000000000..087876e4cd2
--- /dev/null
+++ b/web/vtadmin/src/components/dialog/ErrorDialog.tsx
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+import React from 'react';
+import Dialog from './Dialog';
+import { Icon, Icons } from '../Icon';
+
+export interface ErrorDialogProps {
+ errorTitle?: string;
+ errorDescription: string;
+ isOpen: boolean;
+ onClose: () => void;
+}
+
+const ErrorDialog: React.FC = ({ errorTitle, errorDescription, isOpen, onClose }) => {
+ return (
+
+
+
+
+
+
{errorTitle || 'Error'}
+
{errorDescription}
+
+
+ );
+};
+
+export default ErrorDialog;
diff --git a/web/vtadmin/src/components/dropdown/Dropdown.tsx b/web/vtadmin/src/components/dropdown/Dropdown.tsx
index c6f7e1285c3..a0db3cdd313 100644
--- a/web/vtadmin/src/components/dropdown/Dropdown.tsx
+++ b/web/vtadmin/src/components/dropdown/Dropdown.tsx
@@ -2,10 +2,14 @@ import React, { Fragment } from 'react';
import { Icon, Icons } from '../Icon';
import { Menu, Transition } from '@headlessui/react';
import style from './Dropdown.module.scss';
+import cx from 'classnames';
+
interface DropdownProps {
// Optionally pass in your own button if you don't want it styled like DropdownButton
dropdownButton: React.FC | Icons;
position?: 'top-left' | 'top-right' | 'bottom-right' | 'bottom-left';
+ title?: string;
+ className?: string;
}
const positions: Record = {
@@ -16,14 +20,21 @@ const positions: Record = {
default: 'right-0',
};
-export const DropdownButton: React.FC<{ icon: Icons }> = ({ icon }) => (
+export const DropdownButton: React.FC<{ icon: Icons; title?: string; className?: string }> = ({
+ icon,
+ title,
+ className,
+}) => (